You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Jason Gustafson <ja...@confluent.io.INVALID> on 2023/01/06 01:04:52 UTC

Re: [DISCUSS] KIP-890 Server Side Defense

Hi Justine,

Thanks for the proposal.

I was thinking about the implementation a little bit. In the current
proposal, the behavior depends on whether we have an old or new client. For
old clients, we send `DescribeTransactions` and verify the result and for
new clients, we send `AddPartitionsToTxn`. We might be able to simplify the
implementation if we can use the same request type. For example, what if we
bump the protocol version for `AddPartitionsToTxn` and add a `validateOnly`
flag? For older versions, we can set `validateOnly=true` so that the
request only returns successfully if the partition had already been added.
For new versions, we can set `validateOnly=false` and the partition will be
added to the transaction. The other slightly annoying thing that this would
get around is the need to collect the transaction state for all partitions
even when we only care about a subset.

Some additional improvements to consider:

- We can give `AddPartitionsToTxn` better batch support for inter-broker
usage. Currently we only allow one `TransactionalId` to be specified, but
the broker may get some benefit being able to batch across multiple
transactions.
- Another small improvement is skipping topic authorization checks for
`AddPartitionsToTxn` when the request is from a broker. Perhaps we can add
a field for the `LeaderId` or something like that and require CLUSTER
permission when set.

Best,
Jason



On Mon, Dec 19, 2022 at 3:56 PM Jun Rao <ju...@confluent.io.invalid> wrote:

> Hi, Justine,
>
> Thanks for the explanation. It makes sense to me now.
>
> Jun
>
> On Mon, Dec 19, 2022 at 1:42 PM Justine Olshan
> <jo...@confluent.io.invalid>
> wrote:
>
> > Hi Jun,
> >
> > My understanding of the mechanism is that when we get to the last epoch,
> we
> > increment to the fencing/last epoch and if any further requests come in
> for
> > this producer ID they are fenced. Then the producer gets a new ID and
> > restarts with epoch/sequence 0. The fenced epoch sticks around for the
> > duration of producer.id.expiration.ms and blocks any late messages
> there.
> > The new ID will get to take advantage of the improved semantics around
> > non-zero start sequences. So I think we are covered.
> >
> > The only potential issue is overloading the cache, but hopefully the
> > improvements (lowered producer.id.expiration.ms) will help with that.
> Let
> > me know if you still have concerns.
> >
> > Thanks,
> > Justine
> >
> > On Mon, Dec 19, 2022 at 10:24 AM Jun Rao <ju...@confluent.io.invalid>
> wrote:
> >
> > > Hi, Justine,
> > >
> > > Thanks for the explanation.
> > >
> > > 70. The proposed fencing logic doesn't apply when pid changes, is that
> > > right? If so, I am not sure how complete we are addressing this issue
> if
> > > the pid changes more frequently.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > >
> > > On Fri, Dec 16, 2022 at 9:16 AM Justine Olshan
> > > <jo...@confluent.io.invalid>
> > > wrote:
> > >
> > > > Hi Jun,
> > > >
> > > > Thanks for replying!
> > > >
> > > > 70.We already do the overflow mechanism, so my change would just make
> > it
> > > > happen more often.
> > > > I was also not suggesting a new field in the log, but in the
> response,
> > > > which would be gated by the client version. Sorry if something there
> is
> > > > unclear. I think we are starting to diverge.
> > > > The goal of this KIP is to not change to the marker format at all.
> > > >
> > > > 71. Yes, I guess I was going under the assumption that the log would
> > just
> > > > look at its last epoch and treat it as the current epoch. I suppose
> we
> > > can
> > > > have some special logic that if the last epoch was on a marker we
> > > actually
> > > > expect the next epoch or something like that. We just need to
> > distinguish
> > > > based on whether we had a commit/abort marker.
> > > >
> > > > 72.
> > > > > if the producer epoch hasn't been bumped on the
> > > > broker, it seems that the stucked message will fail the sequence
> > > validation
> > > > and will be ignored. If the producer epoch has been bumped, we ignore
> > the
> > > > sequence check and the stuck message could be appended to the log.
> So,
> > is
> > > > the latter case that we want to guard?
> > > >
> > > > I'm not sure I follow that "the message will fail the sequence
> > > validation".
> > > > In some of these cases, we had an abort marker (due to an error) and
> > then
> > > > the late message comes in with the correct sequence number. This is a
> > > case
> > > > covered by the KIP.
> > > > The latter case is actually not something we've considered here. I
> > think
> > > > generally when we bump the epoch, we are accepting that the sequence
> > does
> > > > not need to be checked anymore. My understanding is also that we
> don't
> > > > typically bump epoch mid transaction (based on a quick look at the
> > code)
> > > > but let me know if that is the case.
> > > >
> > > > Thanks,
> > > > Justine
> > > >
> > > > On Thu, Dec 15, 2022 at 12:23 PM Jun Rao <ju...@confluent.io.invalid>
> > > wrote:
> > > >
> > > > > Hi, Justine,
> > > > >
> > > > > Thanks for the reply.
> > > > >
> > > > > 70. Assigning a new pid on int overflow seems a bit hacky. If we
> > need a
> > > > txn
> > > > > level id, it will be better to model this explicitly. Adding a new
> > > field
> > > > > would require a bit more work since it requires a new txn marker
> > format
> > > > in
> > > > > the log. So, we probably need to guard it with an IBP or metadata
> > > version
> > > > > and document the impact on downgrade once the new format is written
> > to
> > > > the
> > > > > log.
> > > > >
> > > > > 71. Hmm, once the marker is written, the partition will expect the
> > next
> > > > > append to be on the next epoch. Does that cover the case you
> > mentioned?
> > > > >
> > > > > 72. Also, just to be clear on the stucked message issue described
> in
> > > the
> > > > > motivation. With EoS, we also validate the sequence id for
> > idempotency.
> > > > So,
> > > > > with the current logic, if the producer epoch hasn't been bumped on
> > the
> > > > > broker, it seems that the stucked message will fail the sequence
> > > > validation
> > > > > and will be ignored. If the producer epoch has been bumped, we
> ignore
> > > the
> > > > > sequence check and the stuck message could be appended to the log.
> > So,
> > > is
> > > > > the latter case that we want to guard?
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > > On Wed, Dec 14, 2022 at 10:44 AM Justine Olshan
> > > > > <jo...@confluent.io.invalid> wrote:
> > > > >
> > > > > > Matthias — thanks again for taking time to look a this. You said:
> > > > > >
> > > > > > > My proposal was only focusing to avoid dangling
> > > > > >
> > > > > > transactions if records are added without registered partition.
> --
> > > > Maybe
> > > > > >
> > > > > > you can add a few more details to the KIP about this scenario for
> > > > better
> > > > > >
> > > > > > documentation purpose?
> > > > > >
> > > > > >
> > > > > > I'm not sure I understand what you mean here. The motivation
> > section
> > > > > > describes two scenarios about how the record can be added
> without a
> > > > > > registered partition:
> > > > > >
> > > > > >
> > > > > > > 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.
> > > > > >
> > > > > >
> > > > > > > Another way hanging transactions can occur is that a client is
> > > buggy
> > > > > and
> > > > > > may somehow try to write to a partition before it adds the
> > partition
> > > to
> > > > > the
> > > > > > transaction.
> > > > > >
> > > > > >
> > > > > >
> > > > > > For the first example of this would it be helpful to say that
> this
> > > > > message
> > > > > > comes in after the abort, but before the partition is added to
> the
> > > next
> > > > > > transaction so it becomes "hanging." Perhaps the next sentence
> > > > describing
> > > > > > the message becoming part of the next transaction (a different
> > case)
> > > > was
> > > > > > not properly differentiated.
> > > > > >
> > > > > >
> > > > > >
> > > > > > Jun — thanks for reading the KIP.
> > > > > >
> > > > > > 70. The int typing was a concern. Currently we have a mechanism
> in
> > > > place
> > > > > to
> > > > > > fence the final epoch when the epoch is about to overflow and
> > assign
> > > a
> > > > > new
> > > > > > producer ID with epoch 0. Of course, this is a bit tricky when it
> > > comes
> > > > > to
> > > > > > the response back to the client.
> > > > > > Making this a long could be another option, but I wonder are
> there
> > > any
> > > > > > implications on changing this field if the epoch is persisted to
> > > disk?
> > > > > I'd
> > > > > > need to check the usages.
> > > > > >
> > > > > > 71.This was something Matthias asked about as well. I was
> > > considering a
> > > > > > possible edge case where a produce request from a new transaction
> > > > somehow
> > > > > > gets sent right after the marker is written, but before the
> > producer
> > > is
> > > > > > alerted of the newly bumped epoch. In this case, we may include
> > this
> > > > > record
> > > > > > when we don't want to. I suppose we could try to do something
> > client
> > > > side
> > > > > > to bump the epoch after sending an endTxn as well in this
> scenario
> > —
> > > > but
> > > > > I
> > > > > > wonder how it would work when the server is aborting based on a
> > > > > server-side
> > > > > > error. I could also be missing something and this scenario is
> > > actually
> > > > > not
> > > > > > possible.
> > > > > >
> > > > > > Thanks again to everyone reading and commenting. Let me know
> about
> > > any
> > > > > > further questions or comments.
> > > > > >
> > > > > > Justine
> > > > > >
> > > > > > On Wed, Dec 14, 2022 at 9:41 AM Jun Rao <jun@confluent.io.invalid
> >
> > > > > wrote:
> > > > > >
> > > > > > > Hi, Justine,
> > > > > > >
> > > > > > > Thanks for the KIP. A couple of comments.
> > > > > > >
> > > > > > > 70. Currently, the producer epoch is an int. I am not sure if
> > it's
> > > > > enough
> > > > > > > to accommodate all transactions in the lifetime of a producer.
> > > Should
> > > > > we
> > > > > > > change that to a long or add a new long field like txnId?
> > > > > > >
> > > > > > > 71. "it will write the prepare commit message with a bumped
> epoch
> > > and
> > > > > > send
> > > > > > > WriteTxnMarkerRequests with the bumped epoch." Hmm, the epoch
> is
> > > > > > associated
> > > > > > > with the current txn right? So, it seems weird to write a
> commit
> > > > > message
> > > > > > > with a bumped epoch. Should we only bump up the epoch in
> > > > EndTxnResponse
> > > > > > and
> > > > > > > rename the field to sth like nextProducerEpoch?
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Mon, Dec 12, 2022 at 8:54 PM Matthias J. Sax <
> > mjsax@apache.org>
> > > > > > wrote:
> > > > > > >
> > > > > > > > Thanks for the background.
> > > > > > > >
> > > > > > > > 20/30: SGTM. My proposal was only focusing to avoid dangling
> > > > > > > > transactions if records are added without registered
> partition.
> > > --
> > > > > > Maybe
> > > > > > > > you can add a few more details to the KIP about this scenario
> > for
> > > > > > better
> > > > > > > > documentation purpose?
> > > > > > > >
> > > > > > > > 40: I think you hit a fair point about race conditions or
> > client
> > > > bugs
> > > > > > > > (incorrectly not bumping the epoch). The complexity/confusion
> > for
> > > > > using
> > > > > > > > the bumped epoch I see, is mainly for internal debugging, ie,
> > > > > > inspecting
> > > > > > > > log segment dumps -- it seems harder to reason about the
> system
> > > for
> > > > > us
> > > > > > > > humans. But if we get better guarantees, it would be worth to
> > use
> > > > the
> > > > > > > > bumped epoch.
> > > > > > > >
> > > > > > > > 60: as I mentioned already, I don't know the broker internals
> > to
> > > > > > provide
> > > > > > > > more input. So if nobody else chimes in, we should just move
> > > > forward
> > > > > > > > with your proposal.
> > > > > > > >
> > > > > > > >
> > > > > > > > -Matthias
> > > > > > > >
> > > > > > > >
> > > > > > > > On 12/6/22 4:22 PM, Justine Olshan wrote:
> > > > > > > > > Hi all,
> > > > > > > > > After Artem's questions about error behavior, I've
> > re-evaluated
> > > > the
> > > > > > > > > unknown producer ID exception and had some discussions
> > offline.
> > > > > > > > >
> > > > > > > > > I think generally it makes sense to simplify error handling
> > in
> > > > > cases
> > > > > > > like
> > > > > > > > > this and the UNKNOWN_PRODUCER_ID error has a pretty long
> and
> > > > > > > complicated
> > > > > > > > > history. Because of this, I propose adding a new error code
> > > > > > > > ABORTABLE_ERROR
> > > > > > > > > that when encountered by new clients (gated by the produce
> > > > request
> > > > > > > > version)
> > > > > > > > > will simply abort the transaction. This allows the server
> to
> > > have
> > > > > > some
> > > > > > > > say
> > > > > > > > > in whether the client aborts and makes handling much
> simpler.
> > > In
> > > > > the
> > > > > > > > > future, we can also use this error in other situations
> where
> > we
> > > > > want
> > > > > > to
> > > > > > > > > abort the transactions. We can even use on other apis.
> > > > > > > > >
> > > > > > > > > I've added this to the KIP. Let me know if there are any
> > > > questions
> > > > > or
> > > > > > > > > issues.
> > > > > > > > >
> > > > > > > > > Justine
> > > > > > > > >
> > > > > > > > > On Fri, Dec 2, 2022 at 10:22 AM Justine Olshan <
> > > > > jolshan@confluent.io
> > > > > > >
> > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > >> Hey Matthias,
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> 20/30 — Maybe I also didn't express myself clearly. For
> > older
> > > > > > clients
> > > > > > > we
> > > > > > > > >> don't have a way to distinguish between a previous and the
> > > > current
> > > > > > > > >> transaction since we don't have the epoch bump. This means
> > > that
> > > > a
> > > > > > late
> > > > > > > > >> message from the previous transaction may be added to the
> > new
> > > > one.
> > > > > > > With
> > > > > > > > >> older clients — we can't guarantee this won't happen if we
> > > > already
> > > > > > > sent
> > > > > > > > the
> > > > > > > > >> addPartitionsToTxn call (why we make changes for the newer
> > > > client)
> > > > > > but
> > > > > > > > we
> > > > > > > > >> can at least gate some by ensuring that the partition has
> > been
> > > > > added
> > > > > > > to
> > > > > > > > the
> > > > > > > > >> transaction. The rationale here is that there are likely
> > LESS
> > > > late
> > > > > > > > arrivals
> > > > > > > > >> as time goes on, so hopefully most late arrivals will come
> > in
> > > > > BEFORE
> > > > > > > the
> > > > > > > > >> addPartitionsToTxn call. Those that arrive before will be
> > > > properly
> > > > > > > gated
> > > > > > > > >> with the describeTransactions approach.
> > > > > > > > >>
> > > > > > > > >> If we take the approach you suggested, ANY late arrival
> > from a
> > > > > > > previous
> > > > > > > > >> transaction will be added. And we don't want that. I also
> > > don't
> > > > > see
> > > > > > > any
> > > > > > > > >> benefit in sending addPartitionsToTxn over the
> describeTxns
> > > > call.
> > > > > > They
> > > > > > > > will
> > > > > > > > >> both be one extra RPC to the Txn coordinator.
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> To be clear — newer clients will use addPartitionsToTxn
> > > instead
> > > > of
> > > > > > the
> > > > > > > > >> DescribeTxns.
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> 40)
> > > > > > > > >> My concern is that if we have some delay in the client to
> > bump
> > > > the
> > > > > > > > epoch,
> > > > > > > > >> it could continue to send epoch 73 and those records would
> > not
> > > > be
> > > > > > > > fenced.
> > > > > > > > >> Perhaps this is not an issue if we don't allow the next
> > > produce
> > > > to
> > > > > > go
> > > > > > > > >> through before the EndTxn request returns. I'm also
> thinking
> > > > about
> > > > > > > > cases of
> > > > > > > > >> failure. I will need to think on this a bit.
> > > > > > > > >>
> > > > > > > > >> I wasn't sure if it was that confusing. But if we think it
> > is,
> > > > we
> > > > > > can
> > > > > > > > >> investigate other ways.
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> 60)
> > > > > > > > >>
> > > > > > > > >> I'm not sure these are the same purgatories since one is a
> > > > produce
> > > > > > > > >> purgatory (I was planning on using a callback rather than
> > > > > purgatory)
> > > > > > > and
> > > > > > > > >> the other is simply a request to append to the log. Not
> sure
> > > we
> > > > > have
> > > > > > > any
> > > > > > > > >> structure here for ordering, but my understanding is that
> > the
> > > > > broker
> > > > > > > > could
> > > > > > > > >> handle the write request before it hears back from the Txn
> > > > > > > Coordinator.
> > > > > > > > >>
> > > > > > > > >> Let me know if I misunderstood something or something was
> > > > unclear.
> > > > > > > > >>
> > > > > > > > >> Justine
> > > > > > > > >>
> > > > > > > > >> On Thu, Dec 1, 2022 at 12:15 PM Matthias J. Sax <
> > > > mjsax@apache.org
> > > > > >
> > > > > > > > wrote:
> > > > > > > > >>
> > > > > > > > >>> Thanks for the details Justine!
> > > > > > > > >>>
> > > > > > > > >>>> 20)
> > > > > > > > >>>>
> > > > > > > > >>>> The client side change for 2 is removing the
> addPartitions
> > > to
> > > > > > > > >>> transaction
> > > > > > > > >>>> call. We don't need to make this from the producer to
> the
> > > txn
> > > > > > > > >>> coordinator,
> > > > > > > > >>>> only server side.
> > > > > > > > >>>
> > > > > > > > >>> I think I did not express myself clearly. I understand
> that
> > > we
> > > > > can
> > > > > > > (and
> > > > > > > > >>> should) change the producer to not send the
> `addPartitions`
> > > > > request
> > > > > > > any
> > > > > > > > >>> longer. But I don't thinks it's requirement to change the
> > > > broker?
> > > > > > > > >>>
> > > > > > > > >>> What I am trying to say is: as a safe-guard and
> improvement
> > > for
> > > > > > older
> > > > > > > > >>> producers, the partition leader can just send the
> > > > `addPartitions`
> > > > > > > > >>> request to the TX-coordinator in any case -- if the old
> > > > producer
> > > > > > > > >>> correctly did send the `addPartition` request to the
> > > > > TX-coordinator
> > > > > > > > >>> already, the TX-coordinator can just "ignore" is as
> > > idempotent.
> > > > > > > > However,
> > > > > > > > >>> if the old producer has a bug and did forget to sent the
> > > > > > > `addPartition`
> > > > > > > > >>> request, we would now ensure that the partition is indeed
> > > added
> > > > > to
> > > > > > > the
> > > > > > > > >>> TX and thus fix a potential producer bug (even if we
> don't
> > > get
> > > > > the
> > > > > > > > >>> fencing via the bump epoch). -- It seems to be a good
> > > > > improvement?
> > > > > > Or
> > > > > > > > is
> > > > > > > > >>> there a reason to not do this?
> > > > > > > > >>>
> > > > > > > > >>>
> > > > > > > > >>>
> > > > > > > > >>>> 30)
> > > > > > > > >>>>
> > > > > > > > >>>> Transaction is ongoing = partition was added to
> > transaction
> > > > via
> > > > > > > > >>>> addPartitionsToTxn. We check this with the
> > > > DescribeTransactions
> > > > > > > call.
> > > > > > > > >>> Let
> > > > > > > > >>>> me know if this wasn't sufficiently explained here:
> > > > > > > > >>>
> > > > > > > > >>> If we do what I propose in (20), we don't really need to
> > make
> > > > > this
> > > > > > > > >>> `DescribeTransaction` call, as the partition leader adds
> > the
> > > > > > > partition
> > > > > > > > >>> for older clients and we get this check for free.
> > > > > > > > >>>
> > > > > > > > >>>
> > > > > > > > >>>> 40)
> > > > > > > > >>>>
> > > > > > > > >>>> The idea here is that if any messages somehow come in
> > before
> > > > we
> > > > > > get
> > > > > > > > the
> > > > > > > > >>> new
> > > > > > > > >>>> epoch to the producer, they will be fenced. However, if
> we
> > > > don't
> > > > > > > think
> > > > > > > > >>> this
> > > > > > > > >>>> is necessary, it can be discussed
> > > > > > > > >>>
> > > > > > > > >>> I agree that we should have epoch fencing. My question is
> > > > > > different:
> > > > > > > > >>> Assume we are at epoch 73, and we have an ongoing
> > > transaction,
> > > > > that
> > > > > > > is
> > > > > > > > >>> committed. It seems natural to write the "prepare commit"
> > > > marker
> > > > > > and
> > > > > > > > the
> > > > > > > > >>> `WriteTxMarkerRequest` both with epoch 73, too, as it
> > belongs
> > > > to
> > > > > > the
> > > > > > > > >>> current transaction. Of course, we now also bump the
> epoch
> > > and
> > > > > > expect
> > > > > > > > >>> the next requests to have epoch 74, and would reject an
> > > request
> > > > > > with
> > > > > > > > >>> epoch 73, as the corresponding TX for epoch 73 was
> already
> > > > > > committed.
> > > > > > > > >>>
> > > > > > > > >>> It seems you propose to write the "prepare commit marker"
> > and
> > > > > > > > >>> `WriteTxMarkerRequest` with epoch 74 though, what would
> > work,
> > > > but
> > > > > > it
> > > > > > > > >>> seems confusing. Is there a reason why we would use the
> > > bumped
> > > > > > epoch
> > > > > > > 74
> > > > > > > > >>> instead of the current epoch 73?
> > > > > > > > >>>
> > > > > > > > >>>
> > > > > > > > >>>> 60)
> > > > > > > > >>>>
> > > > > > > > >>>> When we are checking if the transaction is ongoing, we
> > need
> > > to
> > > > > > make
> > > > > > > a
> > > > > > > > >>> round
> > > > > > > > >>>> trip from the leader partition to the transaction
> > > coordinator.
> > > > > In
> > > > > > > the
> > > > > > > > >>> time
> > > > > > > > >>>> we are waiting for this message to come back, in theory
> we
> > > > could
> > > > > > > have
> > > > > > > > >>> sent
> > > > > > > > >>>> a commit/abort call that would make the original result
> of
> > > the
> > > > > > check
> > > > > > > > >>> out of
> > > > > > > > >>>> date. That is why we can check the leader state before
> we
> > > > write
> > > > > to
> > > > > > > the
> > > > > > > > >>> log.
> > > > > > > > >>>
> > > > > > > > >>> Thanks. Got it.
> > > > > > > > >>>
> > > > > > > > >>> However, is this really an issue? We put the produce
> > request
> > > in
> > > > > > > > >>> purgatory, so how could we process the
> > > `WriteTxnMarkerRequest`
> > > > > > first?
> > > > > > > > >>> Don't we need to put the `WriteTxnMarkerRequest` into
> > > > purgatory,
> > > > > > too,
> > > > > > > > >>> for this case, and process both request in-order? (Again,
> > my
> > > > > broker
> > > > > > > > >>> knowledge is limited and maybe we don't maintain request
> > > order
> > > > > for
> > > > > > > this
> > > > > > > > >>> case, what seems to be an issue IMHO, and I am wondering
> if
> > > > > > changing
> > > > > > > > >>> request handling to preserve order for this case might be
> > the
> > > > > > cleaner
> > > > > > > > >>> solution?)
> > > > > > > > >>>
> > > > > > > > >>>
> > > > > > > > >>>
> > > > > > > > >>> -Matthias
> > > > > > > > >>>
> > > > > > > > >>>
> > > > > > > > >>>
> > > > > > > > >>>
> > > > > > > > >>> On 11/30/22 3:28 PM, Artem Livshits wrote:
> > > > > > > > >>>> Hi Justine,
> > > > > > > > >>>>
> > > > > > > > >>>> I think the interesting part is not in this logic
> (because
> > > it
> > > > > > tries
> > > > > > > to
> > > > > > > > >>>> figure out when UNKNOWN_PRODUCER_ID is retriable and if
> > it's
> > > > > > > > retryable,
> > > > > > > > >>>> it's definitely not fatal), but what happens when this
> > logic
> > > > > > doesn't
> > > > > > > > >>> return
> > > > > > > > >>>> 'true' and falls through.  In the old clients it seems
> to
> > be
> > > > > > fatal,
> > > > > > > if
> > > > > > > > >>> we
> > > > > > > > >>>> keep the behavior in the new clients, I'd expect it
> would
> > be
> > > > > fatal
> > > > > > > as
> > > > > > > > >>> well.
> > > > > > > > >>>>
> > > > > > > > >>>> -Artem
> > > > > > > > >>>>
> > > > > > > > >>>> On Tue, Nov 29, 2022 at 11:57 AM Justine Olshan
> > > > > > > > >>>> <jo...@confluent.io.invalid> wrote:
> > > > > > > > >>>>
> > > > > > > > >>>>> Hi Artem and Jeff,
> > > > > > > > >>>>>
> > > > > > > > >>>>>
> > > > > > > > >>>>> Thanks for taking a look and sorry for the slow
> response.
> > > > > > > > >>>>>
> > > > > > > > >>>>> You both mentioned the change to handle
> > UNKNOWN_PRODUCER_ID
> > > > > > errors.
> > > > > > > > To
> > > > > > > > >>> be
> > > > > > > > >>>>> clear — this error code will only be sent again when
> the
> > > > > client's
> > > > > > > > >>> request
> > > > > > > > >>>>> version is high enough to ensure we handle it
> correctly.
> > > > > > > > >>>>> The current (Java) client handles this by the following
> > > > > (somewhat
> > > > > > > > long)
> > > > > > > > >>>>> code snippet:
> > > > > > > > >>>>>
> > > > > > > > >>>>> // An UNKNOWN_PRODUCER_ID means that we have lost the
> > > > producer
> > > > > > > state
> > > > > > > > >>> on the
> > > > > > > > >>>>> broker. Depending on the log start
> > > > > > > > >>>>>
> > > > > > > > >>>>> // offset, we may want to retry these, as described for
> > > each
> > > > > case
> > > > > > > > >>> below. If
> > > > > > > > >>>>> none of those apply, then for the
> > > > > > > > >>>>>
> > > > > > > > >>>>> // idempotent producer, we will locally bump the epoch
> > and
> > > > > reset
> > > > > > > the
> > > > > > > > >>>>> sequence numbers of in-flight batches from
> > > > > > > > >>>>>
> > > > > > > > >>>>> // sequence 0, then retry the failed batch, which
> should
> > > now
> > > > > > > succeed.
> > > > > > > > >>> For
> > > > > > > > >>>>> the transactional producer, allow the
> > > > > > > > >>>>>
> > > > > > > > >>>>> // batch to fail. When processing the failed batch, we
> > will
> > > > > > > > transition
> > > > > > > > >>> to
> > > > > > > > >>>>> an abortable error and set a flag
> > > > > > > > >>>>>
> > > > > > > > >>>>> // indicating that we need to bump the epoch (if
> > supported
> > > by
> > > > > the
> > > > > > > > >>> broker).
> > > > > > > > >>>>>
> > > > > > > > >>>>> if (error == Errors.*UNKNOWN_PRODUCER_ID*) {
> > > > > > > > >>>>>
> > > > > > > > >>>>>       if (response.logStartOffset == -1) {
> > > > > > > > >>>>>
> > > > > > > > >>>>>           // We don't know the log start offset with
> this
> > > > > > response.
> > > > > > > > We
> > > > > > > > >>> should
> > > > > > > > >>>>> just retry the request until we get it.
> > > > > > > > >>>>>
> > > > > > > > >>>>>           // The UNKNOWN_PRODUCER_ID error code was
> added
> > > > along
> > > > > > > with
> > > > > > > > >>> the new
> > > > > > > > >>>>> ProduceResponse which includes the
> > > > > > > > >>>>>
> > > > > > > > >>>>>           // logStartOffset. So the '-1' sentinel is
> not
> > > for
> > > > > > > backward
> > > > > > > > >>>>> compatibility. Instead, it is possible for
> > > > > > > > >>>>>
> > > > > > > > >>>>>           // a broker to not know the logStartOffset at
> > > when
> > > > it
> > > > > > is
> > > > > > > > >>> returning
> > > > > > > > >>>>> the response because the partition
> > > > > > > > >>>>>
> > > > > > > > >>>>>           // may have moved away from the broker from
> the
> > > > time
> > > > > > the
> > > > > > > > >>> error was
> > > > > > > > >>>>> initially raised to the time the
> > > > > > > > >>>>>
> > > > > > > > >>>>>           // response was being constructed. In these
> > > cases,
> > > > we
> > > > > > > > should
> > > > > > > > >>> just
> > > > > > > > >>>>> retry the request: we are guaranteed
> > > > > > > > >>>>>
> > > > > > > > >>>>>           // to eventually get a logStartOffset once
> > things
> > > > > > settle
> > > > > > > > down.
> > > > > > > > >>>>>
> > > > > > > > >>>>>           return true;
> > > > > > > > >>>>>
> > > > > > > > >>>>>       }
> > > > > > > > >>>>>
> > > > > > > > >>>>>
> > > > > > > > >>>>>       if (batch.sequenceHasBeenReset()) {
> > > > > > > > >>>>>
> > > > > > > > >>>>>           // When the first inflight batch fails due to
> > the
> > > > > > > > truncation
> > > > > > > > >>> case,
> > > > > > > > >>>>> then the sequences of all the other
> > > > > > > > >>>>>
> > > > > > > > >>>>>           // in flight batches would have been
> restarted
> > > from
> > > > > the
> > > > > > > > >>> beginning.
> > > > > > > > >>>>> However, when those responses
> > > > > > > > >>>>>
> > > > > > > > >>>>>           // come back from the broker, they would also
> > > come
> > > > > with
> > > > > > > an
> > > > > > > > >>>>> UNKNOWN_PRODUCER_ID error. In this case, we should not
> > > > > > > > >>>>>
> > > > > > > > >>>>>           // reset the sequence numbers to the
> beginning.
> > > > > > > > >>>>>
> > > > > > > > >>>>>           return true;
> > > > > > > > >>>>>
> > > > > > > > >>>>>       } else if
> > > > (lastAckedOffset(batch.topicPartition).orElse(
> > > > > > > > >>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > > response.logStartOffset) {
> > > > > > > > >>>>>
> > > > > > > > >>>>>           // The head of the log has been removed,
> > probably
> > > > due
> > > > > > to
> > > > > > > > the
> > > > > > > > >>>>> retention time elapsing. In this case,
> > > > > > > > >>>>>
> > > > > > > > >>>>>           // we expect to lose the producer state. For
> > the
> > > > > > > > transactional
> > > > > > > > >>>>> producer, reset the sequences of all
> > > > > > > > >>>>>
> > > > > > > > >>>>>           // inflight batches to be from the beginning
> > and
> > > > > retry
> > > > > > > > them,
> > > > > > > > >>> so
> > > > > > > > >>>>> that the transaction does not need to
> > > > > > > > >>>>>
> > > > > > > > >>>>>           // be aborted. For the idempotent producer,
> > bump
> > > > the
> > > > > > > epoch
> > > > > > > > to
> > > > > > > > >>> avoid
> > > > > > > > >>>>> reusing (sequence, epoch) pairs
> > > > > > > > >>>>>
> > > > > > > > >>>>>           if (isTransactional()) {
> > > > > > > > >>>>>
> > > > > > > > >>>>>
> > > > > > > > >>>
> > > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > > > > > > > >>>>> this.producerIdAndEpoch);
> > > > > > > > >>>>>
> > > > > > > > >>>>>           } else {
> > > > > > > > >>>>>
> > > > > > > > >>>>>
> > > > >  requestEpochBumpForPartition(batch.topicPartition);
> > > > > > > > >>>>>
> > > > > > > > >>>>>           }
> > > > > > > > >>>>>
> > > > > > > > >>>>>           return true;
> > > > > > > > >>>>>
> > > > > > > > >>>>>       }
> > > > > > > > >>>>>
> > > > > > > > >>>>>
> > > > > > > > >>>>>       if (!isTransactional()) {
> > > > > > > > >>>>>
> > > > > > > > >>>>>           // For the idempotent producer, always retry
> > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > > > > > > >>>>> errors. If the batch has the current
> > > > > > > > >>>>>
> > > > > > > > >>>>>           // producer ID and epoch, request a bump of
> the
> > > > > epoch.
> > > > > > > > >>> Otherwise
> > > > > > > > >>>>> just retry the produce.
> > > > > > > > >>>>>
> > > > > > > > >>>>>
> > >  requestEpochBumpForPartition(batch.topicPartition);
> > > > > > > > >>>>>
> > > > > > > > >>>>>           return true;
> > > > > > > > >>>>>
> > > > > > > > >>>>>       }
> > > > > > > > >>>>>
> > > > > > > > >>>>> }
> > > > > > > > >>>>>
> > > > > > > > >>>>>
> > > > > > > > >>>>> I was considering keeping this behavior — but am open
> to
> > > > > > > simplifying
> > > > > > > > >>> it.
> > > > > > > > >>>>>
> > > > > > > > >>>>>
> > > > > > > > >>>>>
> > > > > > > > >>>>> We are leaving changes to older clients off the table
> > here
> > > > > since
> > > > > > it
> > > > > > > > >>> caused
> > > > > > > > >>>>> many issues for clients in the past. Previously this
> was
> > a
> > > > > fatal
> > > > > > > > error
> > > > > > > > >>> and
> > > > > > > > >>>>> we didn't have the mechanisms in place to detect when
> > this
> > > > was
> > > > > a
> > > > > > > > >>> legitimate
> > > > > > > > >>>>> case vs some bug or gap in the protocol. Ensuring each
> > > > > > transaction
> > > > > > > > has
> > > > > > > > >>> its
> > > > > > > > >>>>> own epoch should close this gap.
> > > > > > > > >>>>>
> > > > > > > > >>>>>
> > > > > > > > >>>>>
> > > > > > > > >>>>>
> > > > > > > > >>>>> And to address Jeff's second point:
> > > > > > > > >>>>> *does the typical produce request path append records
> to
> > > > local
> > > > > > log
> > > > > > > > >>> along*
> > > > > > > > >>>>>
> > > > > > > > >>>>> *with the currentTxnFirstOffset information? I would
> like
> > > to
> > > > > > > > >>> understand*
> > > > > > > > >>>>>
> > > > > > > > >>>>> *when the field is written to disk.*
> > > > > > > > >>>>>
> > > > > > > > >>>>>
> > > > > > > > >>>>> Yes, the first produce request populates this field and
> > > > writes
> > > > > > the
> > > > > > > > >>> offset
> > > > > > > > >>>>> as part of the record batch and also to the producer
> > state
> > > > > > > snapshot.
> > > > > > > > >>> When
> > > > > > > > >>>>> we reload the records on restart and/or reassignment,
> we
> > > > > > repopulate
> > > > > > > > >>> this
> > > > > > > > >>>>> field with the snapshot from disk along with the rest
> of
> > > the
> > > > > > > producer
> > > > > > > > >>>>> state.
> > > > > > > > >>>>>
> > > > > > > > >>>>> Let me know if there are further comments and/or
> > questions.
> > > > > > > > >>>>>
> > > > > > > > >>>>> Thanks,
> > > > > > > > >>>>> Justine
> > > > > > > > >>>>>
> > > > > > > > >>>>> On Tue, Nov 22, 2022 at 9:00 PM Jeff Kim
> > > > > > > > <jeff.kim@confluent.io.invalid
> > > > > > > > >>>>
> > > > > > > > >>>>> wrote:
> > > > > > > > >>>>>
> > > > > > > > >>>>>> Hi Justine,
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> Thanks for the KIP! I have two questions:
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> 1) For new clients, we can once again return an error
> > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > > > > > > >>>>>> for sequences
> > > > > > > > >>>>>> that are non-zero when there is no producer state
> > present
> > > on
> > > > > the
> > > > > > > > >>> server.
> > > > > > > > >>>>>> This will indicate we missed the 0 sequence and we
> don't
> > > yet
> > > > > > want
> > > > > > > to
> > > > > > > > >>>>> write
> > > > > > > > >>>>>> to the log.
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> I would like to understand the current behavior to
> > handle
> > > > > older
> > > > > > > > >>> clients,
> > > > > > > > >>>>>> and if there are any changes we are making. Maybe I'm
> > > > missing
> > > > > > > > >>> something,
> > > > > > > > >>>>>> but we would want to identify whether we missed the 0
> > > > sequence
> > > > > > for
> > > > > > > > >>> older
> > > > > > > > >>>>>> clients, no?
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> 2) Upon returning from the transaction coordinator, we
> > can
> > > > set
> > > > > > the
> > > > > > > > >>>>>> transaction
> > > > > > > > >>>>>> as ongoing on the leader by populating
> > > currentTxnFirstOffset
> > > > > > > > >>>>>> through the typical produce request handling.
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> does the typical produce request path append records
> to
> > > > local
> > > > > > log
> > > > > > > > >>> along
> > > > > > > > >>>>>> with the currentTxnFirstOffset information? I would
> like
> > > to
> > > > > > > > understand
> > > > > > > > >>>>>> when the field is written to disk.
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> Thanks,
> > > > > > > > >>>>>> Jeff
> > > > > > > > >>>>>>
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> On Tue, Nov 22, 2022 at 4:44 PM Artem Livshits
> > > > > > > > >>>>>> <al...@confluent.io.invalid> wrote:
> > > > > > > > >>>>>>
> > > > > > > > >>>>>>> Hi Justine,
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> Thank you for the KIP.  I have one question.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> 5) For new clients, we can once again return an error
> > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> I believe we had problems in the past with returning
> > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > >>>>>>> because it was considered fatal and required client
> > > > restart.
> > > > > > It
> > > > > > > > >>> would
> > > > > > > > >>>>> be
> > > > > > > > >>>>>>> good to spell out the new client behavior when it
> > > receives
> > > > > the
> > > > > > > > error.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> -Artem
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> On Tue, Nov 22, 2022 at 10:00 AM Justine Olshan
> > > > > > > > >>>>>>> <jo...@confluent.io.invalid> wrote:
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>>> Thanks for taking a look Matthias. I've tried to
> > answer
> > > > your
> > > > > > > > >>>>> questions
> > > > > > > > >>>>>>>> below:
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> 10)
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> Right — so the hanging transaction only occurs when
> we
> > > > have
> > > > > a
> > > > > > > late
> > > > > > > > >>>>>>> message
> > > > > > > > >>>>>>>> come in and the partition is never added to a
> > > transaction
> > > > > > again.
> > > > > > > > If
> > > > > > > > >>>>> we
> > > > > > > > >>>>>>>> never add the partition to a transaction, we will
> > never
> > > > > write
> > > > > > a
> > > > > > > > >>>>> marker
> > > > > > > > >>>>>>> and
> > > > > > > > >>>>>>>> never advance the LSO.
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> If we do end up adding the partition to the
> > transaction
> > > (I
> > > > > > > suppose
> > > > > > > > >>>>> this
> > > > > > > > >>>>>>> can
> > > > > > > > >>>>>>>> happen before or after the late message comes in)
> then
> > > we
> > > > > will
> > > > > > > > >>>>> include
> > > > > > > > >>>>>>> the
> > > > > > > > >>>>>>>> late message in the next (incorrect) transaction.
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> So perhaps it is clearer to make the distinction
> > between
> > > > > > > messages
> > > > > > > > >>>>> that
> > > > > > > > >>>>>>>> eventually get added to the transaction (but the
> wrong
> > > > one)
> > > > > or
> > > > > > > > >>>>> messages
> > > > > > > > >>>>>>>> that never get added and become hanging.
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> 20)
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> The client side change for 2 is removing the
> > > addPartitions
> > > > > to
> > > > > > > > >>>>>> transaction
> > > > > > > > >>>>>>>> call. We don't need to make this from the producer
> to
> > > the
> > > > > txn
> > > > > > > > >>>>>>> coordinator,
> > > > > > > > >>>>>>>> only server side.
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> In my opinion, the issue with the addPartitionsToTxn
> > > call
> > > > > for
> > > > > > > > older
> > > > > > > > >>>>>>> clients
> > > > > > > > >>>>>>>> is that we don't have the epoch bump, so we don't
> know
> > > if
> > > > > the
> > > > > > > > >>> message
> > > > > > > > >>>>>>>> belongs to the previous transaction or this one. We
> > need
> > > > to
> > > > > > > check
> > > > > > > > if
> > > > > > > > >>>>>> the
> > > > > > > > >>>>>>>> partition has been added to this transaction. Of
> > course,
> > > > > this
> > > > > > > > means
> > > > > > > > >>>>> we
> > > > > > > > >>>>>>>> won't completely cover the case where we have a
> really
> > > > late
> > > > > > > > message
> > > > > > > > >>>>> and
> > > > > > > > >>>>>>> we
> > > > > > > > >>>>>>>> have added the partition to the new transaction, but
> > > > that's
> > > > > > > > >>>>>> unfortunately
> > > > > > > > >>>>>>>> something we will need the new clients to cover.
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> 30)
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> Transaction is ongoing = partition was added to
> > > > transaction
> > > > > > via
> > > > > > > > >>>>>>>> addPartitionsToTxn. We check this with the
> > > > > > DescribeTransactions
> > > > > > > > >>> call.
> > > > > > > > >>>>>> Let
> > > > > > > > >>>>>>>> me know if this wasn't sufficiently explained here:
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>
> > > > > > > > >>>>>
> > > > > > > > >>>
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> 40)
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> The idea here is that if any messages somehow come
> in
> > > > before
> > > > > > we
> > > > > > > > get
> > > > > > > > >>>>> the
> > > > > > > > >>>>>>> new
> > > > > > > > >>>>>>>> epoch to the producer, they will be fenced. However,
> > if
> > > we
> > > > > > don't
> > > > > > > > >>>>> think
> > > > > > > > >>>>>>> this
> > > > > > > > >>>>>>>> is necessary, it can be discussed
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> 50)
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> It should be synchronous because if we have an event
> > > (ie,
> > > > an
> > > > > > > > error)
> > > > > > > > >>>>>> that
> > > > > > > > >>>>>>>> causes us to need to abort the transaction, we need
> to
> > > > know
> > > > > > > which
> > > > > > > > >>>>>>>> partitions to send transaction markers to. We know
> the
> > > > > > > partitions
> > > > > > > > >>>>>> because
> > > > > > > > >>>>>>>> we added them to the coordinator via the
> > > > addPartitionsToTxn
> > > > > > > call.
> > > > > > > > >>>>>>>> Previously we have had asynchronous calls in the
> past
> > > (ie,
> > > > > > > writing
> > > > > > > > >>>>> the
> > > > > > > > >>>>>>>> commit markers when the transaction is completed)
> but
> > > > often
> > > > > > this
> > > > > > > > >>> just
> > > > > > > > >>>>>>>> causes confusion as we need to wait for some
> > operations
> > > to
> > > > > > > > complete.
> > > > > > > > >>>>> In
> > > > > > > > >>>>>>> the
> > > > > > > > >>>>>>>> writing commit markers case, clients often see
> > > > > > > > >>>>> CONCURRENT_TRANSACTIONs
> > > > > > > > >>>>>>>> error messages and that can be confusing. For that
> > > reason,
> > > > > it
> > > > > > > may
> > > > > > > > be
> > > > > > > > >>>>>>>> simpler to just have synchronous calls — especially
> if
> > > we
> > > > > need
> > > > > > > to
> > > > > > > > >>>>> block
> > > > > > > > >>>>>>> on
> > > > > > > > >>>>>>>> some operation's completion anyway before we can
> start
> > > the
> > > > > > next
> > > > > > > > >>>>>>>> transaction. And yes, I meant coordinator. I will
> fix
> > > > that.
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> 60)
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> When we are checking if the transaction is ongoing,
> we
> > > > need
> > > > > to
> > > > > > > > make
> > > > > > > > >>> a
> > > > > > > > >>>>>>> round
> > > > > > > > >>>>>>>> trip from the leader partition to the transaction
> > > > > coordinator.
> > > > > > > In
> > > > > > > > >>> the
> > > > > > > > >>>>>>> time
> > > > > > > > >>>>>>>> we are waiting for this message to come back, in
> > theory
> > > we
> > > > > > could
> > > > > > > > >>> have
> > > > > > > > >>>>>>> sent
> > > > > > > > >>>>>>>> a commit/abort call that would make the original
> > result
> > > of
> > > > > the
> > > > > > > > check
> > > > > > > > >>>>>> out
> > > > > > > > >>>>>>> of
> > > > > > > > >>>>>>>> date. That is why we can check the leader state
> before
> > > we
> > > > > > write
> > > > > > > to
> > > > > > > > >>>>> the
> > > > > > > > >>>>>>> log.
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> I'm happy to update the KIP if some of these things
> > were
> > > > not
> > > > > > > > clear.
> > > > > > > > >>>>>>>> Thanks,
> > > > > > > > >>>>>>>> Justine
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> On Mon, Nov 21, 2022 at 7:11 PM Matthias J. Sax <
> > > > > > > mjsax@apache.org
> > > > > > > > >
> > > > > > > > >>>>>>> wrote:
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>>> Thanks for the KIP.
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> Couple of clarification questions (I am not a
> broker
> > > > expert
> > > > > > do
> > > > > > > > >>>>> maybe
> > > > > > > > >>>>>>>>> some question are obvious for others, but not for
> me
> > > with
> > > > > my
> > > > > > > lack
> > > > > > > > >>>>> of
> > > > > > > > >>>>>>>>> broker knowledge).
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> (10)
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>>> 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.
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> What happens if the message come in before the next
> > > > > > > > >>>>>> addPartitionsToTxn
> > > > > > > > >>>>>>>>> request? It seems the broker hosting the data
> > > partitions
> > > > > > won't
> > > > > > > > know
> > > > > > > > >>>>>>>>> anything about it and append it to the partition,
> > too?
> > > > What
> > > > > > is
> > > > > > > > the
> > > > > > > > >>>>>>>>> difference between both cases?
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> Also, it seems a TX would only hang, if there is no
> > > > > following
> > > > > > > TX
> > > > > > > > >>>>> that
> > > > > > > > >>>>>>> is
> > > > > > > > >>>>>>>>> either committer or aborted? Thus, for the case
> > above,
> > > > the
> > > > > TX
> > > > > > > > might
> > > > > > > > >>>>>>>>> actually not hang (of course, we might get an EOS
> > > > violation
> > > > > > if
> > > > > > > > the
> > > > > > > > >>>>>>> first
> > > > > > > > >>>>>>>>> TX was aborted and the second committed, or the
> other
> > > way
> > > > > > > > around).
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> (20)
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>>> Of course, 1 and 2 require client-side changes, so
> > for
> > > > > older
> > > > > > > > >>>>>> clients,
> > > > > > > > >>>>>>>>> those approaches won’t apply.
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> For (1) I understand why a client change is
> > necessary,
> > > > but
> > > > > > not
> > > > > > > > sure
> > > > > > > > >>>>>> why
> > > > > > > > >>>>>>>>> we need a client change for (2). Can you elaborate?
> > --
> > > > > Later
> > > > > > > you
> > > > > > > > >>>>>>> explain
> > > > > > > > >>>>>>>>> that we should send a DescribeTransactionRequest,
> > but I
> > > > am
> > > > > > not
> > > > > > > > sure
> > > > > > > > >>>>>>> why?
> > > > > > > > >>>>>>>>> Can't we not just do an implicit AddPartiitonToTx,
> > too?
> > > > If
> > > > > > the
> > > > > > > > old
> > > > > > > > >>>>>>>>> producer correctly registered the partition
> already,
> > > the
> > > > > > > > >>>>>> TX-coordinator
> > > > > > > > >>>>>>>>> can just ignore it as it's an idempotent operation?
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> (30)
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>>> To cover older clients, we will ensure a
> transaction
> > > is
> > > > > > > ongoing
> > > > > > > > >>>>>>> before
> > > > > > > > >>>>>>>>> we write to a transaction
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> Not sure what you mean by this? Can you elaborate?
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> (40)
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>>> [the TX-coordinator] will write the prepare commit
> > > > message
> > > > > > > with
> > > > > > > > a
> > > > > > > > >>>>>>>> bumped
> > > > > > > > >>>>>>>>> epoch and send WriteTxnMarkerRequests with the
> bumped
> > > > > epoch.
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> Why do we use the bumped epoch for both? It seems
> > more
> > > > > > > intuitive
> > > > > > > > to
> > > > > > > > >>>>>> use
> > > > > > > > >>>>>>>>> the current epoch, and only return the bumped epoch
> > to
> > > > the
> > > > > > > > >>>>> producer?
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> (50) "Implicit AddPartitionToTransaction"
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> Why does the implicitly sent request need to be
> > > > > synchronous?
> > > > > > > The
> > > > > > > > >>>>> KIP
> > > > > > > > >>>>>>>>> also says
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>>> in case we need to abort and need to know which
> > > > partitions
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> What do you mean by this?
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>>> we don’t want to write to it before we store in
> the
> > > > > > > transaction
> > > > > > > > >>>>>>> manager
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> Do you mean TX-coordinator instead of "manager"?
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> (60)
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> For older clients and ensuring that the TX is
> > ongoing,
> > > > you
> > > > > > > > >>>>> describe a
> > > > > > > > >>>>>>>>> race condition. I am not sure if I can follow here.
> > Can
> > > > you
> > > > > > > > >>>>>> elaborate?
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> -Matthias
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> On 11/18/22 1:21 PM, Justine Olshan wrote:
> > > > > > > > >>>>>>>>>> Hey all!
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>>> I'd like to start a discussion on my proposal to
> add
> > > > some
> > > > > > > > >>>>>> server-side
> > > > > > > > >>>>>>>>>> checks on transactions to avoid hanging
> > transactions.
> > > I
> > > > > know
> > > > > > > > this
> > > > > > > > >>>>>> has
> > > > > > > > >>>>>>>>> been
> > > > > > > > >>>>>>>>>> an issue for some time, so I really hope this KIP
> > will
> > > > be
> > > > > > > > helpful
> > > > > > > > >>>>>> for
> > > > > > > > >>>>>>>>> many
> > > > > > > > >>>>>>>>>> users of EOS.
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>>> The KIP includes changes that will be compatible
> > with
> > > > old
> > > > > > > > clients
> > > > > > > > >>>>>> and
> > > > > > > > >>>>>>>>>> changes to improve performance and correctness on
> > new
> > > > > > clients.
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>>> Please take a look and leave any comments you may
> > > have!
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>>> KIP:
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>
> > > > > > > > >>>>>
> > > > > > > > >>>
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > > > > >>>>>>>>>> JIRA:
> > > https://issues.apache.org/jira/browse/KAFKA-14402
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>>> Thanks!
> > > > > > > > >>>>>>>>>> Justine
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>
> > > > > > > > >>>>>
> > > > > > > > >>>>
> > > > > > > > >>>
> > > > > > > > >>
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

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
>>> > > > > > > > >> > > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > >
>>> > > > > > > > >> > > > > > > > >
>>> > > > > > > > >> > > > > > > >
>>> > > > > > > > >> > > > > > >
>>> > > > > > > > >> > > > > >
>>> > > > > > > > >> > > > >
>>> > > > > > > > >> > > >
>>> > > > > > > > >> > >
>>> > > > > > > > >> >
>>> > > > > > > > >>
>>> > > > > > > > >
>>> > > > > > > >
>>> > > > > > >
>>> > > > > >
>>> > > > >
>>> > > >
>>> > >
>>> >
>>>
>>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
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>.
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 <jolshan@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 <jun@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 Jun Rao <ju...@confluent.io.INVALID>.
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
> > > > > > > >> > > > > > > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > > >
> > > > > > > >> > > > > > > > > > >
> > > > > > > >> > > > > > > > > >
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
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
> > > > > > >> > > > > > > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > > >
> > > > > > >> > > > > > > > > > > >
> > > > > > >> > > > > > > > > > >
> > > > > > >> > > > > > > > > >
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Jun Rao <ju...@confluent.io.INVALID>.
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 <jo...@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
> > > > > >> > > > > > > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > > >
> > > > > >> > > > > > > > > > > > >
> > > > > >> > > > > > > > > > > >
> > > > > >> > > > > > > > > > >
> > > > > >> > > > > > > > > >
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > >
> > > > > >> > > > > > >
> > > > > >> > > > > >
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
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 <ju...@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
> > > > >> > > > > > > > > > > > > > > > >
> > > > >> > > > > > > > > > > > > > > >
> > > > >> > > > > > > > > > > > > > > >
> > > > >> > > > > > > > > > > > > > > >
> > > > >> > > > > > > > > > > > > > >
> > > > >> > > > > > > > > > > > > >
> > > > >> > > > > > > > > > > > >
> > > > >> > > > > > > > > > > >
> > > > >> > > > > > > > > > >
> > > > >> > > > > > > > > >
> > > > >> > > > > > > > >
> > > > >> > > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Jun Rao <ju...@confluent.io.INVALID>.
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 <jo...@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 <jo...@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 <ju...@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
> > > >> > > > > > > > > > > > > > > > >
> > > >> > > > > > > > > > > > > > > >
> > > >> > > > > > > > > > > > > > > >
> > > >> > > > > > > > > > > > > > > >
> > > >> > > > > > > > > > > > > > >
> > > >> > > > > > > > > > > > > >
> > > >> > > > > > > > > > > > >
> > > >> > > > > > > > > > > >
> > > >> > > > > > > > > > >
> > > >> > > > > > > > > >
> > > >> > > > > > > > >
> > > >> > > > > > > >
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
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 <jo...@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 <ju...@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 <ju...@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
> > >> > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > >
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > >
> > >> > > > > > > > > >
> > >> > > > > > > > >
> > >> > > > > > > >
> > >> > > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Jun Rao <ju...@confluent.io.INVALID>.
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 <jo...@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 <ju...@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 <ju...@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
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > >
> >> > > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
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 <jo...@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 <ju...@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 <ju...@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 <ju...@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
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > >
>> > > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
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 <ju...@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 <ju...@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 <ju...@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 <jun@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
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Jun Rao <ju...@confluent.io.INVALID>.
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 <ju...@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 <ju...@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
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Artem Livshits <al...@confluent.io.INVALID>.
>  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.

I think Jun's point is that we can defer the fencing decision until
transition into complete state (which I believe is what the current logic
is doing) -- just return CONCURRENT_TRANSACTIONS without checking the epoch
while in the prepare state.

That said, we do need to remember the next producer id somewhere in the
prepare state, because in the complete state we would need to make a
fencing decision and let the old producer in if the request is the retry
commit / abort operation.

An alternative could be to not reply to the client until complete state is
written, then we don't have to generate a new producer id during prepare
state.  But that would affect pipelining opportunities and probably require
a separate KIP to discuss the pros and cons.

-Artem

On Mon, Jan 22, 2024 at 11:34 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 <ju...@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 <ju...@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
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
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 <ju...@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 <ju...@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
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Jun Rao <ju...@confluent.io.INVALID>.
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 <ju...@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 <ju...@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
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
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 <ju...@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 <ju...@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
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Jun Rao <ju...@confluent.io.INVALID>.
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 <ju...@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 <ju...@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
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
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 <ju...@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 <ju...@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
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Jun Rao <ju...@confluent.io.INVALID>.
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 <ju...@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 <ju...@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
> > > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
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 <ju...@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 <ju...@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 <ju...@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
> > > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Jun Rao <ju...@confluent.io.INVALID>.
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 <ju...@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 <ju...@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
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
(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 <ju...@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 <ju...@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 <ju...@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
> > > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Jun Rao <ju...@confluent.io.INVALID>.
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 <ju...@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 <ju...@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 <ju...@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 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 <ju...@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 <ju...@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 <ju...@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 Jun Rao <ju...@confluent.io.INVALID>.
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 <ju...@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 <ju...@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 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 <ju...@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 <ju...@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 Jun Rao <ju...@confluent.io.INVALID>.
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 <ju...@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>.
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 <ju...@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
>
>
> On Tue, Apr 18, 2023 at 9:27 AM Justine Olshan
> <jo...@confluent.io.invalid>
> wrote:
>
> > Hi Daniel,
> >
> > You are correct. We need to slightly adjust the response from how it
> > appeared in the KIP. I've been focusing on part 1 (old clients) but when
> we
> > get into part 2, I will address this issue. I will also update this
> thread
> > when we do.
> > Thanks for bringing it up.
> >
> > Justine.
> >
> > On Tue, Apr 18, 2023 at 12:28 AM Dániel Urbán <ur...@gmail.com>
> > wrote:
> >
> > > Hi,
> > >
> > > Sorry for being super-late to this discussion, but a question came to
> my
> > > mind related to the bump-on-txn-end part of the KIP.
> > > I see that the END_TXN_RESPONSE will be updated to also include the new
> > > epoch to be used by the producer - but what about epoch exhaustion?
> What
> > > will happen when the last ongoing transaction uses the max available
> > epoch,
> > > and we try to end it? The producer would need a new producer ID after
> the
> > > epoch bump. Because of this, I think the response should also contain
> the
> > > producer-id-to-use.
> > >
> > > Thanks,
> > > Daniel
> > >
> > > Artem Livshits <al...@confluent.io.invalid> ezt írta (időpont:
> 2023.
> > > jan. 28., Szo, 0:17):
> > >
> > > > >  I'd like to check that, under this KIP's circumstances, if we
> could
> > > ever
> > > > have "partial committed" txns
> > > >
> > > > The goal of this KIP is to harden transaction implementation to never
> > > have
> > > > partially committed txns.
> > > >
> > > > -Artem
> > > >
> > > > On Fri, Jan 27, 2023 at 10:04 AM Guozhang Wang <
> > > guozhang.wang.us@gmail.com
> > > > >
> > > > wrote:
> > > >
> > > > > Thanks folks, I agree with all of your points.
> > > > >
> > > > > I think in the past we did have some cases where abortable txns are
> > > > > handled as fatal ones which we are fixing, but maybe there are some
> > > > > vice versa as well. I'd like to check that, under this KIP's
> > > > > circumstances, if we could ever have "partial committed" txns where
> > > > > some of the data are considered as committed while some others are
> > > > > aborted later. And if there were, could we recover from it. If the
> > > > > answer is "yes and yes", then I think we can bite the complexity
> > > > > bullet inside the client and leave users in comfort and peace :)
> > > > >
> > > > > Guozhang
> > > > >
> > > > > On Fri, Jan 27, 2023 at 9:42 AM Artem Livshits
> > > > > <al...@confluent.io.invalid> wrote:
> > > > > >
> > > > > > I agree.  I also think we should go a little further and be more
> > > > > > prescriptive with our recommendations for error handling (i.e.
> not
> > > just
> > > > > > think if there is a chance, but propose design patterns for
> typical
> > > > > > cases).  For example, if a transaction is aborted, the
> application
> > > must
> > > > > > either keep enough state to be able to redo the operation from
> the
> > > > > > beginning, or reset all state and fully recover from Kafka (i.e.
> > > > > re-consume
> > > > > > data from the last committed offsets).
> > > > > >
> > > > > > For failed commits, we need a way for the application to either
> > learn
> > > > the
> > > > > > result (if commit went through, then continue, otherwise redo
> > > > operation)
> > > > > or
> > > > > > just reset all state and fully recover from Kafka (i.e.
> re-consume
> > > data
> > > > > > from the last committed offsets).
> > > > > >
> > > > > > -Artem
> > > > > >
> > > > > > On Fri, Jan 27, 2023 at 9:31 AM Justine Olshan
> > > > > <jo...@confluent.io.invalid>
> > > > > > wrote:
> > > > > >
> > > > > > > Thanks for clarifying Guozhang,
> > > > > > >
> > > > > > > I like the framing:
> > > > > > > * “if we did abort the txn and continue, could the app ever
> has a
> > > > > chance to
> > > > > > > recover and not have any side effects violating EOS
> guarantees”.*
> > > > > > >
> > > > > > > With this principle in mind, if we could convince ourselves
> that
> > we
> > > > > won't
> > > > > > > have side effects, then we could move forward with a non-fatal
> > > > error. I
> > > > > > > appreciate laying out the cases we've discussed. I think I also
> > > want
> > > > to
> > > > > > > take a look in very close detail and convince myself of the
> right
> > > > > option.
> > > > > > > I'll get back to this thread with the final conclusion as well
> as
> > > > > update
> > > > > > > the KIP.
> > > > > > >
> > > > > > > Thanks for also bringing up KIP-691. I should take a look there
> > as
> > > > > well. :)
> > > > > > >
> > > > > > > Justine
> > > > > > >
> > > > > > > On Thu, Jan 26, 2023 at 5:28 AM Guozhang Wang <
> > > > > guozhang.wang.us@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hello Justine,
> > > > > > > >
> > > > > > > > Regarding which errors should be fatal v.s. abortable, I
> think
> > > the
> > > > > > > > principle would be “if we did abort the txn and continue,
> could
> > > the
> > > > > > > > app ever has a chance to recover and not have any side
> effects
> > > > > > > > violating EOS guarantees”.Of course we can discuss whether
> > > that’s a
> > > > > > > > good principle  e.g. one can argue that maybe it’s okay to
> > > violate
> > > > > EOS
> > > > > > > > some times rather than killing the whole app, whereas my
> > > rationale
> > > > is
> > > > > > > > that since we want to make EOS the default config, it’s
> crucial
> > > > that
> > > > > > > > we try to guarantee it as much as possible and whenever we
> > > detected
> > > > > if
> > > > > > > > there’s a chance that it maybe broken, we should let users
> > know.
> > > > And
> > > > > > > > if we just log an error in app and try to abort and continue,
> > > it’s
> > > > > > > > very likely that users would not be notified and probably
> only
> > > > > realize
> > > > > > > > that after a while, which may cause an even harder scenario
> to
> > > > > > > > debug.If people agree on that principle, we can go back and
> > check
> > > > > this
> > > > > > > > KIP’s scenario: there are several scenarios causing a
> partition
> > > > > leader
> > > > > > > > detects an out of date epoch from a produce requests:
> > > > > > > >
> > > > > > > > * The old produce requests arrive late after a network
> > partition,
> > > > and
> > > > > > > > the old producer is already gone.
> > > > > > > > * The old produce requests arrive late after a network
> > partition,
> > > > but
> > > > > > > > the old producer is still around.
> > > > > > > > * A buggy producer client that did not follow the protocol.
> > > > > > > >
> > > > > > > > For the third case, we should try to let user know asap, and
> as
> > > > clear
> > > > > > > > as possible, and hence it’s best to just stop the client app;
> > for
> > > > the
> > > > > > > > first case, it does not matter since the producer is already
> > > gone;
> > > > > > > > only for the second, probably the least likely case, we need
> to
> > > > > > > > ponder, and there my rationale again is that at that time, we
> > may
> > > > > have
> > > > > > > > already violated the EOS guarantees since there are some
> > partial
> > > > txn
> > > > > > > > records that should be rejected while the txn itself has been
> > > > > > > > committed. In this case I think it’s better to let
> > clients/users
> > > > know
> > > > > > > > as soon as possible as well.
> > > > > > > >
> > > > > > > > Regarding the past scenarios where a fatal error killing the
> > > whole
> > > > > > > > apps, I believe part of that reason is that we were doing an
> > > > > > > > sub-optimal job on clients side handling various error cases
> > and
> > > > > > > > that’s what KIP-691 is trying to resolve, and hence
> personally
> > I
> > > > > would
> > > > > > > > suggest we do not weight in too much on that if we can trust
> > that
> > > > > > > > KIP-691 will be successfully eliminate those
> > > > not-necessary-hard-fail
> > > > > > > > scenarios.
> > > > > > > >
> > > > > > > >
> > > > > > > > Guozhang
> > > > > > > >
> > > > > > > > On Wed, Jan 25, 2023 at 5:51 PM Matthias J. Sax <
> > > mjsax@apache.org>
> > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > So the timestamp would be set when the write happens and
> thus
> > > no
> > > > > > > > > out-of-order data (base in time) can be introduced with
> > > > > "append_time"
> > > > > > > > > config even if a request sits in purgatory first while we
> > check
> > > > > the TX
> > > > > > > > > status.
> > > > > > > > >
> > > > > > > > > That does make sense. Thanks for confirming, that there is
> no
> > > > > > > > > out-of-order issue for this case.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > -Matthias
> > > > > > > > >
> > > > > > > > > On 1/25/23 5:04 PM, Justine Olshan wrote:
> > > > > > > > > > Hey Matthias,
> > > > > > > > > > Let me put it this way, if a producer is checking if a
> > > > > transaction is
> > > > > > > > > > ongoing, then no writes to the partition from the
> producer
> > > will
> > > > > go
> > > > > > > > through
> > > > > > > > > > until the transaction is confirmed ongoing.
> > > > > > > > > >  From then, I think I can apply the writes in the order
> > they
> > > > > came in.
> > > > > > > > Does
> > > > > > > > > > that make sense?
> > > > > > > > > >
> > > > > > > > > > Let me know if I'm missing something.
> > > > > > > > > > Justine
> > > > > > > > > >
> > > > > > > > > > On Wed, Jan 25, 2023 at 4:57 PM Matthias J. Sax <
> > > > > mjsax@apache.org>
> > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > >>> would it build an offset map with just the latest
> > timestamp
> > > > > for a
> > > > > > > > key?
> > > > > > > > > >>
> > > > > > > > > >> Cannot remember the details without reading the KIP, but
> > > yes,
> > > > > > > > something
> > > > > > > > > >> like this (I believe it actually needs to track both,
> > offset
> > > > and
> > > > > > > > > >> timestamp per key).
> > > > > > > > > >>
> > > > > > > > > >>> I wonder if ordering assumptions are baked in there,
> why
> > > not
> > > > > use
> > > > > > > > > >> offset-based compaction.
> > > > > > > > > >>
> > > > > > > > > >> The use case is a compacted topic that does contain
> > > > out-of-order
> > > > > > > data.
> > > > > > > > > >> If you set key k1=v1 @ 5 offset 100 and later key1 = v0
> @
> > 3
> > > at
> > > > > > > offset
> > > > > > > > > >> 200 we want to cleanup v0 with higher offset because
> it's
> > > > > > > out-of-order
> > > > > > > > > >> based on time, but keep v1 what is the actual latest
> > version
> > > > of
> > > > > k1.
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >>> I was also not aware of this "guarantee" with regards
> to
> > > > broker
> > > > > > > side
> > > > > > > > > >> time.
> > > > > > > > > >>
> > > > > > > > > >> As already said: I am not sure if it's a public
> contract,
> > > but
> > > > > based
> > > > > > > on
> > > > > > > > > >> my experience, people might reply on it as "implicit
> > > > contract".
> > > > > --
> > > > > > > > Maybe
> > > > > > > > > >> somebody else knows if it's public or not, and if it
> would
> > > be
> > > > > ok to
> > > > > > > > > >> "break" it.
> > > > > > > > > >>
> > > > > > > > > >>> Let me know if you have any concerns here.
> > > > > > > > > >>
> > > > > > > > > >> My understanding is: While we cannot make an
> offset-order
> > > > > guarantee
> > > > > > > > for
> > > > > > > > > >> interleaved writes of different producer, if the topic
> is
> > > > > configures
> > > > > > > > > >> with "append_time", we "guarantee" (cf. my comment
> above")
> > > > > timestamp
> > > > > > > > > >> order... If that's the case, it would be an issue if we
> > > break
> > > > > this
> > > > > > > > > >> "guarantee".
> > > > > > > > > >>
> > > > > > > > > >> I am not sure when the broker sets the timestamp for
> > > > > "append_time"
> > > > > > > > > >> config? If we do it before putting the request into
> > > purgatory,
> > > > > we
> > > > > > > > have a
> > > > > > > > > >> problem. However, if we set the timestamp when we
> actually
> > > > > process
> > > > > > > the
> > > > > > > > > >> request and do the actual append, it seems there is no
> > > issue,
> > > > > as the
> > > > > > > > > >> request that was waiting in purgatory get the "newest"
> > > > > timestamp and
> > > > > > > > > >> thus cannot introduce out-of-order data.
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> -Matthias
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> On 1/24/23 10:44 AM, Justine Olshan wrote:
> > > > > > > > > >>> Hey Matthias,
> > > > > > > > > >>>
> > > > > > > > > >>> I have actually never heard of KIP-280 so thanks for
> > > bringing
> > > > > it
> > > > > > > up.
> > > > > > > > That
> > > > > > > > > >>> seems interesting. I wonder how it would work though --
> > > would
> > > > > it
> > > > > > > > build an
> > > > > > > > > >>> offset map with just the latest timestamp for a key? I
> > > wonder
> > > > > if
> > > > > > > > ordering
> > > > > > > > > >>> assumptions are baked in there, why not use
> offset-based
> > > > > > > compaction.
> > > > > > > > > >>>
> > > > > > > > > >>> I was also not aware of this "guarantee" with regards
> to
> > > > broker
> > > > > > > side
> > > > > > > > > >> time.
> > > > > > > > > >>> I think that we can do in order handling for a given
> > > > producer,
> > > > > but
> > > > > > > > not
> > > > > > > > > >>> across all producers. However, we can't guarantee that
> > > > anyway.
> > > > > > > > > >>>
> > > > > > > > > >>> Let me know if you have any concerns here.
> > > > > > > > > >>>
> > > > > > > > > >>> Thanks,
> > > > > > > > > >>> Justine
> > > > > > > > > >>>
> > > > > > > > > >>> On Mon, Jan 23, 2023 at 6:33 PM Matthias J. Sax <
> > > > > mjsax@apache.org>
> > > > > > > > > >> wrote:
> > > > > > > > > >>>
> > > > > > > > > >>>> Just a side note about Guozhang comments about
> > timestamps.
> > > > > > > > > >>>>
> > > > > > > > > >>>> If the producer sets the timestamp, putting the record
> > > into
> > > > > > > > purgatory
> > > > > > > > > >>>> seems not to be an issue (as already said: for this
> case
> > > we
> > > > > don't
> > > > > > > > > >>>> guarantee timestamp order between writes of different
> > > > > producers
> > > > > > > > anyway).
> > > > > > > > > >>>> However, if the broker sets the timestamp, the
> > expectation
> > > > is
> > > > > that
> > > > > > > > there
> > > > > > > > > >>>> is no out-of-order data in the partition ever; if we
> > would
> > > > > > > introduce
> > > > > > > > > >>>> out-of-order data for this case (for interleaved
> writes
> > of
> > > > > > > different
> > > > > > > > > >>>> producers), it seems we would violate the current
> > > contract?
> > > > > (To be
> > > > > > > > fair:
> > > > > > > > > >>>> I don't know if that's an official contract, but I
> > assume
> > > > > people
> > > > > > > > rely on
> > > > > > > > > >>>> this behavior -- and it "advertised" in many public
> > > > talks...)
> > > > > > > > > >>>>
> > > > > > > > > >>>> About compaction: there is actually KIP-280 that adds
> > > > > timestamp
> > > > > > > > based
> > > > > > > > > >>>> compaction what is a very useful feature for Kafka
> > Streams
> > > > > with
> > > > > > > > regard
> > > > > > > > > >>>> to out-of-order data handling. So the impact if we
> > > introduce
> > > > > > > > > >>>> out-of-order data could be larger scoped.
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>> -Matthias
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>> On 1/20/23 4:48 PM, Justine Olshan wrote:
> > > > > > > > > >>>>> Hey Artem,
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> I see there is a check for transactional producers.
> I'm
> > > > > wondering
> > > > > > > > if we
> > > > > > > > > >>>>> don't handle the epoch overflow case. I'm also not
> sure
> > > it
> > > > > will
> > > > > > > be
> > > > > > > > a
> > > > > > > > > >> huge
> > > > > > > > > >>>>> issue to extend to transactional producers, but maybe
> > I'm
> > > > > missing
> > > > > > > > > >>>> something.
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> As for the recovery path -- I think Guozhang's point
> > was
> > > if
> > > > > we
> > > > > > > > have a
> > > > > > > > > >> bad
> > > > > > > > > >>>>> client that repeatedly tries to produce without
> adding
> > to
> > > > the
> > > > > > > > > >> transaction
> > > > > > > > > >>>>> we would do the following:
> > > > > > > > > >>>>> a) if not fatal, we just fail the produce request
> over
> > > and
> > > > > over
> > > > > > > > > >>>>> b) if fatal, we fence the producer
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> Here with B, the issue with the client would be made
> > > clear
> > > > > more
> > > > > > > > > >> quickly.
> > > > > > > > > >>>> I
> > > > > > > > > >>>>> suppose there are some intermediate cases where the
> > issue
> > > > > only
> > > > > > > > occurs
> > > > > > > > > >>>>> sometimes, but I wonder if we should consider how to
> > > > recover
> > > > > with
> > > > > > > > > >> clients
> > > > > > > > > >>>>> who don't behave as expected anyway.
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> I think there is a place for the abortable error that
> > we
> > > > are
> > > > > > > > adding --
> > > > > > > > > >>>> just
> > > > > > > > > >>>>> abort and try again. But I think there are also some
> > > cases
> > > > > where
> > > > > > > > trying
> > > > > > > > > >>>> to
> > > > > > > > > >>>>> recover overcomplicates some logic. Especially if we
> > are
> > > > > > > > considering
> > > > > > > > > >>>> older
> > > > > > > > > >>>>> clients -- there I'm not sure if there's a ton we can
> > do
> > > > > besides
> > > > > > > > fail
> > > > > > > > > >> the
> > > > > > > > > >>>>> batch or fence the producer. With newer clients, we
> can
> > > > > consider
> > > > > > > > more
> > > > > > > > > >>>>> options for what can just be recovered after
> aborting.
> > > But
> > > > > epochs
> > > > > > > > might
> > > > > > > > > >>>> be
> > > > > > > > > >>>>> a hard one unless we also want to reset producer ID.
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> Thanks,
> > > > > > > > > >>>>> Justine
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> On Fri, Jan 20, 2023 at 3:59 PM Artem Livshits
> > > > > > > > > >>>>> <al...@confluent.io.invalid> wrote:
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>>>     besides the poorly written client case
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> A poorly written client could create a lot of grief
> to
> > > > > people
> > > > > > > who
> > > > > > > > run
> > > > > > > > > >>>> Kafka
> > > > > > > > > >>>>>> brokers :-), so when deciding to make an error
> fatal I
> > > > > would see
> > > > > > > > if
> > > > > > > > > >>>> there
> > > > > > > > > >>>>>> is a reasonable recovery path rather than how often
> it
> > > > could
> > > > > > > > happen.
> > > > > > > > > >>>> If we
> > > > > > > > > >>>>>> have solid implementation of transactions (which I
> > hope
> > > > > we'll do
> > > > > > > > as a
> > > > > > > > > >>>>>> result of this KIP), it would help to recover from a
> > > large
> > > > > class
> > > > > > > > of
> > > > > > > > > >>>> errors
> > > > > > > > > >>>>>> by just aborting a transaction, even if the cause of
> > > error
> > > > > is a
> > > > > > > > race
> > > > > > > > > >>>>>> condition or etc.
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> -Artem
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> On Fri, Jan 20, 2023 at 3:26 PM Justine Olshan
> > > > > > > > > >>>>>> <jo...@confluent.io.invalid>
> > > > > > > > > >>>>>> wrote:
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>> Artem --
> > > > > > > > > >>>>>>> I guess the discussion path we were going down is
> > when
> > > we
> > > > > > > expect
> > > > > > > > to
> > > > > > > > > >> see
> > > > > > > > > >>>>>>> this error. I mentioned that it was hard to come up
> > > with
> > > > > cases
> > > > > > > > for
> > > > > > > > > >> when
> > > > > > > > > >>>>>> the
> > > > > > > > > >>>>>>> producer would still be around to receive the error
> > > > > besides the
> > > > > > > > > >> poorly
> > > > > > > > > >>>>>>> written client case.
> > > > > > > > > >>>>>>> If we don't expect to have a producer to receive
> the
> > > > > response,
> > > > > > > it
> > > > > > > > > >> sort
> > > > > > > > > >>>> of
> > > > > > > > > >>>>>>> makes sense for it to be fatal.
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> I had some discussion with Jason offline about the
> > > epoch
> > > > > being
> > > > > > > > off
> > > > > > > > > >>>> cases
> > > > > > > > > >>>>>>> and I'm not sure we could find a ton (outside of
> > > produce
> > > > > > > > requests)
> > > > > > > > > >>>> where
> > > > > > > > > >>>>>> we
> > > > > > > > > >>>>>>> could/should recover. I'd be happy to hear some
> > > examples
> > > > > > > though,
> > > > > > > > > >> maybe
> > > > > > > > > >>>>>> I'm
> > > > > > > > > >>>>>>> missing something.
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > >>>>>>> Justine
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> On Fri, Jan 20, 2023 at 3:19 PM Artem Livshits
> > > > > > > > > >>>>>>> <al...@confluent.io.invalid> wrote:
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>> In general, I'd like to avoid fatal errors as much
> > as
> > > > > > > possible,
> > > > > > > > in
> > > > > > > > > >>>> some
> > > > > > > > > >>>>>>>> sense fatal errors just push out recovery logic to
> > the
> > > > > > > > application
> > > > > > > > > >>>>>> which
> > > > > > > > > >>>>>>>> either complicates the application or leads to
> > > > disruption
> > > > > > > (we've
> > > > > > > > > >> seen
> > > > > > > > > >>>>>>> cases
> > > > > > > > > >>>>>>>> when a transient broker error could lead to work
> > > > stoppage
> > > > > when
> > > > > > > > > >>>>>>> applications
> > > > > > > > > >>>>>>>> need to be manually restarted).  I think we should
> > > > strive
> > > > > to
> > > > > > > > define
> > > > > > > > > >>>>>>>> recovery logic for most errors (and/or encapsulate
> > it
> > > in
> > > > > the
> > > > > > > > Kafka
> > > > > > > > > >>>>>> client
> > > > > > > > > >>>>>>>> as much as possible).
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> One benefit of transactions is that they simplify
> > > > recovery
> > > > > > > from
> > > > > > > > > >>>> errors,
> > > > > > > > > >>>>>>>> pretty much any error (that's not handled
> > > transparently
> > > > by
> > > > > > > > retries
> > > > > > > > > >> in
> > > > > > > > > >>>>>>> Kafka
> > > > > > > > > >>>>>>>> client) can be handled by the application via
> > aborting
> > > > the
> > > > > > > > > >> transaction
> > > > > > > > > >>>>>>> and
> > > > > > > > > >>>>>>>> repeating the transactional logic again.  One
> tricky
> > > > > error is
> > > > > > > an
> > > > > > > > > >> error
> > > > > > > > > >>>>>>>> during commit, because we don't know the outcome.
> > For
> > > > > commit
> > > > > > > > > >> errors,
> > > > > > > > > >>>>>> the
> > > > > > > > > >>>>>>>> recommendation should be to retry the commit until
> > it
> > > > > returns
> > > > > > > > the
> > > > > > > > > >>>>>>> specific
> > > > > > > > > >>>>>>>> result (committed or aborted).
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> -Artem
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> On Fri, Jan 20, 2023 at 2:52 PM Justine Olshan
> > > > > > > > > >>>>>>>> <jo...@confluent.io.invalid>
> > > > > > > > > >>>>>>>> wrote:
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>>> That's a fair point about other clients.
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>> I think the abortable error case is interesting
> > > because
> > > > > I'm
> > > > > > > > curious
> > > > > > > > > >>>>>> how
> > > > > > > > > >>>>>>>>> other clients would handle this. I assume they
> > would
> > > > > need to
> > > > > > > > > >>>>>> implement
> > > > > > > > > >>>>>>>>> handling for the error code unless they did
> > something
> > > > > like
> > > > > > > "any
> > > > > > > > > >>>>>> unknown
> > > > > > > > > >>>>>>>>> error codes/any codes that aren't x,y,z are
> > > > retriable." I
> > > > > > > would
> > > > > > > > > >> hope
> > > > > > > > > >>>>>>> that
> > > > > > > > > >>>>>>>>> unknown error codes were fatal, and if the code
> was
> > > > > > > > implemented it
> > > > > > > > > >>>>>>> would
> > > > > > > > > >>>>>>>>> abort the transaction. But I will think on this
> > too.
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>> As for InvalidRecord -- you mentioned it was not
> > > fatal,
> > > > > but
> > > > > > > I'm
> > > > > > > > > >>>>>> taking
> > > > > > > > > >>>>>>> a
> > > > > > > > > >>>>>>>>> look through the code. We would see this on
> > handling
> > > > the
> > > > > > > > produce
> > > > > > > > > >>>>>>>> response.
> > > > > > > > > >>>>>>>>> If I recall correctly, we check if errors are
> > > > retriable.
> > > > > I
> > > > > > > > think
> > > > > > > > > >> this
> > > > > > > > > >>>>>>>> error
> > > > > > > > > >>>>>>>>> would not be retriable. But I guess the concern
> > here
> > > is
> > > > > that
> > > > > > > > it is
> > > > > > > > > >>>>>> not
> > > > > > > > > >>>>>>>>> enough for just that batch to fail. I guess I
> > hadn't
> > > > > > > considered
> > > > > > > > > >> fully
> > > > > > > > > >>>>>>>>> fencing the old producer but there are valid
> > > arguments
> > > > > here
> > > > > > > > why we
> > > > > > > > > >>>>>>> would
> > > > > > > > > >>>>>>>>> want to.
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>> Thanks,
> > > > > > > > > >>>>>>>>> Justine
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>> On Fri, Jan 20, 2023 at 2:35 PM Guozhang Wang <
> > > > > > > > > >>>>>>>> guozhang.wang.us@gmail.com>
> > > > > > > > > >>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>>> Thanks Justine for the replies! I agree with
> most
> > of
> > > > > your
> > > > > > > > > >> thoughts.
> > > > > > > > > >>>>>>>>>>
> > > > > > > > > >>>>>>>>>> Just for 3/7), though I agree for our own AK
> > > producer,
> > > > > since
> > > > > > > > we do
> > > > > > > > > >>>>>>>>>> "nextRequest(boolean hasIncompleteBatches)", we
> > > > > guarantee
> > > > > > > the
> > > > > > > > > >>>>>> end-txn
> > > > > > > > > >>>>>>>>>> would not be sent until we've effectively
> flushed,
> > > but
> > > > > I was
> > > > > > > > > >>>>>>> referring
> > > > > > > > > >>>>>>>>>> to any future bugs or other buggy clients that
> the
> > > > same
> > > > > > > > client may
> > > > > > > > > >>>>>>> get
> > > > > > > > > >>>>>>>>>> into this situation, in which case we should
> give
> > > the
> > > > > > > client a
> > > > > > > > > >>>>>> clear
> > > > > > > > > >>>>>>>>>> msg that "you did something wrong, and hence now
> > you
> > > > > should
> > > > > > > > > >> fatally
> > > > > > > > > >>>>>>>>>> close yourself". What I'm concerned about is
> that,
> > > by
> > > > > seeing
> > > > > > > > an
> > > > > > > > > >>>>>>>>>> "abortable error" or in some rare cases an
> > "invalid
> > > > > record",
> > > > > > > > the
> > > > > > > > > >>>>>>>>>> client could not realize "something that's
> really
> > > bad
> > > > > > > > happened".
> > > > > > > > > >> So
> > > > > > > > > >>>>>>>>>> it's not about adding a new error, it's mainly
> > about
> > > > > those
> > > > > > > > real
> > > > > > > > > >>>>>> buggy
> > > > > > > > > >>>>>>>>>> situations causing such "should never happen"
> > cases,
> > > > the
> > > > > > > > errors
> > > > > > > > > >>>>>>> return
> > > > > > > > > >>>>>>>>>> would not be informative enough.
> > > > > > > > > >>>>>>>>>>
> > > > > > > > > >>>>>>>>>> Thinking in other ways, if we believe that for
> > most
> > > > > cases
> > > > > > > such
> > > > > > > > > >>>>>> error
> > > > > > > > > >>>>>>>>>> codes would not reach the original clients since
> > > they
> > > > > would
> > > > > > > be
> > > > > > > > > >>>>>>>>>> disconnected or even gone by that time, and only
> > in
> > > > some
> > > > > > > rare
> > > > > > > > > >> cases
> > > > > > > > > >>>>>>>>>> they would still be seen by the sending clients,
> > > then
> > > > > why
> > > > > > > not
> > > > > > > > make
> > > > > > > > > >>>>>>>>>> them more fatal and more specific than generic.
> > > > > > > > > >>>>>>>>>>
> > > > > > > > > >>>>>>>>>> Guozhang
> > > > > > > > > >>>>>>>>>>
> > > > > > > > > >>>>>>>>>> On Fri, Jan 20, 2023 at 1:59 PM Justine Olshan
> > > > > > > > > >>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > > > > > > > > >>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>> Hey Guozhang. Thanks for taking a look and for
> > the
> > > > > detailed
> > > > > > > > > >>>>>>> comments!
> > > > > > > > > >>>>>>>>>> I'll
> > > > > > > > > >>>>>>>>>>> do my best to address below.
> > > > > > > > > >>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>> 1. I see what you are saying here, but I think
> I
> > > need
> > > > > to
> > > > > > > look
> > > > > > > > > >>>>>>> through
> > > > > > > > > >>>>>>>>> the
> > > > > > > > > >>>>>>>>>>> sequence of events you mention. Typically we've
> > > seen
> > > > > this
> > > > > > > > issue
> > > > > > > > > >>>>>> in
> > > > > > > > > >>>>>>> a
> > > > > > > > > >>>>>>>>> few
> > > > > > > > > >>>>>>>>>>> cases.
> > > > > > > > > >>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>     One is when we have a producer disconnect
> > when
> > > > > trying
> > > > > > > to
> > > > > > > > > >>>>>> produce.
> > > > > > > > > >>>>>>>>>>> Typically in these cases, we abort the
> > transaction.
> > > > > We've
> > > > > > > > seen
> > > > > > > > > >>>>>> that
> > > > > > > > > >>>>>>>>> after
> > > > > > > > > >>>>>>>>>>> the markers are written, the disconnection can
> > > > > sometimes
> > > > > > > > cause
> > > > > > > > > >>>>>> the
> > > > > > > > > >>>>>>>>>> request
> > > > > > > > > >>>>>>>>>>> to get flushed to the broker. In this case, we
> > > don't
> > > > > need
> > > > > > > > client
> > > > > > > > > >>>>>>>>> handling
> > > > > > > > > >>>>>>>>>>> because the producer we are responding to is
> > gone.
> > > We
> > > > > just
> > > > > > > > needed
> > > > > > > > > >>>>>>> to
> > > > > > > > > >>>>>>>>> make
> > > > > > > > > >>>>>>>>>>> sure we didn't write to the log on the broker
> > side.
> > > > I'm
> > > > > > > > trying to
> > > > > > > > > >>>>>>>> think
> > > > > > > > > >>>>>>>>>> of
> > > > > > > > > >>>>>>>>>>> a case where we do have the client to return
> to.
> > > I'd
> > > > > think
> > > > > > > > the
> > > > > > > > > >>>>>> same
> > > > > > > > > >>>>>>>>>> client
> > > > > > > > > >>>>>>>>>>> couldn't progress to committing the transaction
> > > > unless
> > > > > the
> > > > > > > > > >>>>>> produce
> > > > > > > > > >>>>>>>>>> request
> > > > > > > > > >>>>>>>>>>> returned right? Of course, there is the
> > incorrectly
> > > > > written
> > > > > > > > > >>>>>> clients
> > > > > > > > > >>>>>>>>> case.
> > > > > > > > > >>>>>>>>>>> I'll think on this a bit more and let you know
> > if I
> > > > > come up
> > > > > > > > with
> > > > > > > > > >>>>>>>>> another
> > > > > > > > > >>>>>>>>>>> scenario when we would return to an active
> client
> > > > when
> > > > > the
> > > > > > > > > >>>>>>>> transaction
> > > > > > > > > >>>>>>>>> is
> > > > > > > > > >>>>>>>>>>> no longer ongoing.
> > > > > > > > > >>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>> I was not aware that we checked the result of a
> > > send
> > > > > after
> > > > > > > we
> > > > > > > > > >>>>>>> commit
> > > > > > > > > >>>>>>>>>>> though. I'll need to look into that a bit more.
> > > > > > > > > >>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>> 2. There were some questions about this in the
> > > > > discussion.
> > > > > > > > The
> > > > > > > > > >>>>>> plan
> > > > > > > > > >>>>>>>> is
> > > > > > > > > >>>>>>>>> to
> > > > > > > > > >>>>>>>>>>> handle overflow with the mechanism we currently
> > > have
> > > > > in the
> > > > > > > > > >>>>>>> producer.
> > > > > > > > > >>>>>>>>> If
> > > > > > > > > >>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>> try to bump and the epoch will overflow, we
> > > actually
> > > > > > > > allocate a
> > > > > > > > > >>>>>> new
> > > > > > > > > >>>>>>>>>>> producer ID. I need to confirm the fencing
> logic
> > on
> > > > the
> > > > > > > last
> > > > > > > > > >>>>>> epoch
> > > > > > > > > >>>>>>>> (ie,
> > > > > > > > > >>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>> probably shouldn't allow any records to be
> > produced
> > > > > with
> > > > > > > the
> > > > > > > > > >>>>>> final
> > > > > > > > > >>>>>>>>> epoch
> > > > > > > > > >>>>>>>>>>> since we can never properly fence that one).
> > > > > > > > > >>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>> 3. I can agree with you that the current error
> > > > > handling is
> > > > > > > > > >>>>>> messy. I
> > > > > > > > > >>>>>>>>>> recall
> > > > > > > > > >>>>>>>>>>> taking a look at your KIP a while back, but I
> > > think I
> > > > > > > mostly
> > > > > > > > saw
> > > > > > > > > >>>>>>> the
> > > > > > > > > >>>>>>>>>>> section about how the errors were wrapped.
> Maybe
> > I
> > > > > need to
> > > > > > > > take
> > > > > > > > > >>>>>>>> another
> > > > > > > > > >>>>>>>>>>> look. As for abortable error, the idea was that
> > the
> > > > > > > handling
> > > > > > > > > >>>>>> would
> > > > > > > > > >>>>>>> be
> > > > > > > > > >>>>>>>>>>> simple -- if this error is seen, the
> transaction
> > > > > should be
> > > > > > > > > >>>>>> aborted
> > > > > > > > > >>>>>>> --
> > > > > > > > > >>>>>>>>> no
> > > > > > > > > >>>>>>>>>>> other logic about previous state or requests
> > > > > necessary. Is
> > > > > > > > your
> > > > > > > > > >>>>>>>> concern
> > > > > > > > > >>>>>>>>>>> simply about adding new errors? We were hoping
> to
> > > > have
> > > > > an
> > > > > > > > error
> > > > > > > > > >>>>>>> that
> > > > > > > > > >>>>>>>>>> would
> > > > > > > > > >>>>>>>>>>> have one meaning and many of the current errors
> > > have
> > > > a
> > > > > > > > history of
> > > > > > > > > >>>>>>>>> meaning
> > > > > > > > > >>>>>>>>>>> different things on different client versions.
> > That
> > > > > was the
> > > > > > > > main
> > > > > > > > > >>>>>>>>>> motivation
> > > > > > > > > >>>>>>>>>>> for adding a new error.
> > > > > > > > > >>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>> 4. This is a good point about record timestamp
> > > > > reordering.
> > > > > > > > > >>>>>>> Timestamps
> > > > > > > > > >>>>>>>>>> don't
> > > > > > > > > >>>>>>>>>>> affect compaction, but they do affect retention
> > > > > deletion.
> > > > > > > For
> > > > > > > > > >>>>>> that,
> > > > > > > > > >>>>>>>>> kafka
> > > > > > > > > >>>>>>>>>>> considers the largest timestamp in the segment,
> > so
> > > I
> > > > > think
> > > > > > > a
> > > > > > > > > >>>>>> small
> > > > > > > > > >>>>>>>>> amount
> > > > > > > > > >>>>>>>>>>> of reordering (hopefully on the order of
> > > milliseconds
> > > > > or
> > > > > > > even
> > > > > > > > > >>>>>>>> seconds)
> > > > > > > > > >>>>>>>>>> will
> > > > > > > > > >>>>>>>>>>> be ok. We take timestamps from clients so there
> > is
> > > > > already
> > > > > > > a
> > > > > > > > > >>>>>>>>> possibility
> > > > > > > > > >>>>>>>>>>> for some drift and non-monotonically increasing
> > > > > timestamps.
> > > > > > > > > >>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>> 5. Thanks for catching. The error is there, but
> > > it's
> > > > > > > actually
> > > > > > > > > >>>>>> that
> > > > > > > > > >>>>>>>>> those
> > > > > > > > > >>>>>>>>>>> fields should be 4+! Due to how the message
> > > generator
> > > > > > > works,
> > > > > > > > I
> > > > > > > > > >>>>>>>> actually
> > > > > > > > > >>>>>>>>>>> have to redefine those fields inside the
> > > > > > > > > >>>>>>>>> `"AddPartitionsToTxnTransaction`
> > > > > > > > > >>>>>>>>>>> block for it to build correctly. I'll fix it to
> > be
> > > > > correct.
> > > > > > > > > >>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>> 6. Correct -- we will only add the request to
> > > > > purgatory if
> > > > > > > > the
> > > > > > > > > >>>>>>> cache
> > > > > > > > > >>>>>>>>> has
> > > > > > > > > >>>>>>>>>> no
> > > > > > > > > >>>>>>>>>>> ongoing transaction. I can change the wording
> to
> > > make
> > > > > that
> > > > > > > > > >>>>>> clearer
> > > > > > > > > >>>>>>>> that
> > > > > > > > > >>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>> only place the request in purgatory if we need
> to
> > > > > contact
> > > > > > > the
> > > > > > > > > >>>>>>>>> transaction
> > > > > > > > > >>>>>>>>>>> coordinator.
> > > > > > > > > >>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>> 7. We did take a look at some of the errors and
> > it
> > > > was
> > > > > hard
> > > > > > > > to
> > > > > > > > > >>>>>> come
> > > > > > > > > >>>>>>>> up
> > > > > > > > > >>>>>>>>>> with
> > > > > > > > > >>>>>>>>>>> a good one. I agree that
> InvalidTxnStateException
> > > is
> > > > > ideal
> > > > > > > > except
> > > > > > > > > >>>>>>> for
> > > > > > > > > >>>>>>>>> the
> > > > > > > > > >>>>>>>>>>> fact that it hasn't been returned on Produce
> > > requests
> > > > > > > > before. The
> > > > > > > > > >>>>>>>> error
> > > > > > > > > >>>>>>>>>>> handling for clients is a bit vague (which is
> > why I
> > > > > opened
> > > > > > > > > >>>>>>>> KAFKA-14439
> > > > > > > > > >>>>>>>>>>> <
> > https://issues.apache.org/jira/browse/KAFKA-14439
> > > > >),
> > > > > but
> > > > > > > > the
> > > > > > > > > >>>>>>>> decision
> > > > > > > > > >>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>> made here was to only return errors that have
> > been
> > > > > > > previously
> > > > > > > > > >>>>>>>> returned
> > > > > > > > > >>>>>>>>> to
> > > > > > > > > >>>>>>>>>>> producers. As for not being fatal, I think part
> > of
> > > > the
> > > > > > > > theory was
> > > > > > > > > >>>>>>>> that
> > > > > > > > > >>>>>>>>> in
> > > > > > > > > >>>>>>>>>>> many cases, the producer would be disconnected.
> > > (See
> > > > > point
> > > > > > > > 1) and
> > > > > > > > > >>>>>>>> this
> > > > > > > > > >>>>>>>>>>> would just be an error to return from the
> > server. I
> > > > did
> > > > > > > plan
> > > > > > > > to
> > > > > > > > > >>>>>>> think
> > > > > > > > > >>>>>>>>>> about
> > > > > > > > > >>>>>>>>>>> other cases, so let me know if you think of any
> > as
> > > > > well!
> > > > > > > > > >>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>> Lots to say! Let me know if you have further
> > > > thoughts!
> > > > > > > > > >>>>>>>>>>> Justine
> > > > > > > > > >>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>> On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang
> <
> > > > > > > > > >>>>>>>>>> guozhang.wang.us@gmail.com>
> > > > > > > > > >>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>> Hello Justine,
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>> Thanks for the great write-up! I made a quick
> > pass
> > > > > through
> > > > > > > > it
> > > > > > > > > >>>>>> and
> > > > > > > > > >>>>>>>>> here
> > > > > > > > > >>>>>>>>>>>> are some thoughts (I have not been able to
> read
> > > > > through
> > > > > > > this
> > > > > > > > > >>>>>>> thread
> > > > > > > > > >>>>>>>>> so
> > > > > > > > > >>>>>>>>>>>> pardon me if they have overlapped or subsumed
> by
> > > > > previous
> > > > > > > > > >>>>>>>> comments):
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>> First are some meta ones:
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>> 1. I think we need to also improve the
> client's
> > > > > experience
> > > > > > > > once
> > > > > > > > > >>>>>>> we
> > > > > > > > > >>>>>>>>>>>> have this defence in place. More concretely,
> > say a
> > > > > user's
> > > > > > > > > >>>>>>> producer
> > > > > > > > > >>>>>>>>>>>> code is like following:
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>> future = producer.send();
> > > > > > > > > >>>>>>>>>>>> // producer.flush();
> > > > > > > > > >>>>>>>>>>>> producer.commitTransaction();
> > > > > > > > > >>>>>>>>>>>> future.get();
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>> Which resulted in the order of a)
> > produce-request
> > > > > sent by
> > > > > > > > > >>>>>>> producer,
> > > > > > > > > >>>>>>>>> b)
> > > > > > > > > >>>>>>>>>>>> end-txn-request sent by producer, c)
> > > > end-txn-response
> > > > > sent
> > > > > > > > > >>>>>> back,
> > > > > > > > > >>>>>>> d)
> > > > > > > > > >>>>>>>>>>>> txn-marker-request sent from coordinator to
> > > > partition
> > > > > > > > leader,
> > > > > > > > > >>>>>> e)
> > > > > > > > > >>>>>>>>>>>> produce-request finally received by the
> > partition
> > > > > leader,
> > > > > > > > > >>>>>> before
> > > > > > > > > >>>>>>>> this
> > > > > > > > > >>>>>>>>>>>> KIP e) step would be accepted causing a
> dangling
> > > > txn;
> > > > > now
> > > > > > > it
> > > > > > > > > >>>>>>> would
> > > > > > > > > >>>>>>>> be
> > > > > > > > > >>>>>>>>>>>> rejected in step e) which is good. But from
> the
> > > > > client's
> > > > > > > > point
> > > > > > > > > >>>>>> of
> > > > > > > > > >>>>>>>>> view
> > > > > > > > > >>>>>>>>>>>> now it becomes confusing since the
> > > > > `commitTransaction()`
> > > > > > > > > >>>>>> returns
> > > > > > > > > >>>>>>>>>>>> successfully, but the "future" throws an
> > > > invalid-epoch
> > > > > > > > error,
> > > > > > > > > >>>>>> and
> > > > > > > > > >>>>>>>>> they
> > > > > > > > > >>>>>>>>>>>> are not sure if the transaction did succeed or
> > > not.
> > > > In
> > > > > > > > fact, it
> > > > > > > > > >>>>>>>>>>>> "partially succeeded" with some msgs being
> > > rejected
> > > > > but
> > > > > > > > others
> > > > > > > > > >>>>>>>>>>>> committed successfully.
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>> Of course the easy way to avoid this is,
> always
> > > call
> > > > > > > > > >>>>>>>>>>>> "producer.flush()" before commitTxn and that's
> > > what
> > > > > we do
> > > > > > > > > >>>>>>>> ourselves,
> > > > > > > > > >>>>>>>>>>>> and what we recommend users do. But I suspect
> > not
> > > > > everyone
> > > > > > > > does
> > > > > > > > > >>>>>>> it.
> > > > > > > > > >>>>>>>>> In
> > > > > > > > > >>>>>>>>>>>> fact I just checked the javadoc in
> KafkaProducer
> > > and
> > > > > our
> > > > > > > > code
> > > > > > > > > >>>>>>>> snippet
> > > > > > > > > >>>>>>>>>>>> does not include a `flush()` call. So I'm
> > thinking
> > > > > maybe
> > > > > > > we
> > > > > > > > can
> > > > > > > > > >>>>>>> in
> > > > > > > > > >>>>>>>>>>>> side the `commitTxn` code to enforce flushing
> > > before
> > > > > > > sending
> > > > > > > > > >>>>>> the
> > > > > > > > > >>>>>>>>>>>> end-txn request.
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>> 2. I'd like to clarify a bit details on "just
> > add
> > > > > > > > partitions to
> > > > > > > > > >>>>>>> the
> > > > > > > > > >>>>>>>>>>>> transaction on the first produce request
> during
> > a
> > > > > > > > transaction".
> > > > > > > > > >>>>>>> My
> > > > > > > > > >>>>>>>>>>>> understanding is that the partition leader's
> > cache
> > > > > has the
> > > > > > > > > >>>>>>> producer
> > > > > > > > > >>>>>>>>> id
> > > > > > > > > >>>>>>>>>>>> / sequence / epoch for the latest txn, either
> > > > > on-going or
> > > > > > > is
> > > > > > > > > >>>>>>>>> completed
> > > > > > > > > >>>>>>>>>>>> (upon receiving the marker request from
> > > > coordinator).
> > > > > > > When a
> > > > > > > > > >>>>>>>> produce
> > > > > > > > > >>>>>>>>>>>> request is received, if
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>> * producer's epoch < cached epoch, or
> producer's
> > > > > epoch ==
> > > > > > > > > >>>>>> cached
> > > > > > > > > >>>>>>>>> epoch
> > > > > > > > > >>>>>>>>>>>> but the latest txn is completed, leader
> directly
> > > > > reject
> > > > > > > with
> > > > > > > > > >>>>>>>>>>>> invalid-epoch.
> > > > > > > > > >>>>>>>>>>>> * producer's epoch > cached epoch, park the
> the
> > > > > request
> > > > > > > and
> > > > > > > > > >>>>>> send
> > > > > > > > > >>>>>>>>>>>> add-partitions request to coordinator.
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>> In order to do it, does the coordinator need
> to
> > > bump
> > > > > the
> > > > > > > > > >>>>>> sequence
> > > > > > > > > >>>>>>>> and
> > > > > > > > > >>>>>>>>>>>> reset epoch to 0 when the next epoch is going
> to
> > > > > overflow?
> > > > > > > > If
> > > > > > > > > >>>>>> no
> > > > > > > > > >>>>>>>> need
> > > > > > > > > >>>>>>>>>>>> to do so, then how we handle the (admittedly
> > rare,
> > > > but
> > > > > > > still
> > > > > > > > > >>>>>> may
> > > > > > > > > >>>>>>>>>>>> happen) epoch overflow situation?
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>> 3. I'm a bit concerned about adding a generic
> > > > > > > > "ABORTABLE_ERROR"
> > > > > > > > > >>>>>>>> given
> > > > > > > > > >>>>>>>>>>>> we already have a pretty messy error
> > > classification
> > > > > and
> > > > > > > > error
> > > > > > > > > >>>>>>>>> handling
> > > > > > > > > >>>>>>>>>>>> on the producer clients side --- I have a
> > summary
> > > > > about
> > > > > > > the
> > > > > > > > > >>>>>>> issues
> > > > > > > > > >>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>> a proposal to address this in
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>
> > > > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
> > > > > > > > > >>>>>>>>>>>> -- I understand we do not want to use
> > > > > > > "UNKNOWN_PRODUCER_ID"
> > > > > > > > > >>>>>>> anymore
> > > > > > > > > >>>>>>>>>>>> and in fact we intend to deprecate it in
> KIP-360
> > > and
> > > > > > > > eventually
> > > > > > > > > >>>>>>>>> remove
> > > > > > > > > >>>>>>>>>>>> it; but I'm wondering can we still use
> specific
> > > > error
> > > > > > > codes.
> > > > > > > > > >>>>>> E.g.
> > > > > > > > > >>>>>>>>> what
> > > > > > > > > >>>>>>>>>>>> about "InvalidProducerEpochException" since
> for
> > > new
> > > > > > > clients,
> > > > > > > > > >>>>>> the
> > > > > > > > > >>>>>>>>>>>> actual reason this would actually be rejected
> is
> > > > > indeed
> > > > > > > > because
> > > > > > > > > >>>>>>> the
> > > > > > > > > >>>>>>>>>>>> epoch on the coordinator caused the
> > > > > add-partitions-request
> > > > > > > > from
> > > > > > > > > >>>>>>> the
> > > > > > > > > >>>>>>>>>>>> brokers to be rejected anyways?
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>> 4. It seems we put the producer request into
> > > > purgatory
> > > > > > > > before
> > > > > > > > > >>>>>> we
> > > > > > > > > >>>>>>>> ever
> > > > > > > > > >>>>>>>>>>>> append the records, while other producer's
> > records
> > > > may
> > > > > > > > still be
> > > > > > > > > >>>>>>>>>>>> appended during the time; and that potentially
> > may
> > > > > result
> > > > > > > in
> > > > > > > > > >>>>>> some
> > > > > > > > > >>>>>>>>>>>> re-ordering compared with reception order. I'm
> > not
> > > > > super
> > > > > > > > > >>>>>>> concerned
> > > > > > > > > >>>>>>>>>>>> about it since Kafka does not guarantee
> > reception
> > > > > ordering
> > > > > > > > > >>>>>> across
> > > > > > > > > >>>>>>>>>>>> producers anyways, but it may make the
> > timestamps
> > > of
> > > > > > > records
> > > > > > > > > >>>>>>>> inside a
> > > > > > > > > >>>>>>>>>>>> partition to be more out-of-ordered. Are we
> > aware
> > > of
> > > > > any
> > > > > > > > > >>>>>>> scenarios
> > > > > > > > > >>>>>>>>>>>> such as future enhancements on log compactions
> > > that
> > > > > may be
> > > > > > > > > >>>>>>> affected
> > > > > > > > > >>>>>>>>> by
> > > > > > > > > >>>>>>>>>>>> this effect?
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>> Below are just minor comments:
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>> 5. In "AddPartitionsToTxnTransaction" field of
> > > > > > > > > >>>>>>>>>>>> "AddPartitionsToTxnRequest" RPC, the versions
> of
> > > > those
> > > > > > > inner
> > > > > > > > > >>>>>>> fields
> > > > > > > > > >>>>>>>>>>>> are "0-3" while I thought they should be "0+"
> > > still?
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>> 6. Regarding "we can place the request in a
> > > > purgatory
> > > > > of
> > > > > > > > sorts
> > > > > > > > > >>>>>>> and
> > > > > > > > > >>>>>>>>>>>> check if there is any state for the
> transaction
> > on
> > > > the
> > > > > > > > > >>>>>> broker": i
> > > > > > > > > >>>>>>>>>>>> think at this time when we just do the checks
> > > > against
> > > > > the
> > > > > > > > > >>>>>> cached
> > > > > > > > > >>>>>>>>>>>> state, we do not need to put the request to
> > > > purgatory
> > > > > yet?
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>> 7. This is related to 3) above. I feel using
> > > > > > > > > >>>>>>>> "InvalidRecordException"
> > > > > > > > > >>>>>>>>>>>> for older clients may also be a bit confusing,
> > and
> > > > > also it
> > > > > > > > is
> > > > > > > > > >>>>>> not
> > > > > > > > > >>>>>>>>>>>> fatal -- for old clients, it better to be
> fatal
> > > > since
> > > > > this
> > > > > > > > > >>>>>>>> indicates
> > > > > > > > > >>>>>>>>>>>> the clients is doing something wrong and hence
> > it
> > > > > should
> > > > > > > be
> > > > > > > > > >>>>>>> closed.
> > > > > > > > > >>>>>>>>>>>> And in general I'd prefer to use slightly more
> > > > > specific
> > > > > > > > meaning
> > > > > > > > > >>>>>>>> error
> > > > > > > > > >>>>>>>>>>>> codes for clients. That being said, I also
> feel
> > > > > > > > > >>>>>>>>>>>> "InvalidProducerEpochException" is not
> suitable
> > > for
> > > > > old
> > > > > > > > > >>>>>> versioned
> > > > > > > > > >>>>>>>>>>>> clients, and we'd have to pick one that old
> > > clients
> > > > > > > > recognize.
> > > > > > > > > >>>>>>> I'd
> > > > > > > > > >>>>>>>>>>>> prefer "InvalidTxnStateException" but that one
> > is
> > > > > supposed
> > > > > > > > to
> > > > > > > > > >>>>>> be
> > > > > > > > > >>>>>>>>>>>> returned from txn coordinators only today. I'd
> > > > > suggest we
> > > > > > > > do a
> > > > > > > > > >>>>>>>> quick
> > > > > > > > > >>>>>>>>>>>> check in the current client's code path and
> see
> > if
> > > > > that
> > > > > > > one
> > > > > > > > > >>>>>> would
> > > > > > > > > >>>>>>>> be
> > > > > > > > > >>>>>>>>>>>> handled if it's from a produce-response, and
> if
> > > yes,
> > > > > use
> > > > > > > > this
> > > > > > > > > >>>>>>> one;
> > > > > > > > > >>>>>>>>>>>> otherwise, use "ProducerFencedException" which
> > is
> > > > much
> > > > > > > less
> > > > > > > > > >>>>>>>>> meaningful
> > > > > > > > > >>>>>>>>>>>> but it's still a fatal error.
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>> Thanks,
> > > > > > > > > >>>>>>>>>>>> Guozhang
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>> On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
> > > > > > > > > >>>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > > > > > > > > >>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>> Yeah -- looks like we already have code to
> > handle
> > > > > bumping
> > > > > > > > the
> > > > > > > > > >>>>>>>> epoch
> > > > > > > > > >>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>> when the epoch is Short.MAX_VALUE, we get a
> new
> > > > > producer
> > > > > > > > ID.
> > > > > > > > > >>>>>>>> Since
> > > > > > > > > >>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>> already the behavior, do we want to change it
> > > > > further?
> > > > > > > > > >>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>> Justine
> > > > > > > > > >>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>> On Wed, Jan 18, 2023 at 1:12 PM Justine
> Olshan
> > <
> > > > > > > > > >>>>>>>>> jolshan@confluent.io
> > > > > > > > > >>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>> Hey all, just wanted to quickly update and
> say
> > > > I've
> > > > > > > > > >>>>>> modified
> > > > > > > > > >>>>>>>> the
> > > > > > > > > >>>>>>>>>> KIP to
> > > > > > > > > >>>>>>>>>>>>>> explicitly mention that
> > > > AddOffsetCommitsToTxnRequest
> > > > > > > will
> > > > > > > > > >>>>>> be
> > > > > > > > > >>>>>>>>>> replaced
> > > > > > > > > >>>>>>>>>>>> by
> > > > > > > > > >>>>>>>>>>>>>> a coordinator-side (inter-broker)
> > > > AddPartitionsToTxn
> > > > > > > > > >>>>>> implicit
> > > > > > > > > >>>>>>>>>> request.
> > > > > > > > > >>>>>>>>>>>> This
> > > > > > > > > >>>>>>>>>>>>>> mirrors the user partitions and will
> > implicitly
> > > > add
> > > > > > > offset
> > > > > > > > > >>>>>>>>>> partitions
> > > > > > > > > >>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>> transactions when we commit offsets on them.
> > We
> > > > will
> > > > > > > > > >>>>>>> deprecate
> > > > > > > > > >>>>>>>>>>>> AddOffsetCommitsToTxnRequest
> > > > > > > > > >>>>>>>>>>>>>> for new clients.
> > > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>> Also to address Artem's comments --
> > > > > > > > > >>>>>>>>>>>>>> I'm a bit unsure if the changes here will
> > change
> > > > the
> > > > > > > > > >>>>>> previous
> > > > > > > > > >>>>>>>>>> behavior
> > > > > > > > > >>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>> fencing producers. In the case you mention
> in
> > > the
> > > > > first
> > > > > > > > > >>>>>>>>> paragraph,
> > > > > > > > > >>>>>>>>>> are
> > > > > > > > > >>>>>>>>>>>> you
> > > > > > > > > >>>>>>>>>>>>>> saying we bump the epoch before we try to
> > abort
> > > > the
> > > > > > > > > >>>>>>>> transaction?
> > > > > > > > > >>>>>>>>> I
> > > > > > > > > >>>>>>>>>>>> think I
> > > > > > > > > >>>>>>>>>>>>>> need to understand the scenarios you
> mention a
> > > bit
> > > > > > > better.
> > > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>> As for the second part -- I think it makes
> > sense
> > > > to
> > > > > have
> > > > > > > > > >>>>>> some
> > > > > > > > > >>>>>>>>> sort
> > > > > > > > > >>>>>>>>>> of
> > > > > > > > > >>>>>>>>>>>>>> "sentinel" epoch to signal epoch is about to
> > > > > overflow (I
> > > > > > > > > >>>>>>> think
> > > > > > > > > >>>>>>>> we
> > > > > > > > > >>>>>>>>>> sort
> > > > > > > > > >>>>>>>>>>>> of
> > > > > > > > > >>>>>>>>>>>>>> have this value in place in some ways) so we
> > can
> > > > > codify
> > > > > > > it
> > > > > > > > > >>>>>> in
> > > > > > > > > >>>>>>>> the
> > > > > > > > > >>>>>>>>>> KIP.
> > > > > > > > > >>>>>>>>>>>> I'll
> > > > > > > > > >>>>>>>>>>>>>> look into that and try to update soon.
> > > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>> Thanks,
> > > > > > > > > >>>>>>>>>>>>>> Justine.
> > > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>> On Fri, Jan 13, 2023 at 5:01 PM Artem
> Livshits
> > > > > > > > > >>>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
> > > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>> It's good to know that KIP-588 addressed
> some
> > > of
> > > > > the
> > > > > > > > > >>>>>> issues.
> > > > > > > > > >>>>>>>>>> Looking
> > > > > > > > > >>>>>>>>>>>> at
> > > > > > > > > >>>>>>>>>>>>>>> the code, it still looks like there are
> some
> > > > cases
> > > > > that
> > > > > > > > > >>>>>>> would
> > > > > > > > > >>>>>>>>>> result
> > > > > > > > > >>>>>>>>>>>> in
> > > > > > > > > >>>>>>>>>>>>>>> fatal error, e.g. PRODUCER_FENCED is issued
> > by
> > > > the
> > > > > > > > > >>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>> coordinator
> > > > > > > > > >>>>>>>>>>>>>>> if epoch doesn't match, and the client
> treats
> > > it
> > > > > as a
> > > > > > > > > >>>>>> fatal
> > > > > > > > > >>>>>>>>> error
> > > > > > > > > >>>>>>>>>>>> (code in
> > > > > > > > > >>>>>>>>>>>>>>> TransactionManager request handling).  If
> we
> > > > > consider,
> > > > > > > > for
> > > > > > > > > >>>>>>>>>> example,
> > > > > > > > > >>>>>>>>>>>>>>> committing a transaction that returns a
> > > timeout,
> > > > > but
> > > > > > > > > >>>>>>> actually
> > > > > > > > > >>>>>>>>>>>> succeeds,
> > > > > > > > > >>>>>>>>>>>>>>> trying to abort it or re-commit may result
> in
> > > > > > > > > >>>>>>> PRODUCER_FENCED
> > > > > > > > > >>>>>>>>>> error
> > > > > > > > > >>>>>>>>>>>>>>> (because of epoch bump).
> > > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>> For failed commits, specifically, we need
> to
> > > know
> > > > > the
> > > > > > > > > >>>>>> actual
> > > > > > > > > >>>>>>>>>> outcome,
> > > > > > > > > >>>>>>>>>>>>>>> because if we return an error the
> application
> > > may
> > > > > think
> > > > > > > > > >>>>>> that
> > > > > > > > > >>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>> transaction is aborted and redo the work,
> > > leading
> > > > > to
> > > > > > > > > >>>>>>>> duplicates.
> > > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>> Re: overflowing epoch.  We could either do
> it
> > > on
> > > > > the TC
> > > > > > > > > >>>>>> and
> > > > > > > > > >>>>>>>>> return
> > > > > > > > > >>>>>>>>>>>> both
> > > > > > > > > >>>>>>>>>>>>>>> producer id and epoch (e.g. change the
> > > protocol),
> > > > > or
> > > > > > > > > >>>>>> signal
> > > > > > > > > >>>>>>>> the
> > > > > > > > > >>>>>>>>>> client
> > > > > > > > > >>>>>>>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>>>>> it needs to get a new producer id.
> Checking
> > > for
> > > > > max
> > > > > > > > epoch
> > > > > > > > > >>>>>>>> could
> > > > > > > > > >>>>>>>>>> be a
> > > > > > > > > >>>>>>>>>>>>>>> reasonable signal, the value to check
> should
> > > > > probably
> > > > > > > be
> > > > > > > > > >>>>>>>> present
> > > > > > > > > >>>>>>>>>> in
> > > > > > > > > >>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>> KIP
> > > > > > > > > >>>>>>>>>>>>>>> as this is effectively a part of the
> > contract.
> > > > > Also,
> > > > > > > the
> > > > > > > > > >>>>>> TC
> > > > > > > > > >>>>>>>>>> should
> > > > > > > > > >>>>>>>>>>>>>>> probably return an error if the client
> didn't
> > > > > change
> > > > > > > > > >>>>>>> producer
> > > > > > > > > >>>>>>>> id
> > > > > > > > > >>>>>>>>>> after
> > > > > > > > > >>>>>>>>>>>>>>> hitting max epoch.
> > > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>> -Artem
> > > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>> On Thu, Jan 12, 2023 at 10:31 AM Justine
> > Olshan
> > > > > > > > > >>>>>>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>> Thanks for the discussion Artem.
> > > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>> With respect to the handling of fenced
> > > > producers,
> > > > > we
> > > > > > > > > >>>>>> have
> > > > > > > > > >>>>>>>> some
> > > > > > > > > >>>>>>>>>>>> behavior
> > > > > > > > > >>>>>>>>>>>>>>>> already in place. As of KIP-588:
> > > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>
> > > > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> > > > > > > > > >>>>>>>>>>>>>>>> ,
> > > > > > > > > >>>>>>>>>>>>>>>> we handle timeouts more gracefully. The
> > > producer
> > > > > can
> > > > > > > > > >>>>>>>> recover.
> > > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>> Produce requests can also recover from
> epoch
> > > > > fencing
> > > > > > > by
> > > > > > > > > >>>>>>>>>> aborting the
> > > > > > > > > >>>>>>>>>>>>>>>> transaction and starting over.
> > > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>> What other cases were you considering that
> > > would
> > > > > cause
> > > > > > > > > >>>>>> us
> > > > > > > > > >>>>>>> to
> > > > > > > > > >>>>>>>>>> have a
> > > > > > > > > >>>>>>>>>>>>>>> fenced
> > > > > > > > > >>>>>>>>>>>>>>>> epoch but we'd want to recover?
> > > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>> The first point about handling epoch
> > overflows
> > > > is
> > > > > > > fair.
> > > > > > > > > >>>>>> I
> > > > > > > > > >>>>>>>>> think
> > > > > > > > > >>>>>>>>>>>> there is
> > > > > > > > > >>>>>>>>>>>>>>>> some logic we'd need to consider. (ie, if
> we
> > > are
> > > > > one
> > > > > > > > > >>>>>> away
> > > > > > > > > >>>>>>>> from
> > > > > > > > > >>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>> max
> > > > > > > > > >>>>>>>>>>>>>>>> epoch, we need to reset the producer ID.)
> > I'm
> > > > > still
> > > > > > > > > >>>>>>>> wondering
> > > > > > > > > >>>>>>>>> if
> > > > > > > > > >>>>>>>>>>>> there
> > > > > > > > > >>>>>>>>>>>>>>> is a
> > > > > > > > > >>>>>>>>>>>>>>>> way to direct this from the response, or
> if
> > > > > everything
> > > > > > > > > >>>>>>>> should
> > > > > > > > > >>>>>>>>> be
> > > > > > > > > >>>>>>>>>>>> done on
> > > > > > > > > >>>>>>>>>>>>>>>> the client side. Let me know if you have
> any
> > > > > thoughts
> > > > > > > > > >>>>>>> here.
> > > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>> Thanks,
> > > > > > > > > >>>>>>>>>>>>>>>> Justine
> > > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>> On Tue, Jan 10, 2023 at 4:06 PM Artem
> > Livshits
> > > > > > > > > >>>>>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>> There are some workflows in the client
> that
> > > are
> > > > > > > > > >>>>>> implied
> > > > > > > > > >>>>>>> by
> > > > > > > > > >>>>>>>>>>>> protocol
> > > > > > > > > >>>>>>>>>>>>>>>>> changes, e.g.:
> > > > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>> - for new clients, epoch changes with
> every
> > > > > > > > > >>>>>> transaction
> > > > > > > > > >>>>>>>> and
> > > > > > > > > >>>>>>>>>> can
> > > > > > > > > >>>>>>>>>>>>>>> overflow,
> > > > > > > > > >>>>>>>>>>>>>>>>> in old clients this condition was handled
> > > > > > > > > >>>>>> transparently,
> > > > > > > > > >>>>>>>>>> because
> > > > > > > > > >>>>>>>>>>>> epoch
> > > > > > > > > >>>>>>>>>>>>>>>> was
> > > > > > > > > >>>>>>>>>>>>>>>>> bumped in InitProducerId and it would
> > return
> > > a
> > > > > new
> > > > > > > > > >>>>>>>> producer
> > > > > > > > > >>>>>>>>>> id if
> > > > > > > > > >>>>>>>>>>>>>>> epoch
> > > > > > > > > >>>>>>>>>>>>>>>>> overflows, the new clients would need to
> > > > > implement
> > > > > > > > > >>>>>> some
> > > > > > > > > >>>>>>>>>> workflow
> > > > > > > > > >>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>> refresh
> > > > > > > > > >>>>>>>>>>>>>>>>> producer id
> > > > > > > > > >>>>>>>>>>>>>>>>> - how to handle fenced producers, for new
> > > > clients
> > > > > > > > > >>>>>> epoch
> > > > > > > > > >>>>>>>>>> changes
> > > > > > > > > >>>>>>>>>>>> with
> > > > > > > > > >>>>>>>>>>>>>>>> every
> > > > > > > > > >>>>>>>>>>>>>>>>> transaction, so in presence of failures
> > > during
> > > > > > > > > >>>>>> commits /
> > > > > > > > > >>>>>>>>>> aborts,
> > > > > > > > > >>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>> producer could get easily fenced, old
> > clients
> > > > > would
> > > > > > > > > >>>>>>> pretty
> > > > > > > > > >>>>>>>>>> much
> > > > > > > > > >>>>>>>>>>>> would
> > > > > > > > > >>>>>>>>>>>>>>> get
> > > > > > > > > >>>>>>>>>>>>>>>>> fenced when a new incarnation of the
> > producer
> > > > was
> > > > > > > > > >>>>>>>>> initialized
> > > > > > > > > >>>>>>>>>> with
> > > > > > > > > >>>>>>>>>>>>>>>>> InitProducerId so it's ok to treat as a
> > fatal
> > > > > error,
> > > > > > > > > >>>>>> the
> > > > > > > > > >>>>>>>> new
> > > > > > > > > >>>>>>>>>>>> clients
> > > > > > > > > >>>>>>>>>>>>>>>> would
> > > > > > > > > >>>>>>>>>>>>>>>>> need to implement some workflow to handle
> > > that
> > > > > error,
> > > > > > > > > >>>>>>>>>> otherwise
> > > > > > > > > >>>>>>>>>>>> they
> > > > > > > > > >>>>>>>>>>>>>>>> could
> > > > > > > > > >>>>>>>>>>>>>>>>> get fenced by themselves
> > > > > > > > > >>>>>>>>>>>>>>>>> - in particular (as a subset of the
> > previous
> > > > > issue),
> > > > > > > > > >>>>>>> what
> > > > > > > > > >>>>>>>>>> would
> > > > > > > > > >>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>> client
> > > > > > > > > >>>>>>>>>>>>>>>>> do if it got a timeout during commit?
> > commit
> > > > > > > could've
> > > > > > > > > >>>>>>>>>> succeeded
> > > > > > > > > >>>>>>>>>>>> or
> > > > > > > > > >>>>>>>>>>>>>>>> failed
> > > > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>> Not sure if this has to be defined in the
> > KIP
> > > > as
> > > > > > > > > >>>>>>>>> implementing
> > > > > > > > > >>>>>>>>>>>> those
> > > > > > > > > >>>>>>>>>>>>>>>>> probably wouldn't require protocol
> changes,
> > > but
> > > > > we
> > > > > > > > > >>>>>> have
> > > > > > > > > >>>>>>>>>> multiple
> > > > > > > > > >>>>>>>>>>>>>>>>> implementations of Kafka clients, so
> > probably
> > > > > would
> > > > > > > be
> > > > > > > > > >>>>>>>> good
> > > > > > > > > >>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>> have
> > > > > > > > > >>>>>>>>>>>>>>> some
> > > > > > > > > >>>>>>>>>>>>>>>>> client implementation guidance.  Could
> also
> > > be
> > > > > done
> > > > > > > > > >>>>>> as a
> > > > > > > > > >>>>>>>>>> separate
> > > > > > > > > >>>>>>>>>>>> doc.
> > > > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>> -Artem
> > > > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>> On Mon, Jan 9, 2023 at 3:38 PM Justine
> > Olshan
> > > > > > > > > >>>>>>>>>>>>>>>> <jolshan@confluent.io.invalid
> > > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>> Hey all, I've updated the KIP to
> > incorporate
> > > > > Jason's
> > > > > > > > > >>>>>>>>>>>> suggestions.
> > > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>
> > > > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>> 1. Use AddPartitionsToTxn + verify flag
> to
> > > > > check on
> > > > > > > > > >>>>>>> old
> > > > > > > > > >>>>>>>>>> clients
> > > > > > > > > >>>>>>>>>>>>>>>>>> 2. Updated AddPartitionsToTxn API to
> > support
> > > > > > > > > >>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>> batching
> > > > > > > > > >>>>>>>>>>>>>>>>>> 3. Mention IBP bump
> > > > > > > > > >>>>>>>>>>>>>>>>>> 4. Mention auth change on new
> > > > AddPartitionsToTxn
> > > > > > > > > >>>>>>>> version.
> > > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>> I'm planning on opening a vote soon.
> > > > > > > > > >>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > > > >>>>>>>>>>>>>>>>>> Justine
> > > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:32 PM Justine
> > > Olshan
> > > > <
> > > > > > > > > >>>>>>>>>>>> jolshan@confluent.io
> > > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>> Thanks Jason. Those changes make sense
> to
> > > > me. I
> > > > > > > > > >>>>>> will
> > > > > > > > > >>>>>>>>>> update
> > > > > > > > > >>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>> KIP.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:31 PM Jason
> > > > Gustafson
> > > > > > > > > >>>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
> > > > > > > > > >>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> Hey Justine,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> I was wondering about compatibility
> > here.
> > > > > When
> > > > > > > > > >>>>>> we
> > > > > > > > > >>>>>>>>> send
> > > > > > > > > >>>>>>>>>>>> requests
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> between brokers, we want to ensure
> that
> > > the
> > > > > > > > > >>>>>>> receiving
> > > > > > > > > >>>>>>>>>> broker
> > > > > > > > > >>>>>>>>>>>>>>>>> understands
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> the request (specifically the new
> > fields).
> > > > > > > > > >>>>>>> Typically
> > > > > > > > > >>>>>>>>>> this is
> > > > > > > > > >>>>>>>>>>>> done
> > > > > > > > > >>>>>>>>>>>>>>>> via
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> IBP/metadata version.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> I'm trying to think if there is a way
> > > around
> > > > > it
> > > > > > > > > >>>>>> but
> > > > > > > > > >>>>>>>> I'm
> > > > > > > > > >>>>>>>>>> not
> > > > > > > > > >>>>>>>>>>>> sure
> > > > > > > > > >>>>>>>>>>>>>>>> there
> > > > > > > > > >>>>>>>>>>>>>>>>>> is.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> Yes. I think we would gate usage of
> this
> > > > > behind
> > > > > > > > > >>>>>> an
> > > > > > > > > >>>>>>>> IBP
> > > > > > > > > >>>>>>>>>> bump.
> > > > > > > > > >>>>>>>>>>>> Does
> > > > > > > > > >>>>>>>>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> seem
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> reasonable?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> As for the improvements -- can you
> > > clarify
> > > > > how
> > > > > > > > > >>>>>>> the
> > > > > > > > > >>>>>>>>>> multiple
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> transactional
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking
> > of
> > > a
> > > > > case
> > > > > > > > > >>>>>>>> where
> > > > > > > > > >>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>> wait/batch
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> multiple produce requests together? My
> > > > > > > > > >>>>>>> understanding
> > > > > > > > > >>>>>>>>> for
> > > > > > > > > >>>>>>>>>> now
> > > > > > > > > >>>>>>>>>>>> was
> > > > > > > > > >>>>>>>>>>>>>>> 1
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> transactional ID and one validation
> per
> > 1
> > > > > produce
> > > > > > > > > >>>>>>>>>> request.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> Each call to `AddPartitionsToTxn` is
> > > > > essentially
> > > > > > > > > >>>>>> a
> > > > > > > > > >>>>>>>>> write
> > > > > > > > > >>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> log and must block on replication. The
> > > more
> > > > we
> > > > > > > > > >>>>>> can
> > > > > > > > > >>>>>>>> fit
> > > > > > > > > >>>>>>>>>> into a
> > > > > > > > > >>>>>>>>>>>>>>> single
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> request, the more writes we can do in
> > > > > parallel.
> > > > > > > > > >>>>>> The
> > > > > > > > > >>>>>>>>>>>> alternative
> > > > > > > > > >>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>> make
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> use of more connections, but usually
> we
> > > > prefer
> > > > > > > > > >>>>>>>> batching
> > > > > > > > > >>>>>>>>>>>> since the
> > > > > > > > > >>>>>>>>>>>>>>>>>> network
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> stack is not really optimized for high
> > > > > > > > > >>>>>>>>> connection/request
> > > > > > > > > >>>>>>>>>>>> loads.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> Finally with respect to the
> > > > authorizations, I
> > > > > > > > > >>>>>>> think
> > > > > > > > > >>>>>>>>> it
> > > > > > > > > >>>>>>>>>>>> makes
> > > > > > > > > >>>>>>>>>>>>>>> sense
> > > > > > > > > >>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> skip
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit
> > > confused
> > > > > by
> > > > > > > > > >>>>>> the
> > > > > > > > > >>>>>>>>>> "leader
> > > > > > > > > >>>>>>>>>>>> ID"
> > > > > > > > > >>>>>>>>>>>>>>>> field.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the
> > request
> > > as
> > > > > > > > > >>>>>> from a
> > > > > > > > > >>>>>>>>>> broker
> > > > > > > > > >>>>>>>>>>>> (does
> > > > > > > > > >>>>>>>>>>>>>>> it
> > > > > > > > > >>>>>>>>>>>>>>>>>> matter
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> which one?).
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> We could also make it version-based.
> For
> > > the
> > > > > next
> > > > > > > > > >>>>>>>>>> version, we
> > > > > > > > > >>>>>>>>>>>>>>> could
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> require
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> CLUSTER auth. So clients would not be
> > able
> > > > to
> > > > > use
> > > > > > > > > >>>>>>> the
> > > > > > > > > >>>>>>>>> API
> > > > > > > > > >>>>>>>>>>>>>>> anymore,
> > > > > > > > > >>>>>>>>>>>>>>>>> which
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> probably what we want.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> -Jason
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:43 AM
> Justine
> > > > Olshan
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> As a follow up, I was just thinking
> > about
> > > > the
> > > > > > > > > >>>>>>>>> batching
> > > > > > > > > >>>>>>>>>> a
> > > > > > > > > >>>>>>>>>>>> bit
> > > > > > > > > >>>>>>>>>>>>>>> more.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> I suppose if we have one request in
> > > flight
> > > > > and
> > > > > > > > > >>>>>> we
> > > > > > > > > >>>>>>>>>> queue up
> > > > > > > > > >>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>> other
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> produce requests in some sort of
> > > purgatory,
> > > > > we
> > > > > > > > > >>>>>>>> could
> > > > > > > > > >>>>>>>>>> send
> > > > > > > > > >>>>>>>>>>>>>>>>> information
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> out
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> for all of them rather than one by
> one.
> > > So
> > > > > that
> > > > > > > > > >>>>>>>> would
> > > > > > > > > >>>>>>>>>> be a
> > > > > > > > > >>>>>>>>>>>>>>> benefit
> > > > > > > > > >>>>>>>>>>>>>>>>> of
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> batching partitions to add per
> > > transaction.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> I'll need to think a bit more on the
> > > design
> > > > > of
> > > > > > > > > >>>>>>> this
> > > > > > > > > >>>>>>>>>> part
> > > > > > > > > >>>>>>>>>>>> of the
> > > > > > > > > >>>>>>>>>>>>>>>> KIP,
> > > > > > > > > >>>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> will update the KIP in the next few
> > days.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:22 AM
> Justine
> > > > > Olshan
> > > > > > > > > >>>>>> <
> > > > > > > > > >>>>>>>>>>>>>>>>> jolshan@confluent.io>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>> Hey Jason -- thanks for the input
> -- I
> > > was
> > > > > > > > > >>>>>> just
> > > > > > > > > >>>>>>>>>> digging
> > > > > > > > > >>>>>>>>>>>> a bit
> > > > > > > > > >>>>>>>>>>>>>>>>> deeper
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> into
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>> the design + implementation of the
> > > > > validation
> > > > > > > > > >>>>>>>> calls
> > > > > > > > > >>>>>>>>>> here
> > > > > > > > > >>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>> what
> > > > > > > > > >>>>>>>>>>>>>>>>>> you
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> say
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>> makes sense.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>> I was wondering about compatibility
> > > here.
> > > > > > > > > >>>>>> When
> > > > > > > > > >>>>>>> we
> > > > > > > > > >>>>>>>>>> send
> > > > > > > > > >>>>>>>>>>>>>>> requests
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>> between brokers, we want to ensure
> > that
> > > > the
> > > > > > > > > >>>>>>>>> receiving
> > > > > > > > > >>>>>>>>>>>> broker
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> understands
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>> the request (specifically the new
> > > fields).
> > > > > > > > > >>>>>>>>> Typically
> > > > > > > > > >>>>>>>>>>>> this is
> > > > > > > > > >>>>>>>>>>>>>>>> done
> > > > > > > > > >>>>>>>>>>>>>>>>>> via
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>> IBP/metadata version.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>> I'm trying to think if there is a
> way
> > > > around
> > > > > > > > > >>>>>> it
> > > > > > > > > >>>>>>>> but
> > > > > > > > > >>>>>>>>>> I'm
> > > > > > > > > >>>>>>>>>>>> not
> > > > > > > > > >>>>>>>>>>>>>>> sure
> > > > > > > > > >>>>>>>>>>>>>>>>>> there
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> is.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>> As for the improvements -- can you
> > > clarify
> > > > > > > > > >>>>>> how
> > > > > > > > > >>>>>>>> the
> > > > > > > > > >>>>>>>>>>>> multiple
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> transactional
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>> IDs would help here? Were you
> thinking
> > > of
> > > > a
> > > > > > > > > >>>>>>> case
> > > > > > > > > >>>>>>>>>> where we
> > > > > > > > > >>>>>>>>>>>>>>>>> wait/batch
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>> multiple produce requests together?
> My
> > > > > > > > > >>>>>>>>> understanding
> > > > > > > > > >>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>> now
> > > > > > > > > >>>>>>>>>>>>>>>> was 1
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>> transactional ID and one validation
> > per
> > > 1
> > > > > > > > > >>>>>>> produce
> > > > > > > > > >>>>>>>>>>>> request.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>> Finally with respect to the
> > > > authorizations,
> > > > > I
> > > > > > > > > >>>>>>>> think
> > > > > > > > > >>>>>>>>>> it
> > > > > > > > > >>>>>>>>>>>> makes
> > > > > > > > > >>>>>>>>>>>>>>>> sense
> > > > > > > > > >>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> skip
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit
> > > > confused
> > > > > > > > > >>>>>> by
> > > > > > > > > >>>>>>>> the
> > > > > > > > > >>>>>>>>>>>> "leader
> > > > > > > > > >>>>>>>>>>>>>>> ID"
> > > > > > > > > >>>>>>>>>>>>>>>>>> field.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the
> > > request
> > > > as
> > > > > > > > > >>>>>>>> from a
> > > > > > > > > >>>>>>>>>>>> broker
> > > > > > > > > >>>>>>>>>>>>>>> (does
> > > > > > > > > >>>>>>>>>>>>>>>>> it
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> matter
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>> which one?).
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>> I think I want to adopt these
> > > suggestions,
> > > > > > > > > >>>>>> just
> > > > > > > > > >>>>>>>> had
> > > > > > > > > >>>>>>>>>> a few
> > > > > > > > > >>>>>>>>>>>>>>>>> questions
> > > > > > > > > >>>>>>>>>>>>>>>>>> on
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>> details.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>> On Thu, Jan 5, 2023 at 5:05 PM Jason
> > > > > > > > > >>>>>> Gustafson
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Thanks for the proposal.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> I was thinking about the
> > > implementation a
> > > > > > > > > >>>>>>> little
> > > > > > > > > >>>>>>>>>> bit.
> > > > > > > > > >>>>>>>>>>>> In the
> > > > > > > > > >>>>>>>>>>>>>>>>>> current
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> proposal, the behavior depends on
> > > whether
> > > > > we
> > > > > > > > > >>>>>>>> have
> > > > > > > > > >>>>>>>>> an
> > > > > > > > > >>>>>>>>>>>> old or
> > > > > > > > > >>>>>>>>>>>>>>> new
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> client.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> For
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> old clients, we send
> > > > `DescribeTransactions`
> > > > > > > > > >>>>>>> and
> > > > > > > > > >>>>>>>>>> verify
> > > > > > > > > >>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>> result
> > > > > > > > > >>>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> new clients, we send
> > > > `AddPartitionsToTxn`.
> > > > > > > > > >>>>>> We
> > > > > > > > > >>>>>>>>> might
> > > > > > > > > >>>>>>>>>> be
> > > > > > > > > >>>>>>>>>>>> able
> > > > > > > > > >>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> simplify
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> implementation if we can use the
> same
> > > > > > > > > >>>>>> request
> > > > > > > > > >>>>>>>>> type.
> > > > > > > > > >>>>>>>>>> For
> > > > > > > > > >>>>>>>>>>>>>>>> example,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> what if
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> bump the protocol version for
> > > > > > > > > >>>>>>>> `AddPartitionsToTxn`
> > > > > > > > > >>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>> add a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> `validateOnly`
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> flag? For older versions, we can
> set
> > > > > > > > > >>>>>>>>>>>> `validateOnly=true` so
> > > > > > > > > >>>>>>>>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> request only returns successfully
> if
> > > the
> > > > > > > > > >>>>>>>> partition
> > > > > > > > > >>>>>>>>>> had
> > > > > > > > > >>>>>>>>>>>>>>> already
> > > > > > > > > >>>>>>>>>>>>>>>>> been
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> added.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> For new versions, we can set
> > > > > > > > > >>>>>>>> `validateOnly=false`
> > > > > > > > > >>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>> partition
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> will
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> added to the transaction. The other
> > > > > slightly
> > > > > > > > > >>>>>>>>>> annoying
> > > > > > > > > >>>>>>>>>>>> thing
> > > > > > > > > >>>>>>>>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> get around is the need to collect
> the
> > > > > > > > > >>>>>>>> transaction
> > > > > > > > > >>>>>>>>>> state
> > > > > > > > > >>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>> all
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> partitions
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> even when we only care about a
> > subset.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Some additional improvements to
> > > consider:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> - We can give `AddPartitionsToTxn`
> > > better
> > > > > > > > > >>>>>>> batch
> > > > > > > > > >>>>>>>>>> support
> > > > > > > > > >>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> inter-broker
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> usage. Currently we only allow one
> > > > > > > > > >>>>>>>>>> `TransactionalId` to
> > > > > > > > > >>>>>>>>>>>> be
> > > > > > > > > >>>>>>>>>>>>>>>>>> specified,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> but
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the broker may get some benefit
> being
> > > > able
> > > > > > > > > >>>>>> to
> > > > > > > > > >>>>>>>>> batch
> > > > > > > > > >>>>>>>>>>>> across
> > > > > > > > > >>>>>>>>>>>>>>>>> multiple
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> transactions.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> - Another small improvement is
> > skipping
> > > > > > > > > >>>>>> topic
> > > > > > > > > >>>>>>>>>>>> authorization
> > > > > > > > > >>>>>>>>>>>>>>>>> checks
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> `AddPartitionsToTxn` when the
> request
> > > is
> > > > > > > > > >>>>>> from
> > > > > > > > > >>>>>>> a
> > > > > > > > > >>>>>>>>>> broker.
> > > > > > > > > >>>>>>>>>>>>>>> Perhaps
> > > > > > > > > >>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> can
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> add
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> a field for the `LeaderId` or
> > something
> > > > > like
> > > > > > > > > >>>>>>>> that
> > > > > > > > > >>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>> require
> > > > > > > > > >>>>>>>>>>>>>>>>>> CLUSTER
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> permission when set.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Best,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Jason
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 3:56 PM Jun
> > Rao
> > > > > > > > > >>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation. It
> makes
> > > > sense
> > > > > > > > > >>>>>>> to
> > > > > > > > > >>>>>>>> me
> > > > > > > > > >>>>>>>>>> now.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 1:42 PM
> > > Justine
> > > > > > > > > >>>>>>> Olshan
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> My understanding of the mechanism
> > is
> > > > > > > > > >>>>>> that
> > > > > > > > > >>>>>>>> when
> > > > > > > > > >>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>> get to
> > > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>> last
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> epoch,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> increment to the fencing/last
> epoch
> > > and
> > > > > > > > > >>>>>> if
> > > > > > > > > >>>>>>>> any
> > > > > > > > > >>>>>>>>>>>> further
> > > > > > > > > >>>>>>>>>>>>>>>>> requests
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> come
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> this producer ID they are fenced.
> > > Then
> > > > > > > > > >>>>>> the
> > > > > > > > > >>>>>>>>>> producer
> > > > > > > > > >>>>>>>>>>>>>>> gets a
> > > > > > > > > >>>>>>>>>>>>>>>>> new
> > > > > > > > > >>>>>>>>>>>>>>>>>> ID
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> restarts with epoch/sequence 0.
> The
> > > > > > > > > >>>>>> fenced
> > > > > > > > > >>>>>>>>> epoch
> > > > > > > > > >>>>>>>>>>>> sticks
> > > > > > > > > >>>>>>>>>>>>>>>>> around
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> duration of
> > > producer.id.expiration.ms
> > > > > > > > > >>>>>> and
> > > > > > > > > >>>>>>>>>> blocks
> > > > > > > > > >>>>>>>>>>>> any
> > > > > > > > > >>>>>>>>>>>>>>> late
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> messages
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> there.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> The new ID will get to take
> > advantage
> > > > of
> > > > > > > > > >>>>>>> the
> > > > > > > > > >>>>>>>>>>>> improved
> > > > > > > > > >>>>>>>>>>>>>>>>> semantics
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> around
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> non-zero start sequences. So I
> > think
> > > we
> > > > > > > > > >>>>>>> are
> > > > > > > > > >>>>>>>>>> covered.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> The only potential issue is
> > > overloading
> > > > > > > > > >>>>>>> the
> > > > > > > > > >>>>>>>>>> cache,
> > > > > > > > > >>>>>>>>>>>> but
> > > > > > > > > >>>>>>>>>>>>>>>>>> hopefully
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> improvements (lowered
> > > > > > > > > >>>>>>>>> producer.id.expiration.ms
> > > > > > > > > >>>>>>>>>> )
> > > > > > > > > >>>>>>>>>>>> will
> > > > > > > > > >>>>>>>>>>>>>>> help
> > > > > > > > > >>>>>>>>>>>>>>>>>> with
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> that.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Let
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> me know if you still have
> concerns.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 10:24 AM
> > Jun
> > > > Rao
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> <ju...@confluent.io.invalid>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> 70. The proposed fencing logic
> > > doesn't
> > > > > > > > > >>>>>>>> apply
> > > > > > > > > >>>>>>>>>> when
> > > > > > > > > >>>>>>>>>>>> pid
> > > > > > > > > >>>>>>>>>>>>>>>>>> changes,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> right? If so, I am not sure how
> > > > > > > > > >>>>>> complete
> > > > > > > > > >>>>>>>> we
> > > > > > > > > >>>>>>>>>> are
> > > > > > > > > >>>>>>>>>>>>>>>> addressing
> > > > > > > > > >>>>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> issue
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the pid changes more frequently.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 16, 2022 at 9:16 AM
> > > > > > > > > >>>>>> Justine
> > > > > > > > > >>>>>>>>> Olshan
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for replying!
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> 70.We already do the overflow
> > > > > > > > > >>>>>>> mechanism,
> > > > > > > > > >>>>>>>>> so
> > > > > > > > > >>>>>>>>>> my
> > > > > > > > > >>>>>>>>>>>>>>> change
> > > > > > > > > >>>>>>>>>>>>>>>>> would
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> just
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> make
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> happen more often.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I was also not suggesting a new
> > > > > > > > > >>>>>> field
> > > > > > > > > >>>>>>> in
> > > > > > > > > >>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>> log,
> > > > > > > > > >>>>>>>>>>>>>>> but
> > > > > > > > > >>>>>>>>>>>>>>>> in
> > > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> response,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> which would be gated by the
> > client
> > > > > > > > > >>>>>>>>> version.
> > > > > > > > > >>>>>>>>>>>> Sorry if
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> something
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> there
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> unclear. I think we are
> starting
> > to
> > > > > > > > > >>>>>>>>> diverge.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> The goal of this KIP is to not
> > > > > > > > > >>>>>> change
> > > > > > > > > >>>>>>> to
> > > > > > > > > >>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>> marker
> > > > > > > > > >>>>>>>>>>>>>>>>> format
> > > > > > > > > >>>>>>>>>>>>>>>>>> at
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> all.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> 71. Yes, I guess I was going
> > under
> > > > > > > > > >>>>>> the
> > > > > > > > > >>>>>>>>>>>> assumption
> > > > > > > > > >>>>>>>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> log
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> just
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> look at its last epoch and
> treat
> > it
> > > > > > > > > >>>>>> as
> > > > > > > > > >>>>>>>> the
> > > > > > > > > >>>>>>>>>>>> current
> > > > > > > > > >>>>>>>>>>>>>>>>> epoch. I
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> suppose
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> have some special logic that if
> > the
> > > > > > > > > >>>>>>> last
> > > > > > > > > >>>>>>>>>> epoch
> > > > > > > > > >>>>>>>>>>>> was
> > > > > > > > > >>>>>>>>>>>>>>> on a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> marker
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> actually
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> expect the next epoch or
> > something
> > > > > > > > > >>>>>>> like
> > > > > > > > > >>>>>>>>>> that. We
> > > > > > > > > >>>>>>>>>>>>>>> just
> > > > > > > > > >>>>>>>>>>>>>>>>> need
> > > > > > > > > >>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> distinguish
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> based on whether we had a
> > > > > > > > > >>>>>> commit/abort
> > > > > > > > > >>>>>>>>>> marker.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> 72.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> if the producer epoch hasn't
> > been
> > > > > > > > > >>>>>>>> bumped
> > > > > > > > > >>>>>>>>>> on
> > > > > > > > > >>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the
> stucked
> > > > > > > > > >>>>>>>> message
> > > > > > > > > >>>>>>>>>> will
> > > > > > > > > >>>>>>>>>>>> fail
> > > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> sequence
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> validation
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the
> > > producer
> > > > > > > > > >>>>>>>> epoch
> > > > > > > > > >>>>>>>>>> has
> > > > > > > > > >>>>>>>>>>>> been
> > > > > > > > > >>>>>>>>>>>>>>>>> bumped,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> ignore
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck
> > > message
> > > > > > > > > >>>>>>>> could
> > > > > > > > > >>>>>>>>>> be
> > > > > > > > > >>>>>>>>>>>>>>> appended
> > > > > > > > > >>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> log.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> So,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
> > > > > > > > > >>>>>> guard?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I follow that "the
> > > > > > > > > >>>>>>> message
> > > > > > > > > >>>>>>>>> will
> > > > > > > > > >>>>>>>>>>>> fail
> > > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> sequence
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> validation".
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> In some of these cases, we had
> an
> > > > > > > > > >>>>>>> abort
> > > > > > > > > >>>>>>>>>> marker
> > > > > > > > > >>>>>>>>>>>> (due
> > > > > > > > > >>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>> an
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> error)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> then
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the late message comes in with
> > the
> > > > > > > > > >>>>>>>> correct
> > > > > > > > > >>>>>>>>>>>> sequence
> > > > > > > > > >>>>>>>>>>>>>>>>> number.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> This
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> is a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> case
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> covered by the KIP.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> The latter case is actually not
> > > > > > > > > >>>>>>>> something
> > > > > > > > > >>>>>>>>>> we've
> > > > > > > > > >>>>>>>>>>>>>>>>> considered
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> here. I
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> think
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> generally when we bump the
> epoch,
> > > we
> > > > > > > > > >>>>>>> are
> > > > > > > > > >>>>>>>>>>>> accepting
> > > > > > > > > >>>>>>>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> sequence
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> does
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> not need to be checked anymore.
> > My
> > > > > > > > > >>>>>>>>>>>> understanding is
> > > > > > > > > >>>>>>>>>>>>>>>> also
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> that we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> typically bump epoch mid
> > > transaction
> > > > > > > > > >>>>>>>>> (based
> > > > > > > > > >>>>>>>>>> on a
> > > > > > > > > >>>>>>>>>>>>>>> quick
> > > > > > > > > >>>>>>>>>>>>>>>>> look
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> at
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> code)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but let me know if that is the
> > > case.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 15, 2022 at 12:23
> PM
> > > Jun
> > > > > > > > > >>>>>>> Rao
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the reply.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Assigning a new pid on int
> > > > > > > > > >>>>>>>> overflow
> > > > > > > > > >>>>>>>>>> seems
> > > > > > > > > >>>>>>>>>>>> a
> > > > > > > > > >>>>>>>>>>>>>>> bit
> > > > > > > > > >>>>>>>>>>>>>>>>>> hacky.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> If
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> need a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> level id, it will be better to
> > > > > > > > > >>>>>> model
> > > > > > > > > >>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>> explicitly.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> Adding a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> field
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> would require a bit more work
> > > > > > > > > >>>>>> since
> > > > > > > > > >>>>>>> it
> > > > > > > > > >>>>>>>>>>>> requires a
> > > > > > > > > >>>>>>>>>>>>>>> new
> > > > > > > > > >>>>>>>>>>>>>>>>> txn
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> marker
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> format
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the log. So, we probably need
> to
> > > > > > > > > >>>>>>> guard
> > > > > > > > > >>>>>>>>> it
> > > > > > > > > >>>>>>>>>>>> with an
> > > > > > > > > >>>>>>>>>>>>>>> IBP
> > > > > > > > > >>>>>>>>>>>>>>>>> or
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> metadata
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> version
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and document the impact on
> > > > > > > > > >>>>>> downgrade
> > > > > > > > > >>>>>>>>> once
> > > > > > > > > >>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>> new
> > > > > > > > > >>>>>>>>>>>>>>>>> format
> > > > > > > > > >>>>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> written
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 71. Hmm, once the marker is
> > > > > > > > > >>>>>> written,
> > > > > > > > > >>>>>>>> the
> > > > > > > > > >>>>>>>>>>>> partition
> > > > > > > > > >>>>>>>>>>>>>>>> will
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> expect
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> next
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> append to be on the next
> epoch.
> > > > > > > > > >>>>>> Does
> > > > > > > > > >>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>> cover
> > > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>> case
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> you
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> mentioned?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 72. Also, just to be clear on
> > the
> > > > > > > > > >>>>>>>>> stucked
> > > > > > > > > >>>>>>>>>>>> message
> > > > > > > > > >>>>>>>>>>>>>>>> issue
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> described
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> motivation. With EoS, we also
> > > > > > > > > >>>>>>> validate
> > > > > > > > > >>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>> sequence
> > > > > > > > > >>>>>>>>>>>>>>>> id
> > > > > > > > > >>>>>>>>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> idempotency.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> So,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> with the current logic, if the
> > > > > > > > > >>>>>>>> producer
> > > > > > > > > >>>>>>>>>> epoch
> > > > > > > > > >>>>>>>>>>>>>>> hasn't
> > > > > > > > > >>>>>>>>>>>>>>>>> been
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> bumped on
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the
> > stucked
> > > > > > > > > >>>>>>>>> message
> > > > > > > > > >>>>>>>>>> will
> > > > > > > > > >>>>>>>>>>>>>>> fail
> > > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> sequence
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> validation
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the
> > > > > > > > > >>>>>> producer
> > > > > > > > > >>>>>>>>>> epoch has
> > > > > > > > > >>>>>>>>>>>>>>> been
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> bumped, we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> ignore
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck
> > > > > > > > > >>>>>> message
> > > > > > > > > >>>>>>>>>> could be
> > > > > > > > > >>>>>>>>>>>>>>>> appended
> > > > > > > > > >>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> log.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> So,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want
> to
> > > > > > > > > >>>>>>> guard?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 10:44
> AM
> > > > > > > > > >>>>>>>> Justine
> > > > > > > > > >>>>>>>>>>>> Olshan
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> <jolshan@confluent.io.invalid
> >
> > > > > > > > > >>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias — thanks again for
> > > > > > > > > >>>>>> taking
> > > > > > > > > >>>>>>>>> time
> > > > > > > > > >>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>> look
> > > > > > > > > >>>>>>>>>>>>>>> a
> > > > > > > > > >>>>>>>>>>>>>>>>> this.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> You
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> said:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My proposal was only
> focusing
> > > > > > > > > >>>>>> to
> > > > > > > > > >>>>>>>>> avoid
> > > > > > > > > >>>>>>>>>>>>>>> dangling
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
> > > > > > > > > >>>>>> added
> > > > > > > > > >>>>>>>>>> without
> > > > > > > > > >>>>>>>>>>>>>>>> registered
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> partition.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> --
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more
> details
> > > > > > > > > >>>>>> to
> > > > > > > > > >>>>>>>> the
> > > > > > > > > >>>>>>>>>> KIP
> > > > > > > > > >>>>>>>>>>>> about
> > > > > > > > > >>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> scenario
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> better
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I understand
> what
> > > > > > > > > >>>>>> you
> > > > > > > > > >>>>>>>>> mean
> > > > > > > > > >>>>>>>>>>>> here.
> > > > > > > > > >>>>>>>>>>>>>>> The
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> motivation
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> section
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> describes two scenarios about
> > > > > > > > > >>>>>> how
> > > > > > > > > >>>>>>>> the
> > > > > > > > > >>>>>>>>>> record
> > > > > > > > > >>>>>>>>>>>>>>> can be
> > > > > > > > > >>>>>>>>>>>>>>>>>> added
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> without a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> registered partition:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Another way hanging
> > > > > > > > > >>>>>> transactions
> > > > > > > > > >>>>>>>> can
> > > > > > > > > >>>>>>>>>>>> occur is
> > > > > > > > > >>>>>>>>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>>>>>>> a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> client
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> buggy
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> may somehow try to write to a
> > > > > > > > > >>>>>>>>> partition
> > > > > > > > > >>>>>>>>>>>> before
> > > > > > > > > >>>>>>>>>>>>>>> it
> > > > > > > > > >>>>>>>>>>>>>>>>> adds
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> partition
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> For the first example of this
> > > > > > > > > >>>>>>> would
> > > > > > > > > >>>>>>>> it
> > > > > > > > > >>>>>>>>>> be
> > > > > > > > > >>>>>>>>>>>>>>> helpful
> > > > > > > > > >>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>> say
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> comes in after the abort, but
> > > > > > > > > >>>>>>> before
> > > > > > > > > >>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>> partition
> > > > > > > > > >>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> added
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> next
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction so it becomes
> > > > > > > > > >>>>>>> "hanging."
> > > > > > > > > >>>>>>>>>>>> Perhaps the
> > > > > > > > > >>>>>>>>>>>>>>>> next
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> sentence
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> describing
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the message becoming part of
> > the
> > > > > > > > > >>>>>>>> next
> > > > > > > > > >>>>>>>>>>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>>>>>> (a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> different
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> case)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> was
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not properly differentiated.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun — thanks for reading the
> > > > > > > > > >>>>>> KIP.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. The int typing was a
> > > > > > > > > >>>>>> concern.
> > > > > > > > > >>>>>>>>>> Currently
> > > > > > > > > >>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>> have a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> mechanism
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> place
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> fence the final epoch when
> the
> > > > > > > > > >>>>>>> epoch
> > > > > > > > > >>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>> about to
> > > > > > > > > >>>>>>>>>>>>>>>>>> overflow
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> assign
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer ID with epoch 0. Of
> > > > > > > > > >>>>>>> course,
> > > > > > > > > >>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>> is a
> > > > > > > > > >>>>>>>>>>>>>>> bit
> > > > > > > > > >>>>>>>>>>>>>>>>>> tricky
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> when it
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> comes
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response back to the
> > client.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Making this a long could be
> > > > > > > > > >>>>>>> another
> > > > > > > > > >>>>>>>>>> option,
> > > > > > > > > >>>>>>>>>>>> but
> > > > > > > > > >>>>>>>>>>>>>>> I
> > > > > > > > > >>>>>>>>>>>>>>>>>> wonder
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> are
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> there
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> implications on changing this
> > > > > > > > > >>>>>>> field
> > > > > > > > > >>>>>>>> if
> > > > > > > > > >>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>> epoch is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> persisted
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> disk?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to check the usages.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 71.This was something
> Matthias
> > > > > > > > > >>>>>>> asked
> > > > > > > > > >>>>>>>>>> about
> > > > > > > > > >>>>>>>>>>>> as
> > > > > > > > > >>>>>>>>>>>>>>>> well. I
> > > > > > > > > >>>>>>>>>>>>>>>>>> was
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> considering a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible edge case where a
> > > > > > > > > >>>>>> produce
> > > > > > > > > >>>>>>>>>> request
> > > > > > > > > >>>>>>>>>>>> from
> > > > > > > > > >>>>>>>>>>>>>>> a
> > > > > > > > > >>>>>>>>>>>>>>>> new
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> somehow
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> gets sent right after the
> > marker
> > > > > > > > > >>>>>>> is
> > > > > > > > > >>>>>>>>>>>> written, but
> > > > > > > > > >>>>>>>>>>>>>>>>> before
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> alerted of the newly bumped
> > > > > > > > > >>>>>> epoch.
> > > > > > > > > >>>>>>>> In
> > > > > > > > > >>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>> case, we
> > > > > > > > > >>>>>>>>>>>>>>>>> may
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> include
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> record
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we don't want to. I
> > suppose
> > > > > > > > > >>>>>>> we
> > > > > > > > > >>>>>>>>>> could
> > > > > > > > > >>>>>>>>>>>> try
> > > > > > > > > >>>>>>>>>>>>>>> to do
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> something
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> client
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> side
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to bump the epoch after
> sending
> > > > > > > > > >>>>>> an
> > > > > > > > > >>>>>>>>>> endTxn as
> > > > > > > > > >>>>>>>>>>>>>>> well
> > > > > > > > > >>>>>>>>>>>>>>>> in
> > > > > > > > > >>>>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> scenario
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> —
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> I
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wonder how it would work when
> > > > > > > > > >>>>>> the
> > > > > > > > > >>>>>>>>>> server is
> > > > > > > > > >>>>>>>>>>>>>>>> aborting
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> based
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> on
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> error. I could also be
> missing
> > > > > > > > > >>>>>>>>>> something and
> > > > > > > > > >>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> scenario
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> actually
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again to everyone
> > reading
> > > > > > > > > >>>>>>> and
> > > > > > > > > >>>>>>>>>>>> commenting.
> > > > > > > > > >>>>>>>>>>>>>>>> Let
> > > > > > > > > >>>>>>>>>>>>>>>>> me
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> know
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> about
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> further questions or
> comments.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 9:41
> AM
> > > > > > > > > >>>>>>> Jun
> > > > > > > > > >>>>>>>>> Rao
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP. A couple
> > > > > > > > > >>>>>> of
> > > > > > > > > >>>>>>>>>> comments.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Currently, the producer
> > > > > > > > > >>>>>>> epoch
> > > > > > > > > >>>>>>>> is
> > > > > > > > > >>>>>>>>>> an
> > > > > > > > > >>>>>>>>>>>> int.
> > > > > > > > > >>>>>>>>>>>>>>> I am
> > > > > > > > > >>>>>>>>>>>>>>>>> not
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> sure
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> it's
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> enough
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to accommodate all
> > > > > > > > > >>>>>> transactions
> > > > > > > > > >>>>>>> in
> > > > > > > > > >>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>> lifetime
> > > > > > > > > >>>>>>>>>>>>>>>> of
> > > > > > > > > >>>>>>>>>>>>>>>>> a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> producer.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Should
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> change that to a long or
> add a
> > > > > > > > > >>>>>>> new
> > > > > > > > > >>>>>>>>>> long
> > > > > > > > > >>>>>>>>>>>> field
> > > > > > > > > >>>>>>>>>>>>>>>> like
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> txnId?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 71. "it will write the
> prepare
> > > > > > > > > >>>>>>>>> commit
> > > > > > > > > >>>>>>>>>>>> message
> > > > > > > > > >>>>>>>>>>>>>>>> with
> > > > > > > > > >>>>>>>>>>>>>>>>> a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> bumped
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> send
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WriteTxnMarkerRequests with
> > > > > > > > > >>>>>> the
> > > > > > > > > >>>>>>>>> bumped
> > > > > > > > > >>>>>>>>>>>> epoch."
> > > > > > > > > >>>>>>>>>>>>>>>> Hmm,
> > > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> associated
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the current txn right?
> > > > > > > > > >>>>>> So,
> > > > > > > > > >>>>>>> it
> > > > > > > > > >>>>>>>>>> seems
> > > > > > > > > >>>>>>>>>>>>>>> weird to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> write a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> commit
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with a bumped epoch. Should
> we
> > > > > > > > > >>>>>>>> only
> > > > > > > > > >>>>>>>>>> bump
> > > > > > > > > >>>>>>>>>>>> up
> > > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>> epoch
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> in
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> EndTxnResponse
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rename the field to sth like
> > > > > > > > > >>>>>>>>>>>>>>> nextProducerEpoch?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 12, 2022 at 8:54
> > > > > > > > > >>>>>> PM
> > > > > > > > > >>>>>>>>>> Matthias
> > > > > > > > > >>>>>>>>>>>> J.
> > > > > > > > > >>>>>>>>>>>>>>> Sax <
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the background.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30: SGTM. My proposal
> was
> > > > > > > > > >>>>>>>> only
> > > > > > > > > >>>>>>>>>>>> focusing
> > > > > > > > > >>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>> avoid
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> dangling
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
> > > > > > > > > >>>>>>>> added
> > > > > > > > > >>>>>>>>>>>> without
> > > > > > > > > >>>>>>>>>>>>>>>>>> registered
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> partition.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> --
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more
> > > > > > > > > >>>>>> details
> > > > > > > > > >>>>>>>> to
> > > > > > > > > >>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>> KIP
> > > > > > > > > >>>>>>>>>>>>>>> about
> > > > > > > > > >>>>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> scenario
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40: I think you hit a fair
> > > > > > > > > >>>>>>> point
> > > > > > > > > >>>>>>>>>> about
> > > > > > > > > >>>>>>>>>>>> race
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> conditions
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> or
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> client
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> bugs
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (incorrectly not bumping
> the
> > > > > > > > > >>>>>>>>>> epoch). The
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> complexity/confusion
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> using
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the bumped epoch I see, is
> > > > > > > > > >>>>>>>> mainly
> > > > > > > > > >>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>> internal
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> debugging,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> ie,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> inspecting
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log segment dumps -- it
> > > > > > > > > >>>>>> seems
> > > > > > > > > >>>>>>>>>> harder to
> > > > > > > > > >>>>>>>>>>>>>>> reason
> > > > > > > > > >>>>>>>>>>>>>>>>>> about
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> system
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> us
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> humans. But if we get
> better
> > > > > > > > > >>>>>>>>>>>> guarantees, it
> > > > > > > > > >>>>>>>>>>>>>>>> would
> > > > > > > > > >>>>>>>>>>>>>>>>>> be
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> worth to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> use
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped epoch.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60: as I mentioned already,
> > > > > > > > > >>>>>> I
> > > > > > > > > >>>>>>>>> don't
> > > > > > > > > >>>>>>>>>>>> know the
> > > > > > > > > >>>>>>>>>>>>>>>>> broker
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> internals
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> provide
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more input. So if nobody
> > > > > > > > > >>>>>> else
> > > > > > > > > >>>>>>>>> chimes
> > > > > > > > > >>>>>>>>>>>> in, we
> > > > > > > > > >>>>>>>>>>>>>>>>> should
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> just
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> move
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> forward
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with your proposal.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 12/6/22 4:22 PM, Justine
> > > > > > > > > >>>>>>>> Olshan
> > > > > > > > > >>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi all,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> After Artem's questions
> > > > > > > > > >>>>>>> about
> > > > > > > > > >>>>>>>>>> error
> > > > > > > > > >>>>>>>>>>>>>>> behavior,
> > > > > > > > > >>>>>>>>>>>>>>>>>> I've
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> re-evaluated
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unknown producer ID
> > > > > > > > > >>>>>>> exception
> > > > > > > > > >>>>>>>>> and
> > > > > > > > > >>>>>>>>>> had
> > > > > > > > > >>>>>>>>>>>> some
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> discussions
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> offline.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think generally it makes
> > > > > > > > > >>>>>>>> sense
> > > > > > > > > >>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>> simplify
> > > > > > > > > >>>>>>>>>>>>>>>>>> error
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> handling
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> cases
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this and the
> > > > > > > > > >>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > > >>>>>>>>>> error
> > > > > > > > > >>>>>>>>>>>>>>> has a
> > > > > > > > > >>>>>>>>>>>>>>>>>> pretty
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> long
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complicated
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> history. Because of this,
> > > > > > > > > >>>>>> I
> > > > > > > > > >>>>>>>>>> propose
> > > > > > > > > >>>>>>>>>>>>>>> adding a
> > > > > > > > > >>>>>>>>>>>>>>>>> new
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> error
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> code
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ABORTABLE_ERROR
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that when encountered by
> > > > > > > > > >>>>>> new
> > > > > > > > > >>>>>>>>>> clients
> > > > > > > > > >>>>>>>>>>>>>>> (gated
> > > > > > > > > >>>>>>>>>>>>>>>> by
> > > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> produce
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will simply abort the
> > > > > > > > > >>>>>>>>> transaction.
> > > > > > > > > >>>>>>>>>>>> This
> > > > > > > > > >>>>>>>>>>>>>>>> allows
> > > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> server
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> some
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> say
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in whether the client
> > > > > > > > > >>>>>> aborts
> > > > > > > > > >>>>>>>> and
> > > > > > > > > >>>>>>>>>> makes
> > > > > > > > > >>>>>>>>>>>>>>>> handling
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> much
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> simpler.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> future, we can also use
> > > > > > > > > >>>>>> this
> > > > > > > > > >>>>>>>>>> error in
> > > > > > > > > >>>>>>>>>>>>>>> other
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> situations
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> where
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> want
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> abort the transactions. We
> > > > > > > > > >>>>>>> can
> > > > > > > > > >>>>>>>>>> even
> > > > > > > > > >>>>>>>>>>>> use on
> > > > > > > > > >>>>>>>>>>>>>>>>> other
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> apis.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've added this to the
> > > > > > > > > >>>>>> KIP.
> > > > > > > > > >>>>>>>> Let
> > > > > > > > > >>>>>>>>> me
> > > > > > > > > >>>>>>>>>>>> know if
> > > > > > > > > >>>>>>>>>>>>>>>>> there
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> are
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> any
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> questions
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> or
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 2, 2022 at
> > > > > > > > > >>>>>> 10:22
> > > > > > > > > >>>>>>>> AM
> > > > > > > > > >>>>>>>>>>>> Justine
> > > > > > > > > >>>>>>>>>>>>>>>> Olshan
> > > > > > > > > >>>>>>>>>>>>>>>>> <
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> jolshan@confluent.io
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey Matthias,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30 — Maybe I also
> > > > > > > > > >>>>>> didn't
> > > > > > > > > >>>>>>>>>> express
> > > > > > > > > >>>>>>>>>>>>>>> myself
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> clearly.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> For
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't have a way to
> > > > > > > > > >>>>>>>> distinguish
> > > > > > > > > >>>>>>>>>>>> between a
> > > > > > > > > >>>>>>>>>>>>>>>>>> previous
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> current
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction since we
> > > > > > > > > >>>>>> don't
> > > > > > > > > >>>>>>>> have
> > > > > > > > > >>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>> epoch
> > > > > > > > > >>>>>>>>>>>>>>>>> bump.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> This
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> means
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message from the previous
> > > > > > > > > >>>>>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>>>>> may be
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> added to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> one.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> With
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older clients — we can't
> > > > > > > > > >>>>>>>>>> guarantee
> > > > > > > > > >>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>> won't
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> happen
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> if we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> already
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call
> > > > > > > > > >>>>>>> (why
> > > > > > > > > >>>>>>>> we
> > > > > > > > > >>>>>>>>>> make
> > > > > > > > > >>>>>>>>>>>>>>> changes
> > > > > > > > > >>>>>>>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> newer
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> client)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can at least gate some by
> > > > > > > > > >>>>>>>>>> ensuring
> > > > > > > > > >>>>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> partition
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> has
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> been
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> added
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. The
> > > > > > > > > >>>>>> rationale
> > > > > > > > > >>>>>>>> here
> > > > > > > > > >>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>>>>>> there
> > > > > > > > > >>>>>>>>>>>>>>>>>> are
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> likely
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> LESS
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrivals
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as time goes on, so
> > > > > > > > > >>>>>>> hopefully
> > > > > > > > > >>>>>>>>>> most
> > > > > > > > > >>>>>>>>>>>> late
> > > > > > > > > >>>>>>>>>>>>>>>>> arrivals
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> will
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> come
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> BEFORE
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call.
> > > > > > > > > >>>>>>>> Those
> > > > > > > > > >>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>>>>> arrive
> > > > > > > > > >>>>>>>>>>>>>>>>>> before
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> will
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> properly
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> gated
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> > > > > > > > > >>>>>>> describeTransactions
> > > > > > > > > >>>>>>>>>>>> approach.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we take the approach
> > > > > > > > > >>>>>> you
> > > > > > > > > >>>>>>>>>>>> suggested,
> > > > > > > > > >>>>>>>>>>>>>>> ANY
> > > > > > > > > >>>>>>>>>>>>>>>>> late
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> arrival
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> from a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> previous
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction will be
> > > > > > > > > >>>>>> added.
> > > > > > > > > >>>>>>>> And
> > > > > > > > > >>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>> don't
> > > > > > > > > >>>>>>>>>>>>>>> want
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> that. I
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> also
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> see
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> benefit in sending
> > > > > > > > > >>>>>>>>>> addPartitionsToTxn
> > > > > > > > > >>>>>>>>>>>>>>> over
> > > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> describeTxns
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> They
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> both be one extra RPC to
> > > > > > > > > >>>>>>> the
> > > > > > > > > >>>>>>>>> Txn
> > > > > > > > > >>>>>>>>>>>>>>>> coordinator.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To be clear — newer
> > > > > > > > > >>>>>> clients
> > > > > > > > > >>>>>>>>> will
> > > > > > > > > >>>>>>>>>> use
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> instead
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTxns.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that if we
> > > > > > > > > >>>>>>> have
> > > > > > > > > >>>>>>>>>> some
> > > > > > > > > >>>>>>>>>>>> delay
> > > > > > > > > >>>>>>>>>>>>>>> in
> > > > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> client
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> bump
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it could continue to send
> > > > > > > > > >>>>>>>> epoch
> > > > > > > > > >>>>>>>>>> 73
> > > > > > > > > >>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>> those
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> records
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fenced.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Perhaps this is not an
> > > > > > > > > >>>>>>> issue
> > > > > > > > > >>>>>>>> if
> > > > > > > > > >>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>> don't
> > > > > > > > > >>>>>>>>>>>>>>>> allow
> > > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> next
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> produce
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> go
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through before the EndTxn
> > > > > > > > > >>>>>>>>> request
> > > > > > > > > >>>>>>>>>>>>>>> returns.
> > > > > > > > > >>>>>>>>>>>>>>>> I'm
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> also
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> thinking
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> about
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cases of
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure. I will need to
> > > > > > > > > >>>>>>> think
> > > > > > > > > >>>>>>>>> on
> > > > > > > > > >>>>>>>>>>>> this a
> > > > > > > > > >>>>>>>>>>>>>>> bit.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I wasn't sure if it was
> > > > > > > > > >>>>>>> that
> > > > > > > > > >>>>>>>>>>>> confusing.
> > > > > > > > > >>>>>>>>>>>>>>> But
> > > > > > > > > >>>>>>>>>>>>>>>> if
> > > > > > > > > >>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> think it
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> is,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> investigate other ways.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure these are
> > > > > > > > > >>>>>> the
> > > > > > > > > >>>>>>>> same
> > > > > > > > > >>>>>>>>>>>>>>> purgatories
> > > > > > > > > >>>>>>>>>>>>>>>>>> since
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> one
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> is a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> produce
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory (I was planning
> > > > > > > > > >>>>>>> on
> > > > > > > > > >>>>>>>>>> using a
> > > > > > > > > >>>>>>>>>>>>>>>> callback
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> rather
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> than
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the other is simply a
> > > > > > > > > >>>>>>> request
> > > > > > > > > >>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>> append
> > > > > > > > > >>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> log.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> Not
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> sure
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> structure here for
> > > > > > > > > >>>>>>> ordering,
> > > > > > > > > >>>>>>>>> but
> > > > > > > > > >>>>>>>>>> my
> > > > > > > > > >>>>>>>>>>>>>>>>>> understanding
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handle the write request
> > > > > > > > > >>>>>>>> before
> > > > > > > > > >>>>>>>>>> it
> > > > > > > > > >>>>>>>>>>>> hears
> > > > > > > > > >>>>>>>>>>>>>>>> back
> > > > > > > > > >>>>>>>>>>>>>>>>>> from
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Txn
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Coordinator.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if I
> > > > > > > > > >>>>>>>> misunderstood
> > > > > > > > > >>>>>>>>>>>> something
> > > > > > > > > >>>>>>>>>>>>>>> or
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> something
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> was
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> unclear.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 1, 2022 at
> > > > > > > > > >>>>>>> 12:15
> > > > > > > > > >>>>>>>> PM
> > > > > > > > > >>>>>>>>>>>> Matthias
> > > > > > > > > >>>>>>>>>>>>>>> J.
> > > > > > > > > >>>>>>>>>>>>>>>>> Sax
> > > > > > > > > >>>>>>>>>>>>>>>>>> <
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the details
> > > > > > > > > >>>>>>>>> Justine!
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side change
> > > > > > > > > >>>>>>> for
> > > > > > > > > >>>>>>>> 2
> > > > > > > > > >>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>> removing
> > > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> addPartitions
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need to
> > > > > > > > > >>>>>>> make
> > > > > > > > > >>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>> from
> > > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> producer
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think I did not
> > > > > > > > > >>>>>> express
> > > > > > > > > >>>>>>>>> myself
> > > > > > > > > >>>>>>>>>>>>>>> clearly. I
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> understand
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should) change the
> > > > > > > > > >>>>>>> producer
> > > > > > > > > >>>>>>>> to
> > > > > > > > > >>>>>>>>>> not
> > > > > > > > > >>>>>>>>>>>> send
> > > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> longer. But I don't
> > > > > > > > > >>>>>> thinks
> > > > > > > > > >>>>>>>>> it's
> > > > > > > > > >>>>>>>>>>>>>>> requirement
> > > > > > > > > >>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> change
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> broker?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What I am trying to say
> > > > > > > > > >>>>>>> is:
> > > > > > > > > >>>>>>>>> as a
> > > > > > > > > >>>>>>>>>>>>>>> safe-guard
> > > > > > > > > >>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> improvement
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producers, the partition
> > > > > > > > > >>>>>>>>> leader
> > > > > > > > > >>>>>>>>>> can
> > > > > > > > > >>>>>>>>>>>> just
> > > > > > > > > >>>>>>>>>>>>>>>> send
> > > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request to the
> > > > > > > > > >>>>>>>> TX-coordinator
> > > > > > > > > >>>>>>>>>> in any
> > > > > > > > > >>>>>>>>>>>>>>> case
> > > > > > > > > >>>>>>>>>>>>>>>> --
> > > > > > > > > >>>>>>>>>>>>>>>>> if
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> old
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> correctly did send the
> > > > > > > > > >>>>>>>>>>>> `addPartition`
> > > > > > > > > >>>>>>>>>>>>>>>> request
> > > > > > > > > >>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> already, the
> > > > > > > > > >>>>>>> TX-coordinator
> > > > > > > > > >>>>>>>>> can
> > > > > > > > > >>>>>>>>>> just
> > > > > > > > > >>>>>>>>>>>>>>>> "ignore"
> > > > > > > > > >>>>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> as
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> idempotent.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if the old producer has
> > > > > > > > > >>>>>> a
> > > > > > > > > >>>>>>>> bug
> > > > > > > > > >>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>> did
> > > > > > > > > >>>>>>>>>>>>>>>> forget
> > > > > > > > > >>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> sent
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `addPartition`
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request, we would now
> > > > > > > > > >>>>>>> ensure
> > > > > > > > > >>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>> partition
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> indeed
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> added
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX and thus fix a
> > > > > > > > > >>>>>>> potential
> > > > > > > > > >>>>>>>>>>>> producer bug
> > > > > > > > > >>>>>>>>>>>>>>>>> (even
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> if we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> get
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fencing via the bump
> > > > > > > > > >>>>>>> epoch).
> > > > > > > > > >>>>>>>>> --
> > > > > > > > > >>>>>>>>>> It
> > > > > > > > > >>>>>>>>>>>>>>> seems to
> > > > > > > > > >>>>>>>>>>>>>>>>> be
> > > > > > > > > >>>>>>>>>>>>>>>>>> a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> good
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Or
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there a reason to not do
> > > > > > > > > >>>>>>>> this?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is ongoing
> > > > > > > > > >>>>>> =
> > > > > > > > > >>>>>>>>>> partition
> > > > > > > > > >>>>>>>>>>>> was
> > > > > > > > > >>>>>>>>>>>>>>>> added
> > > > > > > > > >>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> via
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn. We
> > > > > > > > > >>>>>>>> check
> > > > > > > > > >>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>> with
> > > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this wasn't
> > > > > > > > > >>>>>>>>>> sufficiently
> > > > > > > > > >>>>>>>>>>>>>>>>> explained
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> here:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do what I propose
> > > > > > > > > >>>>>> in
> > > > > > > > > >>>>>>>>>> (20), we
> > > > > > > > > >>>>>>>>>>>>>>> don't
> > > > > > > > > >>>>>>>>>>>>>>>>>> really
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> need
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> make
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `DescribeTransaction`
> > > > > > > > > >>>>>>> call,
> > > > > > > > > >>>>>>>> as
> > > > > > > > > >>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>> partition
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> leader
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> adds
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for older clients and we
> > > > > > > > > >>>>>>> get
> > > > > > > > > >>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>> check
> > > > > > > > > >>>>>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>> free.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is that
> > > > > > > > > >>>>>> if
> > > > > > > > > >>>>>>>> any
> > > > > > > > > >>>>>>>>>>>> messages
> > > > > > > > > >>>>>>>>>>>>>>>>> somehow
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> come
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> in
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> before
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the producer,
> > > > > > > > > >>>>>>> they
> > > > > > > > > >>>>>>>>>> will be
> > > > > > > > > >>>>>>>>>>>>>>>> fenced.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> However,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it can be
> > > > > > > > > >>>>>>>>>> discussed
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree that we should
> > > > > > > > > >>>>>>> have
> > > > > > > > > >>>>>>>>>> epoch
> > > > > > > > > >>>>>>>>>>>>>>> fencing.
> > > > > > > > > >>>>>>>>>>>>>>>> My
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> question is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> different:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Assume we are at epoch
> > > > > > > > > >>>>>> 73,
> > > > > > > > > >>>>>>>> and
> > > > > > > > > >>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>> have
> > > > > > > > > >>>>>>>>>>>>>>> an
> > > > > > > > > >>>>>>>>>>>>>>>>>> ongoing
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed. It seems
> > > > > > > > > >>>>>>> natural
> > > > > > > > > >>>>>>>> to
> > > > > > > > > >>>>>>>>>>>> write the
> > > > > > > > > >>>>>>>>>>>>>>>>>> "prepare
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> commit"
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> marker
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
> > > > > > > > > >>>>>>> both
> > > > > > > > > >>>>>>>>> with
> > > > > > > > > >>>>>>>>>>>> epoch
> > > > > > > > > >>>>>>>>>>>>>>> 73,
> > > > > > > > > >>>>>>>>>>>>>>>>> too,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> as
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> it
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> belongs
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current transaction. Of
> > > > > > > > > >>>>>>>>> course,
> > > > > > > > > >>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>> now
> > > > > > > > > >>>>>>>>>>>>>>> also
> > > > > > > > > >>>>>>>>>>>>>>>>>> bump
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> expect
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the next requests to
> > > > > > > > > >>>>>> have
> > > > > > > > > >>>>>>>>> epoch
> > > > > > > > > >>>>>>>>>> 74,
> > > > > > > > > >>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>> would
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> reject
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> an
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch 73, as the
> > > > > > > > > >>>>>>>> corresponding
> > > > > > > > > >>>>>>>>>> TX
> > > > > > > > > >>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>> epoch
> > > > > > > > > >>>>>>>>>>>>>>>>> 73
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> was
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> already
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems you propose to
> > > > > > > > > >>>>>>>> write
> > > > > > > > > >>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>> "prepare
> > > > > > > > > >>>>>>>>>>>>>>>>>> commit
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> marker"
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
> > > > > > > > > >>>>>>> with
> > > > > > > > > >>>>>>>>>> epoch 74
> > > > > > > > > >>>>>>>>>>>>>>>> though,
> > > > > > > > > >>>>>>>>>>>>>>>>>> what
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> work,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems confusing. Is
> > > > > > > > > >>>>>> there
> > > > > > > > > >>>>>>> a
> > > > > > > > > >>>>>>>>>> reason
> > > > > > > > > >>>>>>>>>>>> why
> > > > > > > > > >>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>> would
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> use
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> bumped
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 74
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of the current
> > > > > > > > > >>>>>>> epoch
> > > > > > > > > >>>>>>>>> 73?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are checking if
> > > > > > > > > >>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>> transaction is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> ongoing,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the leader
> > > > > > > > > >>>>>>>>> partition
> > > > > > > > > >>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for this
> > > > > > > > > >>>>>>>>>> message to
> > > > > > > > > >>>>>>>>>>>> come
> > > > > > > > > >>>>>>>>>>>>>>>>> back,
> > > > > > > > > >>>>>>>>>>>>>>>>>> in
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> theory
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
> > > > > > > > > >>>>>> that
> > > > > > > > > >>>>>>>>> would
> > > > > > > > > >>>>>>>>>>>> make the
> > > > > > > > > >>>>>>>>>>>>>>>>>> original
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> result
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out of
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why we
> > > > > > > > > >>>>>> can
> > > > > > > > > >>>>>>>>> check
> > > > > > > > > >>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>> leader
> > > > > > > > > >>>>>>>>>>>>>>>>>> state
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> before
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks. Got it.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, is this really
> > > > > > > > > >>>>>> an
> > > > > > > > > >>>>>>>>>> issue?
> > > > > > > > > >>>>>>>>>>>> We put
> > > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> produce
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory, so how could
> > > > > > > > > >>>>>> we
> > > > > > > > > >>>>>>>>>> process
> > > > > > > > > >>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> first?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Don't we need to put the
> > > > > > > > > >>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> into
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> too,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for this case, and
> > > > > > > > > >>>>>> process
> > > > > > > > > >>>>>>>>> both
> > > > > > > > > >>>>>>>>>>>> request
> > > > > > > > > >>>>>>>>>>>>>>>>>> in-order?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> (Again,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> my
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> knowledge is limited and
> > > > > > > > > >>>>>>>> maybe
> > > > > > > > > >>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>> don't
> > > > > > > > > >>>>>>>>>>>>>>>>>> maintain
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> order
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case, what seems to be
> > > > > > > > > >>>>>> an
> > > > > > > > > >>>>>>>>> issue
> > > > > > > > > >>>>>>>>>>>> IMHO,
> > > > > > > > > >>>>>>>>>>>>>>> and I
> > > > > > > > > >>>>>>>>>>>>>>>>> am
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> wondering
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> changing
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request handling to
> > > > > > > > > >>>>>>> preserve
> > > > > > > > > >>>>>>>>>> order
> > > > > > > > > >>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>> case
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> might be
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> cleaner
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution?)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/30/22 3:28 PM,
> > > > > > > > > >>>>>> Artem
> > > > > > > > > >>>>>>>>>> Livshits
> > > > > > > > > >>>>>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think the interesting
> > > > > > > > > >>>>>>>> part
> > > > > > > > > >>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>> not in
> > > > > > > > > >>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> logic
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> (because
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> tries
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> figure out when
> > > > > > > > > >>>>>>>>>>>> UNKNOWN_PRODUCER_ID is
> > > > > > > > > >>>>>>>>>>>>>>>>>> retriable
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> it's
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retryable,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's definitely not
> > > > > > > > > >>>>>>> fatal),
> > > > > > > > > >>>>>>>>> but
> > > > > > > > > >>>>>>>>>>>> what
> > > > > > > > > >>>>>>>>>>>>>>>> happens
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> when
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> logic
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'true' and falls
> > > > > > > > > >>>>>> through.
> > > > > > > > > >>>>>>>> In
> > > > > > > > > >>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>> old
> > > > > > > > > >>>>>>>>>>>>>>>>> clients
> > > > > > > > > >>>>>>>>>>>>>>>>>> it
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> seems
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> keep the behavior in
> > > > > > > > > >>>>>> the
> > > > > > > > > >>>>>>>> new
> > > > > > > > > >>>>>>>>>>>> clients,
> > > > > > > > > >>>>>>>>>>>>>>> I'd
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> expect it
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> well.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 29, 2022 at
> > > > > > > > > >>>>>>>> 11:57
> > > > > > > > > >>>>>>>>>> AM
> > > > > > > > > >>>>>>>>>>>>>>> Justine
> > > > > > > > > >>>>>>>>>>>>>>>>>> Olshan
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>> <jolshan@confluent.io.invalid
> > > > > > > > > >>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Artem and Jeff,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
> > > > > > > > > >>>>>> look
> > > > > > > > > >>>>>>>> and
> > > > > > > > > >>>>>>>>>>>> sorry for
> > > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>> slow
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> response.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You both mentioned the
> > > > > > > > > >>>>>>>>> change
> > > > > > > > > >>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>> handle
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear — this error
> > > > > > > > > >>>>>> code
> > > > > > > > > >>>>>>>> will
> > > > > > > > > >>>>>>>>>> only
> > > > > > > > > >>>>>>>>>>>> be
> > > > > > > > > >>>>>>>>>>>>>>> sent
> > > > > > > > > >>>>>>>>>>>>>>>>>> again
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> when
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> client's
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version is high enough
> > > > > > > > > >>>>>>> to
> > > > > > > > > >>>>>>>>>> ensure
> > > > > > > > > >>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>> handle
> > > > > > > > > >>>>>>>>>>>>>>>>> it
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> correctly.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The current (Java)
> > > > > > > > > >>>>>>> client
> > > > > > > > > >>>>>>>>>> handles
> > > > > > > > > >>>>>>>>>>>>>>> this by
> > > > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> following
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> (somewhat
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> long)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code snippet:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // An
> > > > > > > > > >>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > > >>>>>>>>>> means
> > > > > > > > > >>>>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>> have
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> lost
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker. Depending on
> > > > > > > > > >>>>>> the
> > > > > > > > > >>>>>>>> log
> > > > > > > > > >>>>>>>>>> start
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // offset, we may want
> > > > > > > > > >>>>>>> to
> > > > > > > > > >>>>>>>>>> retry
> > > > > > > > > >>>>>>>>>>>>>>> these, as
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> described
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> each
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> case
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below. If
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> none of those apply,
> > > > > > > > > >>>>>>> then
> > > > > > > > > >>>>>>>>> for
> > > > > > > > > >>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // idempotent
> > > > > > > > > >>>>>> producer,
> > > > > > > > > >>>>>>> we
> > > > > > > > > >>>>>>>>>> will
> > > > > > > > > >>>>>>>>>>>>>>> locally
> > > > > > > > > >>>>>>>>>>>>>>>>> bump
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> reset
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence numbers of
> > > > > > > > > >>>>>>>>> in-flight
> > > > > > > > > >>>>>>>>>>>> batches
> > > > > > > > > >>>>>>>>>>>>>>>> from
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // sequence 0, then
> > > > > > > > > >>>>>>> retry
> > > > > > > > > >>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>> failed
> > > > > > > > > >>>>>>>>>>>>>>>> batch,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> which
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> should
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> now
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> succeed.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the transactional
> > > > > > > > > >>>>>>>> producer,
> > > > > > > > > >>>>>>>>>> allow
> > > > > > > > > >>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // batch to fail. When
> > > > > > > > > >>>>>>>>>> processing
> > > > > > > > > >>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>> failed
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> batch,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transition
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an abortable error and
> > > > > > > > > >>>>>>>> set a
> > > > > > > > > >>>>>>>>>> flag
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // indicating that we
> > > > > > > > > >>>>>>> need
> > > > > > > > > >>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>> bump the
> > > > > > > > > >>>>>>>>>>>>>>>>> epoch
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> (if
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> supported
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> by
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker).
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if (error ==
> > > > > > > > > >>>>>>>>>>>>>>>> Errors.*UNKNOWN_PRODUCER_ID*)
> > > > > > > > > >>>>>>>>>>>>>>>>> {
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > > > > > > > > >>>>>>>>>> (response.logStartOffset
> > > > > > > > > >>>>>>>>>>>> ==
> > > > > > > > > >>>>>>>>>>>>>>> -1)
> > > > > > > > > >>>>>>>>>>>>>>>> {
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // We
> don't
> > > > > > > > > >>>>>>> know
> > > > > > > > > >>>>>>>>>> the log
> > > > > > > > > >>>>>>>>>>>>>>> start
> > > > > > > > > >>>>>>>>>>>>>>>>>> offset
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> with
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> response.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the request
> > > > > > > > > >>>>>>>> until
> > > > > > > > > >>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>> get
> > > > > > > > > >>>>>>>>>>>>>>> it.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // The
> > > > > > > > > >>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > > >>>>>>>>>>>>>>>> error
> > > > > > > > > >>>>>>>>>>>>>>>>>> code
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> was
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> added
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> along
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProduceResponse which
> > > > > > > > > >>>>>>>>>> includes the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              //
> > > > > > > > > >>>>>>>> logStartOffset.
> > > > > > > > > >>>>>>>>>> So
> > > > > > > > > >>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>> '-1'
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> sentinel
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> backward
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatibility.
> > > > > > > > > >>>>>> Instead,
> > > > > > > > > >>>>>>> it
> > > > > > > > > >>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>> possible
> > > > > > > > > >>>>>>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // a
> broker
> > > > > > > > > >>>>>> to
> > > > > > > > > >>>>>>>> not
> > > > > > > > > >>>>>>>>>> know
> > > > > > > > > >>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> logStartOffset at
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> when
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returning
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response because
> > > > > > > > > >>>>>> the
> > > > > > > > > >>>>>>>>>> partition
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // may
> have
> > > > > > > > > >>>>>>>> moved
> > > > > > > > > >>>>>>>>>> away
> > > > > > > > > >>>>>>>>>>>> from
> > > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> broker
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> from
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error was
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initially raised to
> > > > > > > > > >>>>>> the
> > > > > > > > > >>>>>>>> time
> > > > > > > > > >>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              //
> response
> > > > > > > > > >>>>>>> was
> > > > > > > > > >>>>>>>>>> being
> > > > > > > > > >>>>>>>>>>>>>>>>> constructed.
> > > > > > > > > >>>>>>>>>>>>>>>>>> In
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> these
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> cases,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retry the request: we
> > > > > > > > > >>>>>>> are
> > > > > > > > > >>>>>>>>>>>> guaranteed
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // to
> > > > > > > > > >>>>>>> eventually
> > > > > > > > > >>>>>>>>>> get a
> > > > > > > > > >>>>>>>>>>>>>>>>>> logStartOffset
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> once
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> things
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> settle
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> down.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return
> > true;
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > > > > > > > > >>>>>>>>>>>>>>> (batch.sequenceHasBeenReset()) {
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // When
> the
> > > > > > > > > >>>>>>>> first
> > > > > > > > > >>>>>>>>>>>> inflight
> > > > > > > > > >>>>>>>>>>>>>>>> batch
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> fails
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> due to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> truncation
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> then the sequences of
> > > > > > > > > >>>>>>> all
> > > > > > > > > >>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>> other
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // in
> > flight
> > > > > > > > > >>>>>>>>> batches
> > > > > > > > > >>>>>>>>>>>> would
> > > > > > > > > >>>>>>>>>>>>>>> have
> > > > > > > > > >>>>>>>>>>>>>>>>>> been
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> restarted
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> from
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> beginning.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, when those
> > > > > > > > > >>>>>>>>> responses
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // come
> > back
> > > > > > > > > >>>>>>>> from
> > > > > > > > > >>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>> broker,
> > > > > > > > > >>>>>>>>>>>>>>>>> they
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> would
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> also
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> come
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > > >>>>>>> error.
> > > > > > > > > >>>>>>>>> In
> > > > > > > > > >>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>> case,
> > > > > > > > > >>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> should
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // reset
> > the
> > > > > > > > > >>>>>>>>>> sequence
> > > > > > > > > >>>>>>>>>>>>>>> numbers
> > > > > > > > > >>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> beginning.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return
> > true;
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          } else if
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>> (lastAckedOffset(batch.topicPartition).orElse(
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> response.logStartOffset) {
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // The
> head
> > > > > > > > > >>>>>> of
> > > > > > > > > >>>>>>>> the
> > > > > > > > > >>>>>>>>>> log
> > > > > > > > > >>>>>>>>>>>> has
> > > > > > > > > >>>>>>>>>>>>>>> been
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> removed,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> probably
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> due
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retention time
> > > > > > > > > >>>>>> elapsing.
> > > > > > > > > >>>>>>>> In
> > > > > > > > > >>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>> case,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // we
> > expect
> > > > > > > > > >>>>>>> to
> > > > > > > > > >>>>>>>>>> lose the
> > > > > > > > > >>>>>>>>>>>>>>>> producer
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> state.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> For
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer, reset the
> > > > > > > > > >>>>>>>>> sequences
> > > > > > > > > >>>>>>>>>> of
> > > > > > > > > >>>>>>>>>>>> all
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              //
> inflight
> > > > > > > > > >>>>>>>>> batches
> > > > > > > > > >>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>> be
> > > > > > > > > >>>>>>>>>>>>>>> from
> > > > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> beginning
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> retry
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the transaction
> > > > > > > > > >>>>>>> does
> > > > > > > > > >>>>>>>>> not
> > > > > > > > > >>>>>>>>>>>> need to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // be
> > > > > > > > > >>>>>> aborted.
> > > > > > > > > >>>>>>>> For
> > > > > > > > > >>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>> idempotent
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> producer,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> bump
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> avoid
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reusing (sequence,
> > > > > > > > > >>>>>>> epoch)
> > > > > > > > > >>>>>>>>>> pairs
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              if
> > > > > > > > > >>>>>>>>>> (isTransactional()) {
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>
> > > > > > > >
> txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>> this.producerIdAndEpoch);
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              } else {
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>
> > > > requestEpochBumpForPartition(batch.topicPartition);
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              }
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return
> > true;
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > > > > > > > > >>>>>>>>> (!isTransactional())
> > > > > > > > > >>>>>>>>>> {
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // For
> the
> > > > > > > > > >>>>>>>>>> idempotent
> > > > > > > > > >>>>>>>>>>>>>>> producer,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> always
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> retry
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors. If the batch
> > > > > > > > > >>>>>> has
> > > > > > > > > >>>>>>>> the
> > > > > > > > > >>>>>>>>>>>> current
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              //
> producer
> > > > > > > > > >>>>>> ID
> > > > > > > > > >>>>>>>> and
> > > > > > > > > >>>>>>>>>>>> epoch,
> > > > > > > > > >>>>>>>>>>>>>>>>> request a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> bump
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Otherwise
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the
> > > > > > > > > >>>>>> produce.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>
> > > requestEpochBumpForPartition(batch.topicPartition);
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return
> > true;
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I was considering
> > > > > > > > > >>>>>>> keeping
> > > > > > > > > >>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>> behavior —
> > > > > > > > > >>>>>>>>>>>>>>>>> but
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> am
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> open
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simplifying
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We are leaving changes
> > > > > > > > > >>>>>>> to
> > > > > > > > > >>>>>>>>>> older
> > > > > > > > > >>>>>>>>>>>>>>> clients
> > > > > > > > > >>>>>>>>>>>>>>>> off
> > > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> table
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> here
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> since
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> caused
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many issues for
> > > > > > > > > >>>>>> clients
> > > > > > > > > >>>>>>> in
> > > > > > > > > >>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>> past.
> > > > > > > > > >>>>>>>>>>>>>>>>>> Previously
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> was
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we didn't have the
> > > > > > > > > >>>>>>>>> mechanisms
> > > > > > > > > >>>>>>>>>> in
> > > > > > > > > >>>>>>>>>>>>>>> place to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> detect
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> when
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> was
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> legitimate
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case vs some bug or
> > > > > > > > > >>>>>> gap
> > > > > > > > > >>>>>>> in
> > > > > > > > > >>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>> protocol.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> Ensuring
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> each
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> its
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> own epoch should close
> > > > > > > > > >>>>>>>> this
> > > > > > > > > >>>>>>>>>> gap.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> And to address Jeff's
> > > > > > > > > >>>>>>>> second
> > > > > > > > > >>>>>>>>>>>> point:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *does the typical
> > > > > > > > > >>>>>>> produce
> > > > > > > > > >>>>>>>>>> request
> > > > > > > > > >>>>>>>>>>>> path
> > > > > > > > > >>>>>>>>>>>>>>>>> append
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> records
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> local
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> log
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along*
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *with the
> > > > > > > > > >>>>>>>>>> currentTxnFirstOffset
> > > > > > > > > >>>>>>>>>>>>>>>>> information?
> > > > > > > > > >>>>>>>>>>>>>>>>>> I
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> would
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> like
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand*
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *when the field is
> > > > > > > > > >>>>>>> written
> > > > > > > > > >>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>> disk.*
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, the first produce
> > > > > > > > > >>>>>>>>> request
> > > > > > > > > >>>>>>>>>>>>>>> populates
> > > > > > > > > >>>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> field
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> writes
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> offset
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as part of the record
> > > > > > > > > >>>>>>>> batch
> > > > > > > > > >>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>> also
> > > > > > > > > >>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> producer
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> state
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> snapshot.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we reload the records
> > > > > > > > > >>>>>> on
> > > > > > > > > >>>>>>>>>> restart
> > > > > > > > > >>>>>>>>>>>>>>> and/or
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> reassignment,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> repopulate
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> field with the
> > > > > > > > > >>>>>> snapshot
> > > > > > > > > >>>>>>>> from
> > > > > > > > > >>>>>>>>>> disk
> > > > > > > > > >>>>>>>>>>>>>>> along
> > > > > > > > > >>>>>>>>>>>>>>>>> with
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> rest
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if there
> > > > > > > > > >>>>>> are
> > > > > > > > > >>>>>>>>>> further
> > > > > > > > > >>>>>>>>>>>>>>> comments
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> and/or
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> questions.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > > > > > > > > >>>>>> at
> > > > > > > > > >>>>>>>> 9:00
> > > > > > > > > >>>>>>>>>> PM
> > > > > > > > > >>>>>>>>>>>> Jeff
> > > > > > > > > >>>>>>>>>>>>>>> Kim
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>> <jeff.kim@confluent.io.invalid
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP! I
> > > > > > > > > >>>>>>>> have
> > > > > > > > > >>>>>>>>>> two
> > > > > > > > > >>>>>>>>>>>>>>>> questions:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1) For new clients,
> > > > > > > > > >>>>>> we
> > > > > > > > > >>>>>>>> can
> > > > > > > > > >>>>>>>>>> once
> > > > > > > > > >>>>>>>>>>>> again
> > > > > > > > > >>>>>>>>>>>>>>>>> return
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> an
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> error
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for sequences
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that are non-zero
> > > > > > > > > >>>>>> when
> > > > > > > > > >>>>>>>>> there
> > > > > > > > > >>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>> no
> > > > > > > > > >>>>>>>>>>>>>>>>> producer
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> state
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> present
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> on
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This will indicate we
> > > > > > > > > >>>>>>>>> missed
> > > > > > > > > >>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>> 0
> > > > > > > > > >>>>>>>>>>>>>>>>> sequence
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> yet
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> want
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the log.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would like to
> > > > > > > > > >>>>>>>> understand
> > > > > > > > > >>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>> current
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> behavior
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> handle
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and if there are any
> > > > > > > > > >>>>>>>>> changes
> > > > > > > > > >>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>> are
> > > > > > > > > >>>>>>>>>>>>>>>>> making.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> Maybe
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> I'm
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> missing
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but we would want to
> > > > > > > > > >>>>>>>>> identify
> > > > > > > > > >>>>>>>>>>>>>>> whether we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> missed
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the 0
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> sequence
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients, no?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2) Upon returning
> > > > > > > > > >>>>>> from
> > > > > > > > > >>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> coordinator, we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> set
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as ongoing on the
> > > > > > > > > >>>>>>> leader
> > > > > > > > > >>>>>>>> by
> > > > > > > > > >>>>>>>>>>>>>>> populating
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> currentTxnFirstOffset
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through the typical
> > > > > > > > > >>>>>>>> produce
> > > > > > > > > >>>>>>>>>>>> request
> > > > > > > > > >>>>>>>>>>>>>>>>>> handling.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does the typical
> > > > > > > > > >>>>>>> produce
> > > > > > > > > >>>>>>>>>> request
> > > > > > > > > >>>>>>>>>>>> path
> > > > > > > > > >>>>>>>>>>>>>>>>> append
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> records
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> local
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> log
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> > > > > > > > > >>>>>>>>>> currentTxnFirstOffset
> > > > > > > > > >>>>>>>>>>>>>>>>> information?
> > > > > > > > > >>>>>>>>>>>>>>>>>> I
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> would
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> like
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when the field is
> > > > > > > > > >>>>>>> written
> > > > > > > > > >>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>> disk.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeff
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > > > > > > > > >>>>>> at
> > > > > > > > > >>>>>>>>> 4:44
> > > > > > > > > >>>>>>>>>> PM
> > > > > > > > > >>>>>>>>>>>> Artem
> > > > > > > > > >>>>>>>>>>>>>>>>>> Livshits
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
> > > > > > > > > >>>>>>> alivshits@confluent.io
> > > > > > > > > >>>>>>>>>> .invalid>
> > > > > > > > > >>>>>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for the
> > > > > > > > > >>>>>> KIP.
> > > > > > > > > >>>>>>>> I
> > > > > > > > > >>>>>>>>>> have
> > > > > > > > > >>>>>>>>>>>> one
> > > > > > > > > >>>>>>>>>>>>>>>>>> question.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 5) For new clients,
> > > > > > > > > >>>>>> we
> > > > > > > > > >>>>>>>> can
> > > > > > > > > >>>>>>>>>> once
> > > > > > > > > >>>>>>>>>>>>>>> again
> > > > > > > > > >>>>>>>>>>>>>>>>>> return
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> an
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> error
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe we had
> > > > > > > > > >>>>>>>> problems
> > > > > > > > > >>>>>>>>>> in the
> > > > > > > > > >>>>>>>>>>>>>>> past
> > > > > > > > > >>>>>>>>>>>>>>>>> with
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> returning
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because it was
> > > > > > > > > >>>>>>>> considered
> > > > > > > > > >>>>>>>>>> fatal
> > > > > > > > > >>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>> required
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> client
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> restart.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> It
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good to spell out
> > > > > > > > > >>>>>> the
> > > > > > > > > >>>>>>>> new
> > > > > > > > > >>>>>>>>>> client
> > > > > > > > > >>>>>>>>>>>>>>>> behavior
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> when
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> it
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> receives
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > > > > > > > > >>>>>>> at
> > > > > > > > > >>>>>>>>>> 10:00 AM
> > > > > > > > > >>>>>>>>>>>>>>>> Justine
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> Olshan
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > > > > > >>>>>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
> > > > > > > > > >>>>>>>> look
> > > > > > > > > >>>>>>>>>>>> Matthias.
> > > > > > > > > >>>>>>>>>>>>>>>> I've
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> tried
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> answer
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> your
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> questions
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 10)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Right — so the
> > > > > > > > > >>>>>>> hanging
> > > > > > > > > >>>>>>>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>>>>>> only
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> occurs
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> when
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> come in and the
> > > > > > > > > >>>>>>>> partition
> > > > > > > > > >>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>> never
> > > > > > > > > >>>>>>>>>>>>>>>> added
> > > > > > > > > >>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> again.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never add the
> > > > > > > > > >>>>>>> partition
> > > > > > > > > >>>>>>>>> to
> > > > > > > > > >>>>>>>>>> a
> > > > > > > > > >>>>>>>>>>>>>>>>> transaction,
> > > > > > > > > >>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> will
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> never
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> marker
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never advance the
> > > > > > > > > >>>>>>> LSO.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do end up
> > > > > > > > > >>>>>>> adding
> > > > > > > > > >>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>> partition
> > > > > > > > > >>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (I
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suppose
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happen before or
> > > > > > > > > >>>>>>> after
> > > > > > > > > >>>>>>>>> the
> > > > > > > > > >>>>>>>>>> late
> > > > > > > > > >>>>>>>>>>>>>>>> message
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> comes
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> in)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> then
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> include
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late message in the
> > > > > > > > > >>>>>>>> next
> > > > > > > > > >>>>>>>>>>>>>>> (incorrect)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> transaction.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> So perhaps it is
> > > > > > > > > >>>>>>>> clearer
> > > > > > > > > >>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>> make
> > > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> distinction
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> between
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> eventually get
> > > > > > > > > >>>>>> added
> > > > > > > > > >>>>>>> to
> > > > > > > > > >>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>>>>>>>> (but
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> wrong
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> one)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> or
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that never get
> > > > > > > > > >>>>>> added
> > > > > > > > > >>>>>>>> and
> > > > > > > > > >>>>>>>>>> become
> > > > > > > > > >>>>>>>>>>>>>>>> hanging.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side
> > > > > > > > > >>>>>>> change
> > > > > > > > > >>>>>>>>> for
> > > > > > > > > >>>>>>>>>> 2 is
> > > > > > > > > >>>>>>>>>>>>>>>> removing
> > > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> addPartitions
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need
> > > > > > > > > >>>>>>> to
> > > > > > > > > >>>>>>>>> make
> > > > > > > > > >>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>> from
> > > > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In my opinion, the
> > > > > > > > > >>>>>>>> issue
> > > > > > > > > >>>>>>>>>> with
> > > > > > > > > >>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> call
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is that we don't
> > > > > > > > > >>>>>> have
> > > > > > > > > >>>>>>>> the
> > > > > > > > > >>>>>>>>>> epoch
> > > > > > > > > >>>>>>>>>>>>>>> bump,
> > > > > > > > > >>>>>>>>>>>>>>>> so
> > > > > > > > > >>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> don't
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> know
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> belongs to the
> > > > > > > > > >>>>>>> previous
> > > > > > > > > >>>>>>>>>>>>>>> transaction or
> > > > > > > > > >>>>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> one.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> We
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition has been
> > > > > > > > > >>>>>>>> added
> > > > > > > > > >>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> transaction.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> Of
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> course,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> means
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't completely
> > > > > > > > > >>>>>>> cover
> > > > > > > > > >>>>>>>>> the
> > > > > > > > > >>>>>>>>>> case
> > > > > > > > > >>>>>>>>>>>>>>> where
> > > > > > > > > >>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> have a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> really
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have added the
> > > > > > > > > >>>>>>>> partition
> > > > > > > > > >>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>> the new
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> transaction,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> that's
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something we will
> > > > > > > > > >>>>>>> need
> > > > > > > > > >>>>>>>>> the
> > > > > > > > > >>>>>>>>>> new
> > > > > > > > > >>>>>>>>>>>>>>> clients
> > > > > > > > > >>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> cover.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is
> > > > > > > > > >>>>>>> ongoing
> > > > > > > > > >>>>>>>> =
> > > > > > > > > >>>>>>>>>>>> partition
> > > > > > > > > >>>>>>>>>>>>>>> was
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> added to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> via
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn.
> > > > > > > > > >>>>>>> We
> > > > > > > > > >>>>>>>>>> check
> > > > > > > > > >>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>> with
> > > > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this
> > > > > > > > > >>>>>>> wasn't
> > > > > > > > > >>>>>>>>>>>> sufficiently
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> explained
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> here:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>
> > > > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is
> > > > > > > > > >>>>>> that
> > > > > > > > > >>>>>>>> if
> > > > > > > > > >>>>>>>>>> any
> > > > > > > > > >>>>>>>>>>>>>>> messages
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> somehow
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> come
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> before
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the
> > > > > > > > > >>>>>>> producer,
> > > > > > > > > >>>>>>>>> they
> > > > > > > > > >>>>>>>>>>>> will be
> > > > > > > > > >>>>>>>>>>>>>>>>>> fenced.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> However,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it
> > > > > > > > > >>>>>> can
> > > > > > > > > >>>>>>> be
> > > > > > > > > >>>>>>>>>>>> discussed
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 50)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It should be
> > > > > > > > > >>>>>>>> synchronous
> > > > > > > > > >>>>>>>>>>>> because
> > > > > > > > > >>>>>>>>>>>>>>> if we
> > > > > > > > > >>>>>>>>>>>>>>>>>> have
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> an
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> event
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (ie,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> an
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes us to need
> > > > > > > > > >>>>>> to
> > > > > > > > > >>>>>>>>> abort
> > > > > > > > > >>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>> transaction,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> need
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions to send
> > > > > > > > > >>>>>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>>>>> markers
> > > > > > > > > >>>>>>>>>>>>>>>>> to.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> We
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> know
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we added them to
> > > > > > > > > >>>>>> the
> > > > > > > > > >>>>>>>>>>>> coordinator
> > > > > > > > > >>>>>>>>>>>>>>> via
> > > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Previously we have
> > > > > > > > > >>>>>>> had
> > > > > > > > > >>>>>>>>>>>> asynchronous
> > > > > > > > > >>>>>>>>>>>>>>>>> calls
> > > > > > > > > >>>>>>>>>>>>>>>>>> in
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> past
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (ie,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit markers when
> > > > > > > > > >>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>> transaction is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> completed)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> often
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes confusion as
> > > > > > > > > >>>>>>> we
> > > > > > > > > >>>>>>>>>> need to
> > > > > > > > > >>>>>>>>>>>> wait
> > > > > > > > > >>>>>>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>> some
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> operations
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complete.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing commit
> > > > > > > > > >>>>>>> markers
> > > > > > > > > >>>>>>>>>> case,
> > > > > > > > > >>>>>>>>>>>>>>> clients
> > > > > > > > > >>>>>>>>>>>>>>>>> often
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> see
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>> CONCURRENT_TRANSACTIONs
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error messages and
> > > > > > > > > >>>>>>> that
> > > > > > > > > >>>>>>>>>> can be
> > > > > > > > > >>>>>>>>>>>>>>>>> confusing.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> For
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> reason,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> may
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simpler to just
> > > > > > > > > >>>>>> have
> > > > > > > > > >>>>>>>>>>>> synchronous
> > > > > > > > > >>>>>>>>>>>>>>>> calls —
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> especially
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> block
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some operation's
> > > > > > > > > >>>>>>>>> completion
> > > > > > > > > >>>>>>>>>>>> anyway
> > > > > > > > > >>>>>>>>>>>>>>>>> before
> > > > > > > > > >>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> can
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> start
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> next
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. And
> > > > > > > > > >>>>>>> yes, I
> > > > > > > > > >>>>>>>>>> meant
> > > > > > > > > >>>>>>>>>>>>>>>>>> coordinator. I
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> will
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> fix
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> that.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are
> > > > > > > > > >>>>>> checking
> > > > > > > > > >>>>>>> if
> > > > > > > > > >>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> ongoing,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the
> > > > > > > > > >>>>>> leader
> > > > > > > > > >>>>>>>>>> partition
> > > > > > > > > >>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for
> > > > > > > > > >>>>>>> this
> > > > > > > > > >>>>>>>>>>>> message to
> > > > > > > > > >>>>>>>>>>>>>>>> come
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> back,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> in
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> theory
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
> > > > > > > > > >>>>>>>> that
> > > > > > > > > >>>>>>>>>> would
> > > > > > > > > >>>>>>>>>>>> make
> > > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> original
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> result
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why
> > > > > > > > > >>>>>> we
> > > > > > > > > >>>>>>>> can
> > > > > > > > > >>>>>>>>>> check
> > > > > > > > > >>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>> leader
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> state
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> before
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm happy to update
> > > > > > > > > >>>>>>> the
> > > > > > > > > >>>>>>>>>> KIP if
> > > > > > > > > >>>>>>>>>>>>>>> some of
> > > > > > > > > >>>>>>>>>>>>>>>>>> these
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> things
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> were
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Nov 21,
> > > > > > > > > >>>>>> 2022
> > > > > > > > > >>>>>>> at
> > > > > > > > > >>>>>>>>>> 7:11 PM
> > > > > > > > > >>>>>>>>>>>>>>>> Matthias
> > > > > > > > > >>>>>>>>>>>>>>>>>> J.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> Sax <
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the
> > > > > > > > > >>>>>> KIP.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Couple of
> > > > > > > > > >>>>>>>> clarification
> > > > > > > > > >>>>>>>>>>>> questions
> > > > > > > > > >>>>>>>>>>>>>>> (I
> > > > > > > > > >>>>>>>>>>>>>>>> am
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> not a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> broker
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> expert
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> do
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some question are
> > > > > > > > > >>>>>>>>> obvious
> > > > > > > > > >>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>> others,
> > > > > > > > > >>>>>>>>>>>>>>>>> but
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> not
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> me
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> my
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lack
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker knowledge).
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (10)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What happens if
> > > > > > > > > >>>>>> the
> > > > > > > > > >>>>>>>>>> message
> > > > > > > > > >>>>>>>>>>>> come
> > > > > > > > > >>>>>>>>>>>>>>> in
> > > > > > > > > >>>>>>>>>>>>>>>>>> before
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> next
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request? It seems
> > > > > > > > > >>>>>>> the
> > > > > > > > > >>>>>>>>>> broker
> > > > > > > > > >>>>>>>>>>>>>>> hosting
> > > > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> data
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anything about it
> > > > > > > > > >>>>>>> and
> > > > > > > > > >>>>>>>>>> append
> > > > > > > > > >>>>>>>>>>>> it to
> > > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> partition,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> too?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> What
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> difference between
> > > > > > > > > >>>>>>>> both
> > > > > > > > > >>>>>>>>>> cases?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, it seems a
> > > > > > > > > >>>>>> TX
> > > > > > > > > >>>>>>>>> would
> > > > > > > > > >>>>>>>>>> only
> > > > > > > > > >>>>>>>>>>>>>>> hang,
> > > > > > > > > >>>>>>>>>>>>>>>> if
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> there
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> is no
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> following
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> either committer
> > > > > > > > > >>>>>> or
> > > > > > > > > >>>>>>>>>> aborted?
> > > > > > > > > >>>>>>>>>>>> Thus,
> > > > > > > > > >>>>>>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> case
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> above,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> TX
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> might
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually not hang
> > > > > > > > > >>>>>>> (of
> > > > > > > > > >>>>>>>>>> course,
> > > > > > > > > >>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>> might
> > > > > > > > > >>>>>>>>>>>>>>>>>> get
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> an
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> EOS
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> violation
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX was aborted and
> > > > > > > > > >>>>>>> the
> > > > > > > > > >>>>>>>>>> second
> > > > > > > > > >>>>>>>>>>>>>>>>> committed,
> > > > > > > > > >>>>>>>>>>>>>>>>>> or
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> other
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> way
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around).
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (20)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Of course, 1 and
> > > > > > > > > >>>>>> 2
> > > > > > > > > >>>>>>>>>> require
> > > > > > > > > >>>>>>>>>>>>>>>> client-side
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> changes, so
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> those approaches
> > > > > > > > > >>>>>>> won’t
> > > > > > > > > >>>>>>>>>> apply.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For (1) I
> > > > > > > > > >>>>>> understand
> > > > > > > > > >>>>>>>>> why a
> > > > > > > > > >>>>>>>>>>>> client
> > > > > > > > > >>>>>>>>>>>>>>>>> change
> > > > > > > > > >>>>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> necessary,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we need a client
> > > > > > > > > >>>>>>>> change
> > > > > > > > > >>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>> (2).
> > > > > > > > > >>>>>>>>>>>>>>> Can
> > > > > > > > > >>>>>>>>>>>>>>>>> you
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> --
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Later
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> explain
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that we should
> > > > > > > > > >>>>>> send
> > > > > > > > > >>>>>>> a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> DescribeTransactionRequest,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> but I
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> am
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Can't we not just
> > > > > > > > > >>>>>> do
> > > > > > > > > >>>>>>>> an
> > > > > > > > > >>>>>>>>>>>> implicit
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> AddPartiitonToTx,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> too?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> If
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> old
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer correctly
> > > > > > > > > >>>>>>>>>> registered
> > > > > > > > > >>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>> partition
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> already,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can just ignore it
> > > > > > > > > >>>>>>> as
> > > > > > > > > >>>>>>>>>> it's an
> > > > > > > > > >>>>>>>>>>>>>>>>> idempotent
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> operation?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (30)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To cover older
> > > > > > > > > >>>>>>>> clients,
> > > > > > > > > >>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>> will
> > > > > > > > > >>>>>>>>>>>>>>>>> ensure a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ongoing
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> before
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we write to a
> > > > > > > > > >>>>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Not sure what you
> > > > > > > > > >>>>>>> mean
> > > > > > > > > >>>>>>>>> by
> > > > > > > > > >>>>>>>>>>>> this?
> > > > > > > > > >>>>>>>>>>>>>>> Can
> > > > > > > > > >>>>>>>>>>>>>>>> you
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (40)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [the
> > > > > > > > > >>>>>>> TX-coordinator]
> > > > > > > > > >>>>>>>>> will
> > > > > > > > > >>>>>>>>>>>> write
> > > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> prepare
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> commit
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch and send
> > > > > > > > > >>>>>>>>>>>>>>> WriteTxnMarkerRequests
> > > > > > > > > >>>>>>>>>>>>>>>>>> with
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> bumped
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why do we use the
> > > > > > > > > >>>>>>>> bumped
> > > > > > > > > >>>>>>>>>>>> epoch for
> > > > > > > > > >>>>>>>>>>>>>>>>> both?
> > > > > > > > > >>>>>>>>>>>>>>>>>> It
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> seems
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> more
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> intuitive
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the current epoch,
> > > > > > > > > >>>>>>> and
> > > > > > > > > >>>>>>>>>> only
> > > > > > > > > >>>>>>>>>>>> return
> > > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> bumped
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (50) "Implicit
> > > > > > > > > >>>>>>>>>>>>>>>>> AddPartitionToTransaction"
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why does the
> > > > > > > > > >>>>>>>> implicitly
> > > > > > > > > >>>>>>>>>> sent
> > > > > > > > > >>>>>>>>>>>>>>> request
> > > > > > > > > >>>>>>>>>>>>>>>>> need
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> be
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> synchronous?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also says
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in case we need
> > > > > > > > > >>>>>> to
> > > > > > > > > >>>>>>>>> abort
> > > > > > > > > >>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>> need to
> > > > > > > > > >>>>>>>>>>>>>>>>>> know
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> which
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What do you mean
> > > > > > > > > >>>>>> by
> > > > > > > > > >>>>>>>>> this?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we don’t want to
> > > > > > > > > >>>>>>>> write
> > > > > > > > > >>>>>>>>>> to it
> > > > > > > > > >>>>>>>>>>>>>>> before
> > > > > > > > > >>>>>>>>>>>>>>>> we
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> store
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> in
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> manager
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Do you mean
> > > > > > > > > >>>>>>>>> TX-coordinator
> > > > > > > > > >>>>>>>>>>>>>>> instead of
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> "manager"?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (60)
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For older clients
> > > > > > > > > >>>>>>> and
> > > > > > > > > >>>>>>>>>> ensuring
> > > > > > > > > >>>>>>>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > > >>>>>>>>>>>>>>>>>> TX
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> ongoing,
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> describe a
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> race condition. I
> > > > > > > > > >>>>>> am
> > > > > > > > > >>>>>>>> not
> > > > > > > > > >>>>>>>>>> sure
> > > > > > > > > >>>>>>>>>>>> if I
> > > > > > > > > >>>>>>>>>>>>>>>> can
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> follow
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> here.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> Can
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/18/22 1:21
> > > > > > > > > >>>>>> PM,
> > > > > > > > > >>>>>>>>>> Justine
> > > > > > > > > >>>>>>>>>>>>>>> Olshan
> > > > > > > > > >>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey all!
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to
> > > > > > > > > >>>>>> start a
> > > > > > > > > >>>>>>>>>>>> discussion
> > > > > > > > > >>>>>>>>>>>>>>> on my
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> proposal
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> add
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> some
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checks on
> > > > > > > > > >>>>>>>> transactions
> > > > > > > > > >>>>>>>>> to
> > > > > > > > > >>>>>>>>>>>> avoid
> > > > > > > > > >>>>>>>>>>>>>>>>> hanging
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> transactions.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> been
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an issue for some
> > > > > > > > > >>>>>>>> time,
> > > > > > > > > >>>>>>>>>> so I
> > > > > > > > > >>>>>>>>>>>>>>> really
> > > > > > > > > >>>>>>>>>>>>>>>>> hope
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> this
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> KIP
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> helpful
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users of EOS.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The KIP includes
> > > > > > > > > >>>>>>>>> changes
> > > > > > > > > >>>>>>>>>> that
> > > > > > > > > >>>>>>>>>>>>>>> will
> > > > > > > > > >>>>>>>>>>>>>>>> be
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> compatible
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> old
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> changes to
> > > > > > > > > >>>>>> improve
> > > > > > > > > >>>>>>>>>>>> performance
> > > > > > > > > >>>>>>>>>>>>>>> and
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>> correctness
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> on
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients.
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Please take a
> > > > > > > > > >>>>>> look
> > > > > > > > > >>>>>>>> and
> > > > > > > > > >>>>>>>>>> leave
> > > > > > > > > >>>>>>>>>>>> any
> > > > > > > > > >>>>>>>>>>>>>>>>>> comments
> > > > > > > > > >>>>>>>>>>>>>>>>>>>> you
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> may
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> have!
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>
> > > > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> JIRA:
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>
> https://issues.apache.org/jira/browse/KAFKA-14402
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>>>
> > > > > > > > > >>>>>>>>>>
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>
> > > > > > > > > >>
> > > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Jun Rao <ju...@confluent.io.INVALID>.
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


On Tue, Apr 18, 2023 at 9:27 AM Justine Olshan <jo...@confluent.io.invalid>
wrote:

> Hi Daniel,
>
> You are correct. We need to slightly adjust the response from how it
> appeared in the KIP. I've been focusing on part 1 (old clients) but when we
> get into part 2, I will address this issue. I will also update this thread
> when we do.
> Thanks for bringing it up.
>
> Justine.
>
> On Tue, Apr 18, 2023 at 12:28 AM Dániel Urbán <ur...@gmail.com>
> wrote:
>
> > Hi,
> >
> > Sorry for being super-late to this discussion, but a question came to my
> > mind related to the bump-on-txn-end part of the KIP.
> > I see that the END_TXN_RESPONSE will be updated to also include the new
> > epoch to be used by the producer - but what about epoch exhaustion? What
> > will happen when the last ongoing transaction uses the max available
> epoch,
> > and we try to end it? The producer would need a new producer ID after the
> > epoch bump. Because of this, I think the response should also contain the
> > producer-id-to-use.
> >
> > Thanks,
> > Daniel
> >
> > Artem Livshits <al...@confluent.io.invalid> ezt írta (időpont: 2023.
> > jan. 28., Szo, 0:17):
> >
> > > >  I'd like to check that, under this KIP's circumstances, if we could
> > ever
> > > have "partial committed" txns
> > >
> > > The goal of this KIP is to harden transaction implementation to never
> > have
> > > partially committed txns.
> > >
> > > -Artem
> > >
> > > On Fri, Jan 27, 2023 at 10:04 AM Guozhang Wang <
> > guozhang.wang.us@gmail.com
> > > >
> > > wrote:
> > >
> > > > Thanks folks, I agree with all of your points.
> > > >
> > > > I think in the past we did have some cases where abortable txns are
> > > > handled as fatal ones which we are fixing, but maybe there are some
> > > > vice versa as well. I'd like to check that, under this KIP's
> > > > circumstances, if we could ever have "partial committed" txns where
> > > > some of the data are considered as committed while some others are
> > > > aborted later. And if there were, could we recover from it. If the
> > > > answer is "yes and yes", then I think we can bite the complexity
> > > > bullet inside the client and leave users in comfort and peace :)
> > > >
> > > > Guozhang
> > > >
> > > > On Fri, Jan 27, 2023 at 9:42 AM Artem Livshits
> > > > <al...@confluent.io.invalid> wrote:
> > > > >
> > > > > I agree.  I also think we should go a little further and be more
> > > > > prescriptive with our recommendations for error handling (i.e. not
> > just
> > > > > think if there is a chance, but propose design patterns for typical
> > > > > cases).  For example, if a transaction is aborted, the application
> > must
> > > > > either keep enough state to be able to redo the operation from the
> > > > > beginning, or reset all state and fully recover from Kafka (i.e.
> > > > re-consume
> > > > > data from the last committed offsets).
> > > > >
> > > > > For failed commits, we need a way for the application to either
> learn
> > > the
> > > > > result (if commit went through, then continue, otherwise redo
> > > operation)
> > > > or
> > > > > just reset all state and fully recover from Kafka (i.e. re-consume
> > data
> > > > > from the last committed offsets).
> > > > >
> > > > > -Artem
> > > > >
> > > > > On Fri, Jan 27, 2023 at 9:31 AM Justine Olshan
> > > > <jo...@confluent.io.invalid>
> > > > > wrote:
> > > > >
> > > > > > Thanks for clarifying Guozhang,
> > > > > >
> > > > > > I like the framing:
> > > > > > * “if we did abort the txn and continue, could the app ever has a
> > > > chance to
> > > > > > recover and not have any side effects violating EOS guarantees”.*
> > > > > >
> > > > > > With this principle in mind, if we could convince ourselves that
> we
> > > > won't
> > > > > > have side effects, then we could move forward with a non-fatal
> > > error. I
> > > > > > appreciate laying out the cases we've discussed. I think I also
> > want
> > > to
> > > > > > take a look in very close detail and convince myself of the right
> > > > option.
> > > > > > I'll get back to this thread with the final conclusion as well as
> > > > update
> > > > > > the KIP.
> > > > > >
> > > > > > Thanks for also bringing up KIP-691. I should take a look there
> as
> > > > well. :)
> > > > > >
> > > > > > Justine
> > > > > >
> > > > > > On Thu, Jan 26, 2023 at 5:28 AM Guozhang Wang <
> > > > guozhang.wang.us@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hello Justine,
> > > > > > >
> > > > > > > Regarding which errors should be fatal v.s. abortable, I think
> > the
> > > > > > > principle would be “if we did abort the txn and continue, could
> > the
> > > > > > > app ever has a chance to recover and not have any side effects
> > > > > > > violating EOS guarantees”.Of course we can discuss whether
> > that’s a
> > > > > > > good principle  e.g. one can argue that maybe it’s okay to
> > violate
> > > > EOS
> > > > > > > some times rather than killing the whole app, whereas my
> > rationale
> > > is
> > > > > > > that since we want to make EOS the default config, it’s crucial
> > > that
> > > > > > > we try to guarantee it as much as possible and whenever we
> > detected
> > > > if
> > > > > > > there’s a chance that it maybe broken, we should let users
> know.
> > > And
> > > > > > > if we just log an error in app and try to abort and continue,
> > it’s
> > > > > > > very likely that users would not be notified and probably only
> > > > realize
> > > > > > > that after a while, which may cause an even harder scenario to
> > > > > > > debug.If people agree on that principle, we can go back and
> check
> > > > this
> > > > > > > KIP’s scenario: there are several scenarios causing a partition
> > > > leader
> > > > > > > detects an out of date epoch from a produce requests:
> > > > > > >
> > > > > > > * The old produce requests arrive late after a network
> partition,
> > > and
> > > > > > > the old producer is already gone.
> > > > > > > * The old produce requests arrive late after a network
> partition,
> > > but
> > > > > > > the old producer is still around.
> > > > > > > * A buggy producer client that did not follow the protocol.
> > > > > > >
> > > > > > > For the third case, we should try to let user know asap, and as
> > > clear
> > > > > > > as possible, and hence it’s best to just stop the client app;
> for
> > > the
> > > > > > > first case, it does not matter since the producer is already
> > gone;
> > > > > > > only for the second, probably the least likely case, we need to
> > > > > > > ponder, and there my rationale again is that at that time, we
> may
> > > > have
> > > > > > > already violated the EOS guarantees since there are some
> partial
> > > txn
> > > > > > > records that should be rejected while the txn itself has been
> > > > > > > committed. In this case I think it’s better to let
> clients/users
> > > know
> > > > > > > as soon as possible as well.
> > > > > > >
> > > > > > > Regarding the past scenarios where a fatal error killing the
> > whole
> > > > > > > apps, I believe part of that reason is that we were doing an
> > > > > > > sub-optimal job on clients side handling various error cases
> and
> > > > > > > that’s what KIP-691 is trying to resolve, and hence personally
> I
> > > > would
> > > > > > > suggest we do not weight in too much on that if we can trust
> that
> > > > > > > KIP-691 will be successfully eliminate those
> > > not-necessary-hard-fail
> > > > > > > scenarios.
> > > > > > >
> > > > > > >
> > > > > > > Guozhang
> > > > > > >
> > > > > > > On Wed, Jan 25, 2023 at 5:51 PM Matthias J. Sax <
> > mjsax@apache.org>
> > > > > > wrote:
> > > > > > > >
> > > > > > > > So the timestamp would be set when the write happens and thus
> > no
> > > > > > > > out-of-order data (base in time) can be introduced with
> > > > "append_time"
> > > > > > > > config even if a request sits in purgatory first while we
> check
> > > > the TX
> > > > > > > > status.
> > > > > > > >
> > > > > > > > That does make sense. Thanks for confirming, that there is no
> > > > > > > > out-of-order issue for this case.
> > > > > > > >
> > > > > > > >
> > > > > > > > -Matthias
> > > > > > > >
> > > > > > > > On 1/25/23 5:04 PM, Justine Olshan wrote:
> > > > > > > > > Hey Matthias,
> > > > > > > > > Let me put it this way, if a producer is checking if a
> > > > transaction is
> > > > > > > > > ongoing, then no writes to the partition from the producer
> > will
> > > > go
> > > > > > > through
> > > > > > > > > until the transaction is confirmed ongoing.
> > > > > > > > >  From then, I think I can apply the writes in the order
> they
> > > > came in.
> > > > > > > Does
> > > > > > > > > that make sense?
> > > > > > > > >
> > > > > > > > > Let me know if I'm missing something.
> > > > > > > > > Justine
> > > > > > > > >
> > > > > > > > > On Wed, Jan 25, 2023 at 4:57 PM Matthias J. Sax <
> > > > mjsax@apache.org>
> > > > > > > wrote:
> > > > > > > > >
> > > > > > > > >>> would it build an offset map with just the latest
> timestamp
> > > > for a
> > > > > > > key?
> > > > > > > > >>
> > > > > > > > >> Cannot remember the details without reading the KIP, but
> > yes,
> > > > > > > something
> > > > > > > > >> like this (I believe it actually needs to track both,
> offset
> > > and
> > > > > > > > >> timestamp per key).
> > > > > > > > >>
> > > > > > > > >>> I wonder if ordering assumptions are baked in there, why
> > not
> > > > use
> > > > > > > > >> offset-based compaction.
> > > > > > > > >>
> > > > > > > > >> The use case is a compacted topic that does contain
> > > out-of-order
> > > > > > data.
> > > > > > > > >> If you set key k1=v1 @ 5 offset 100 and later key1 = v0 @
> 3
> > at
> > > > > > offset
> > > > > > > > >> 200 we want to cleanup v0 with higher offset because it's
> > > > > > out-of-order
> > > > > > > > >> based on time, but keep v1 what is the actual latest
> version
> > > of
> > > > k1.
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >>> I was also not aware of this "guarantee" with regards to
> > > broker
> > > > > > side
> > > > > > > > >> time.
> > > > > > > > >>
> > > > > > > > >> As already said: I am not sure if it's a public contract,
> > but
> > > > based
> > > > > > on
> > > > > > > > >> my experience, people might reply on it as "implicit
> > > contract".
> > > > --
> > > > > > > Maybe
> > > > > > > > >> somebody else knows if it's public or not, and if it would
> > be
> > > > ok to
> > > > > > > > >> "break" it.
> > > > > > > > >>
> > > > > > > > >>> Let me know if you have any concerns here.
> > > > > > > > >>
> > > > > > > > >> My understanding is: While we cannot make an offset-order
> > > > guarantee
> > > > > > > for
> > > > > > > > >> interleaved writes of different producer, if the topic is
> > > > configures
> > > > > > > > >> with "append_time", we "guarantee" (cf. my comment above")
> > > > timestamp
> > > > > > > > >> order... If that's the case, it would be an issue if we
> > break
> > > > this
> > > > > > > > >> "guarantee".
> > > > > > > > >>
> > > > > > > > >> I am not sure when the broker sets the timestamp for
> > > > "append_time"
> > > > > > > > >> config? If we do it before putting the request into
> > purgatory,
> > > > we
> > > > > > > have a
> > > > > > > > >> problem. However, if we set the timestamp when we actually
> > > > process
> > > > > > the
> > > > > > > > >> request and do the actual append, it seems there is no
> > issue,
> > > > as the
> > > > > > > > >> request that was waiting in purgatory get the "newest"
> > > > timestamp and
> > > > > > > > >> thus cannot introduce out-of-order data.
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> -Matthias
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> On 1/24/23 10:44 AM, Justine Olshan wrote:
> > > > > > > > >>> Hey Matthias,
> > > > > > > > >>>
> > > > > > > > >>> I have actually never heard of KIP-280 so thanks for
> > bringing
> > > > it
> > > > > > up.
> > > > > > > That
> > > > > > > > >>> seems interesting. I wonder how it would work though --
> > would
> > > > it
> > > > > > > build an
> > > > > > > > >>> offset map with just the latest timestamp for a key? I
> > wonder
> > > > if
> > > > > > > ordering
> > > > > > > > >>> assumptions are baked in there, why not use offset-based
> > > > > > compaction.
> > > > > > > > >>>
> > > > > > > > >>> I was also not aware of this "guarantee" with regards to
> > > broker
> > > > > > side
> > > > > > > > >> time.
> > > > > > > > >>> I think that we can do in order handling for a given
> > > producer,
> > > > but
> > > > > > > not
> > > > > > > > >>> across all producers. However, we can't guarantee that
> > > anyway.
> > > > > > > > >>>
> > > > > > > > >>> Let me know if you have any concerns here.
> > > > > > > > >>>
> > > > > > > > >>> Thanks,
> > > > > > > > >>> Justine
> > > > > > > > >>>
> > > > > > > > >>> On Mon, Jan 23, 2023 at 6:33 PM Matthias J. Sax <
> > > > mjsax@apache.org>
> > > > > > > > >> wrote:
> > > > > > > > >>>
> > > > > > > > >>>> Just a side note about Guozhang comments about
> timestamps.
> > > > > > > > >>>>
> > > > > > > > >>>> If the producer sets the timestamp, putting the record
> > into
> > > > > > > purgatory
> > > > > > > > >>>> seems not to be an issue (as already said: for this case
> > we
> > > > don't
> > > > > > > > >>>> guarantee timestamp order between writes of different
> > > > producers
> > > > > > > anyway).
> > > > > > > > >>>> However, if the broker sets the timestamp, the
> expectation
> > > is
> > > > that
> > > > > > > there
> > > > > > > > >>>> is no out-of-order data in the partition ever; if we
> would
> > > > > > introduce
> > > > > > > > >>>> out-of-order data for this case (for interleaved writes
> of
> > > > > > different
> > > > > > > > >>>> producers), it seems we would violate the current
> > contract?
> > > > (To be
> > > > > > > fair:
> > > > > > > > >>>> I don't know if that's an official contract, but I
> assume
> > > > people
> > > > > > > rely on
> > > > > > > > >>>> this behavior -- and it "advertised" in many public
> > > talks...)
> > > > > > > > >>>>
> > > > > > > > >>>> About compaction: there is actually KIP-280 that adds
> > > > timestamp
> > > > > > > based
> > > > > > > > >>>> compaction what is a very useful feature for Kafka
> Streams
> > > > with
> > > > > > > regard
> > > > > > > > >>>> to out-of-order data handling. So the impact if we
> > introduce
> > > > > > > > >>>> out-of-order data could be larger scoped.
> > > > > > > > >>>>
> > > > > > > > >>>>
> > > > > > > > >>>> -Matthias
> > > > > > > > >>>>
> > > > > > > > >>>>
> > > > > > > > >>>> On 1/20/23 4:48 PM, Justine Olshan wrote:
> > > > > > > > >>>>> Hey Artem,
> > > > > > > > >>>>>
> > > > > > > > >>>>> I see there is a check for transactional producers. I'm
> > > > wondering
> > > > > > > if we
> > > > > > > > >>>>> don't handle the epoch overflow case. I'm also not sure
> > it
> > > > will
> > > > > > be
> > > > > > > a
> > > > > > > > >> huge
> > > > > > > > >>>>> issue to extend to transactional producers, but maybe
> I'm
> > > > missing
> > > > > > > > >>>> something.
> > > > > > > > >>>>>
> > > > > > > > >>>>> As for the recovery path -- I think Guozhang's point
> was
> > if
> > > > we
> > > > > > > have a
> > > > > > > > >> bad
> > > > > > > > >>>>> client that repeatedly tries to produce without adding
> to
> > > the
> > > > > > > > >> transaction
> > > > > > > > >>>>> we would do the following:
> > > > > > > > >>>>> a) if not fatal, we just fail the produce request over
> > and
> > > > over
> > > > > > > > >>>>> b) if fatal, we fence the producer
> > > > > > > > >>>>>
> > > > > > > > >>>>> Here with B, the issue with the client would be made
> > clear
> > > > more
> > > > > > > > >> quickly.
> > > > > > > > >>>> I
> > > > > > > > >>>>> suppose there are some intermediate cases where the
> issue
> > > > only
> > > > > > > occurs
> > > > > > > > >>>>> sometimes, but I wonder if we should consider how to
> > > recover
> > > > with
> > > > > > > > >> clients
> > > > > > > > >>>>> who don't behave as expected anyway.
> > > > > > > > >>>>>
> > > > > > > > >>>>> I think there is a place for the abortable error that
> we
> > > are
> > > > > > > adding --
> > > > > > > > >>>> just
> > > > > > > > >>>>> abort and try again. But I think there are also some
> > cases
> > > > where
> > > > > > > trying
> > > > > > > > >>>> to
> > > > > > > > >>>>> recover overcomplicates some logic. Especially if we
> are
> > > > > > > considering
> > > > > > > > >>>> older
> > > > > > > > >>>>> clients -- there I'm not sure if there's a ton we can
> do
> > > > besides
> > > > > > > fail
> > > > > > > > >> the
> > > > > > > > >>>>> batch or fence the producer. With newer clients, we can
> > > > consider
> > > > > > > more
> > > > > > > > >>>>> options for what can just be recovered after aborting.
> > But
> > > > epochs
> > > > > > > might
> > > > > > > > >>>> be
> > > > > > > > >>>>> a hard one unless we also want to reset producer ID.
> > > > > > > > >>>>>
> > > > > > > > >>>>> Thanks,
> > > > > > > > >>>>> Justine
> > > > > > > > >>>>>
> > > > > > > > >>>>>
> > > > > > > > >>>>>
> > > > > > > > >>>>> On Fri, Jan 20, 2023 at 3:59 PM Artem Livshits
> > > > > > > > >>>>> <al...@confluent.io.invalid> wrote:
> > > > > > > > >>>>>
> > > > > > > > >>>>>>>     besides the poorly written client case
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> A poorly written client could create a lot of grief to
> > > > people
> > > > > > who
> > > > > > > run
> > > > > > > > >>>> Kafka
> > > > > > > > >>>>>> brokers :-), so when deciding to make an error fatal I
> > > > would see
> > > > > > > if
> > > > > > > > >>>> there
> > > > > > > > >>>>>> is a reasonable recovery path rather than how often it
> > > could
> > > > > > > happen.
> > > > > > > > >>>> If we
> > > > > > > > >>>>>> have solid implementation of transactions (which I
> hope
> > > > we'll do
> > > > > > > as a
> > > > > > > > >>>>>> result of this KIP), it would help to recover from a
> > large
> > > > class
> > > > > > > of
> > > > > > > > >>>> errors
> > > > > > > > >>>>>> by just aborting a transaction, even if the cause of
> > error
> > > > is a
> > > > > > > race
> > > > > > > > >>>>>> condition or etc.
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> -Artem
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> On Fri, Jan 20, 2023 at 3:26 PM Justine Olshan
> > > > > > > > >>>>>> <jo...@confluent.io.invalid>
> > > > > > > > >>>>>> wrote:
> > > > > > > > >>>>>>
> > > > > > > > >>>>>>> Artem --
> > > > > > > > >>>>>>> I guess the discussion path we were going down is
> when
> > we
> > > > > > expect
> > > > > > > to
> > > > > > > > >> see
> > > > > > > > >>>>>>> this error. I mentioned that it was hard to come up
> > with
> > > > cases
> > > > > > > for
> > > > > > > > >> when
> > > > > > > > >>>>>> the
> > > > > > > > >>>>>>> producer would still be around to receive the error
> > > > besides the
> > > > > > > > >> poorly
> > > > > > > > >>>>>>> written client case.
> > > > > > > > >>>>>>> If we don't expect to have a producer to receive the
> > > > response,
> > > > > > it
> > > > > > > > >> sort
> > > > > > > > >>>> of
> > > > > > > > >>>>>>> makes sense for it to be fatal.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> I had some discussion with Jason offline about the
> > epoch
> > > > being
> > > > > > > off
> > > > > > > > >>>> cases
> > > > > > > > >>>>>>> and I'm not sure we could find a ton (outside of
> > produce
> > > > > > > requests)
> > > > > > > > >>>> where
> > > > > > > > >>>>>> we
> > > > > > > > >>>>>>> could/should recover. I'd be happy to hear some
> > examples
> > > > > > though,
> > > > > > > > >> maybe
> > > > > > > > >>>>>> I'm
> > > > > > > > >>>>>>> missing something.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> Thanks,
> > > > > > > > >>>>>>> Justine
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> On Fri, Jan 20, 2023 at 3:19 PM Artem Livshits
> > > > > > > > >>>>>>> <al...@confluent.io.invalid> wrote:
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>>> In general, I'd like to avoid fatal errors as much
> as
> > > > > > possible,
> > > > > > > in
> > > > > > > > >>>> some
> > > > > > > > >>>>>>>> sense fatal errors just push out recovery logic to
> the
> > > > > > > application
> > > > > > > > >>>>>> which
> > > > > > > > >>>>>>>> either complicates the application or leads to
> > > disruption
> > > > > > (we've
> > > > > > > > >> seen
> > > > > > > > >>>>>>> cases
> > > > > > > > >>>>>>>> when a transient broker error could lead to work
> > > stoppage
> > > > when
> > > > > > > > >>>>>>> applications
> > > > > > > > >>>>>>>> need to be manually restarted).  I think we should
> > > strive
> > > > to
> > > > > > > define
> > > > > > > > >>>>>>>> recovery logic for most errors (and/or encapsulate
> it
> > in
> > > > the
> > > > > > > Kafka
> > > > > > > > >>>>>> client
> > > > > > > > >>>>>>>> as much as possible).
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> One benefit of transactions is that they simplify
> > > recovery
> > > > > > from
> > > > > > > > >>>> errors,
> > > > > > > > >>>>>>>> pretty much any error (that's not handled
> > transparently
> > > by
> > > > > > > retries
> > > > > > > > >> in
> > > > > > > > >>>>>>> Kafka
> > > > > > > > >>>>>>>> client) can be handled by the application via
> aborting
> > > the
> > > > > > > > >> transaction
> > > > > > > > >>>>>>> and
> > > > > > > > >>>>>>>> repeating the transactional logic again.  One tricky
> > > > error is
> > > > > > an
> > > > > > > > >> error
> > > > > > > > >>>>>>>> during commit, because we don't know the outcome.
> For
> > > > commit
> > > > > > > > >> errors,
> > > > > > > > >>>>>> the
> > > > > > > > >>>>>>>> recommendation should be to retry the commit until
> it
> > > > returns
> > > > > > > the
> > > > > > > > >>>>>>> specific
> > > > > > > > >>>>>>>> result (committed or aborted).
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> -Artem
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> On Fri, Jan 20, 2023 at 2:52 PM Justine Olshan
> > > > > > > > >>>>>>>> <jo...@confluent.io.invalid>
> > > > > > > > >>>>>>>> wrote:
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>>> That's a fair point about other clients.
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> I think the abortable error case is interesting
> > because
> > > > I'm
> > > > > > > curious
> > > > > > > > >>>>>> how
> > > > > > > > >>>>>>>>> other clients would handle this. I assume they
> would
> > > > need to
> > > > > > > > >>>>>> implement
> > > > > > > > >>>>>>>>> handling for the error code unless they did
> something
> > > > like
> > > > > > "any
> > > > > > > > >>>>>> unknown
> > > > > > > > >>>>>>>>> error codes/any codes that aren't x,y,z are
> > > retriable." I
> > > > > > would
> > > > > > > > >> hope
> > > > > > > > >>>>>>> that
> > > > > > > > >>>>>>>>> unknown error codes were fatal, and if the code was
> > > > > > > implemented it
> > > > > > > > >>>>>>> would
> > > > > > > > >>>>>>>>> abort the transaction. But I will think on this
> too.
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> As for InvalidRecord -- you mentioned it was not
> > fatal,
> > > > but
> > > > > > I'm
> > > > > > > > >>>>>> taking
> > > > > > > > >>>>>>> a
> > > > > > > > >>>>>>>>> look through the code. We would see this on
> handling
> > > the
> > > > > > > produce
> > > > > > > > >>>>>>>> response.
> > > > > > > > >>>>>>>>> If I recall correctly, we check if errors are
> > > retriable.
> > > > I
> > > > > > > think
> > > > > > > > >> this
> > > > > > > > >>>>>>>> error
> > > > > > > > >>>>>>>>> would not be retriable. But I guess the concern
> here
> > is
> > > > that
> > > > > > > it is
> > > > > > > > >>>>>> not
> > > > > > > > >>>>>>>>> enough for just that batch to fail. I guess I
> hadn't
> > > > > > considered
> > > > > > > > >> fully
> > > > > > > > >>>>>>>>> fencing the old producer but there are valid
> > arguments
> > > > here
> > > > > > > why we
> > > > > > > > >>>>>>> would
> > > > > > > > >>>>>>>>> want to.
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> Thanks,
> > > > > > > > >>>>>>>>> Justine
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> On Fri, Jan 20, 2023 at 2:35 PM Guozhang Wang <
> > > > > > > > >>>>>>>> guozhang.wang.us@gmail.com>
> > > > > > > > >>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>>> Thanks Justine for the replies! I agree with most
> of
> > > > your
> > > > > > > > >> thoughts.
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>>> Just for 3/7), though I agree for our own AK
> > producer,
> > > > since
> > > > > > > we do
> > > > > > > > >>>>>>>>>> "nextRequest(boolean hasIncompleteBatches)", we
> > > > guarantee
> > > > > > the
> > > > > > > > >>>>>> end-txn
> > > > > > > > >>>>>>>>>> would not be sent until we've effectively flushed,
> > but
> > > > I was
> > > > > > > > >>>>>>> referring
> > > > > > > > >>>>>>>>>> to any future bugs or other buggy clients that the
> > > same
> > > > > > > client may
> > > > > > > > >>>>>>> get
> > > > > > > > >>>>>>>>>> into this situation, in which case we should give
> > the
> > > > > > client a
> > > > > > > > >>>>>> clear
> > > > > > > > >>>>>>>>>> msg that "you did something wrong, and hence now
> you
> > > > should
> > > > > > > > >> fatally
> > > > > > > > >>>>>>>>>> close yourself". What I'm concerned about is that,
> > by
> > > > seeing
> > > > > > > an
> > > > > > > > >>>>>>>>>> "abortable error" or in some rare cases an
> "invalid
> > > > record",
> > > > > > > the
> > > > > > > > >>>>>>>>>> client could not realize "something that's really
> > bad
> > > > > > > happened".
> > > > > > > > >> So
> > > > > > > > >>>>>>>>>> it's not about adding a new error, it's mainly
> about
> > > > those
> > > > > > > real
> > > > > > > > >>>>>> buggy
> > > > > > > > >>>>>>>>>> situations causing such "should never happen"
> cases,
> > > the
> > > > > > > errors
> > > > > > > > >>>>>>> return
> > > > > > > > >>>>>>>>>> would not be informative enough.
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>>> Thinking in other ways, if we believe that for
> most
> > > > cases
> > > > > > such
> > > > > > > > >>>>>> error
> > > > > > > > >>>>>>>>>> codes would not reach the original clients since
> > they
> > > > would
> > > > > > be
> > > > > > > > >>>>>>>>>> disconnected or even gone by that time, and only
> in
> > > some
> > > > > > rare
> > > > > > > > >> cases
> > > > > > > > >>>>>>>>>> they would still be seen by the sending clients,
> > then
> > > > why
> > > > > > not
> > > > > > > make
> > > > > > > > >>>>>>>>>> them more fatal and more specific than generic.
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>>> Guozhang
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>>> On Fri, Jan 20, 2023 at 1:59 PM Justine Olshan
> > > > > > > > >>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > > > > > > > >>>>>>>>>>>
> > > > > > > > >>>>>>>>>>> Hey Guozhang. Thanks for taking a look and for
> the
> > > > detailed
> > > > > > > > >>>>>>> comments!
> > > > > > > > >>>>>>>>>> I'll
> > > > > > > > >>>>>>>>>>> do my best to address below.
> > > > > > > > >>>>>>>>>>>
> > > > > > > > >>>>>>>>>>> 1. I see what you are saying here, but I think I
> > need
> > > > to
> > > > > > look
> > > > > > > > >>>>>>> through
> > > > > > > > >>>>>>>>> the
> > > > > > > > >>>>>>>>>>> sequence of events you mention. Typically we've
> > seen
> > > > this
> > > > > > > issue
> > > > > > > > >>>>>> in
> > > > > > > > >>>>>>> a
> > > > > > > > >>>>>>>>> few
> > > > > > > > >>>>>>>>>>> cases.
> > > > > > > > >>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>     One is when we have a producer disconnect
> when
> > > > trying
> > > > > > to
> > > > > > > > >>>>>> produce.
> > > > > > > > >>>>>>>>>>> Typically in these cases, we abort the
> transaction.
> > > > We've
> > > > > > > seen
> > > > > > > > >>>>>> that
> > > > > > > > >>>>>>>>> after
> > > > > > > > >>>>>>>>>>> the markers are written, the disconnection can
> > > > sometimes
> > > > > > > cause
> > > > > > > > >>>>>> the
> > > > > > > > >>>>>>>>>> request
> > > > > > > > >>>>>>>>>>> to get flushed to the broker. In this case, we
> > don't
> > > > need
> > > > > > > client
> > > > > > > > >>>>>>>>> handling
> > > > > > > > >>>>>>>>>>> because the producer we are responding to is
> gone.
> > We
> > > > just
> > > > > > > needed
> > > > > > > > >>>>>>> to
> > > > > > > > >>>>>>>>> make
> > > > > > > > >>>>>>>>>>> sure we didn't write to the log on the broker
> side.
> > > I'm
> > > > > > > trying to
> > > > > > > > >>>>>>>> think
> > > > > > > > >>>>>>>>>> of
> > > > > > > > >>>>>>>>>>> a case where we do have the client to return to.
> > I'd
> > > > think
> > > > > > > the
> > > > > > > > >>>>>> same
> > > > > > > > >>>>>>>>>> client
> > > > > > > > >>>>>>>>>>> couldn't progress to committing the transaction
> > > unless
> > > > the
> > > > > > > > >>>>>> produce
> > > > > > > > >>>>>>>>>> request
> > > > > > > > >>>>>>>>>>> returned right? Of course, there is the
> incorrectly
> > > > written
> > > > > > > > >>>>>> clients
> > > > > > > > >>>>>>>>> case.
> > > > > > > > >>>>>>>>>>> I'll think on this a bit more and let you know
> if I
> > > > come up
> > > > > > > with
> > > > > > > > >>>>>>>>> another
> > > > > > > > >>>>>>>>>>> scenario when we would return to an active client
> > > when
> > > > the
> > > > > > > > >>>>>>>> transaction
> > > > > > > > >>>>>>>>> is
> > > > > > > > >>>>>>>>>>> no longer ongoing.
> > > > > > > > >>>>>>>>>>>
> > > > > > > > >>>>>>>>>>> I was not aware that we checked the result of a
> > send
> > > > after
> > > > > > we
> > > > > > > > >>>>>>> commit
> > > > > > > > >>>>>>>>>>> though. I'll need to look into that a bit more.
> > > > > > > > >>>>>>>>>>>
> > > > > > > > >>>>>>>>>>> 2. There were some questions about this in the
> > > > discussion.
> > > > > > > The
> > > > > > > > >>>>>> plan
> > > > > > > > >>>>>>>> is
> > > > > > > > >>>>>>>>> to
> > > > > > > > >>>>>>>>>>> handle overflow with the mechanism we currently
> > have
> > > > in the
> > > > > > > > >>>>>>> producer.
> > > > > > > > >>>>>>>>> If
> > > > > > > > >>>>>>>>>> we
> > > > > > > > >>>>>>>>>>> try to bump and the epoch will overflow, we
> > actually
> > > > > > > allocate a
> > > > > > > > >>>>>> new
> > > > > > > > >>>>>>>>>>> producer ID. I need to confirm the fencing logic
> on
> > > the
> > > > > > last
> > > > > > > > >>>>>> epoch
> > > > > > > > >>>>>>>> (ie,
> > > > > > > > >>>>>>>>>> we
> > > > > > > > >>>>>>>>>>> probably shouldn't allow any records to be
> produced
> > > > with
> > > > > > the
> > > > > > > > >>>>>> final
> > > > > > > > >>>>>>>>> epoch
> > > > > > > > >>>>>>>>>>> since we can never properly fence that one).
> > > > > > > > >>>>>>>>>>>
> > > > > > > > >>>>>>>>>>> 3. I can agree with you that the current error
> > > > handling is
> > > > > > > > >>>>>> messy. I
> > > > > > > > >>>>>>>>>> recall
> > > > > > > > >>>>>>>>>>> taking a look at your KIP a while back, but I
> > think I
> > > > > > mostly
> > > > > > > saw
> > > > > > > > >>>>>>> the
> > > > > > > > >>>>>>>>>>> section about how the errors were wrapped. Maybe
> I
> > > > need to
> > > > > > > take
> > > > > > > > >>>>>>>> another
> > > > > > > > >>>>>>>>>>> look. As for abortable error, the idea was that
> the
> > > > > > handling
> > > > > > > > >>>>>> would
> > > > > > > > >>>>>>> be
> > > > > > > > >>>>>>>>>>> simple -- if this error is seen, the transaction
> > > > should be
> > > > > > > > >>>>>> aborted
> > > > > > > > >>>>>>> --
> > > > > > > > >>>>>>>>> no
> > > > > > > > >>>>>>>>>>> other logic about previous state or requests
> > > > necessary. Is
> > > > > > > your
> > > > > > > > >>>>>>>> concern
> > > > > > > > >>>>>>>>>>> simply about adding new errors? We were hoping to
> > > have
> > > > an
> > > > > > > error
> > > > > > > > >>>>>>> that
> > > > > > > > >>>>>>>>>> would
> > > > > > > > >>>>>>>>>>> have one meaning and many of the current errors
> > have
> > > a
> > > > > > > history of
> > > > > > > > >>>>>>>>> meaning
> > > > > > > > >>>>>>>>>>> different things on different client versions.
> That
> > > > was the
> > > > > > > main
> > > > > > > > >>>>>>>>>> motivation
> > > > > > > > >>>>>>>>>>> for adding a new error.
> > > > > > > > >>>>>>>>>>>
> > > > > > > > >>>>>>>>>>> 4. This is a good point about record timestamp
> > > > reordering.
> > > > > > > > >>>>>>> Timestamps
> > > > > > > > >>>>>>>>>> don't
> > > > > > > > >>>>>>>>>>> affect compaction, but they do affect retention
> > > > deletion.
> > > > > > For
> > > > > > > > >>>>>> that,
> > > > > > > > >>>>>>>>> kafka
> > > > > > > > >>>>>>>>>>> considers the largest timestamp in the segment,
> so
> > I
> > > > think
> > > > > > a
> > > > > > > > >>>>>> small
> > > > > > > > >>>>>>>>> amount
> > > > > > > > >>>>>>>>>>> of reordering (hopefully on the order of
> > milliseconds
> > > > or
> > > > > > even
> > > > > > > > >>>>>>>> seconds)
> > > > > > > > >>>>>>>>>> will
> > > > > > > > >>>>>>>>>>> be ok. We take timestamps from clients so there
> is
> > > > already
> > > > > > a
> > > > > > > > >>>>>>>>> possibility
> > > > > > > > >>>>>>>>>>> for some drift and non-monotonically increasing
> > > > timestamps.
> > > > > > > > >>>>>>>>>>>
> > > > > > > > >>>>>>>>>>> 5. Thanks for catching. The error is there, but
> > it's
> > > > > > actually
> > > > > > > > >>>>>> that
> > > > > > > > >>>>>>>>> those
> > > > > > > > >>>>>>>>>>> fields should be 4+! Due to how the message
> > generator
> > > > > > works,
> > > > > > > I
> > > > > > > > >>>>>>>> actually
> > > > > > > > >>>>>>>>>>> have to redefine those fields inside the
> > > > > > > > >>>>>>>>> `"AddPartitionsToTxnTransaction`
> > > > > > > > >>>>>>>>>>> block for it to build correctly. I'll fix it to
> be
> > > > correct.
> > > > > > > > >>>>>>>>>>>
> > > > > > > > >>>>>>>>>>> 6. Correct -- we will only add the request to
> > > > purgatory if
> > > > > > > the
> > > > > > > > >>>>>>> cache
> > > > > > > > >>>>>>>>> has
> > > > > > > > >>>>>>>>>> no
> > > > > > > > >>>>>>>>>>> ongoing transaction. I can change the wording to
> > make
> > > > that
> > > > > > > > >>>>>> clearer
> > > > > > > > >>>>>>>> that
> > > > > > > > >>>>>>>>>> we
> > > > > > > > >>>>>>>>>>> only place the request in purgatory if we need to
> > > > contact
> > > > > > the
> > > > > > > > >>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>> coordinator.
> > > > > > > > >>>>>>>>>>>
> > > > > > > > >>>>>>>>>>> 7. We did take a look at some of the errors and
> it
> > > was
> > > > hard
> > > > > > > to
> > > > > > > > >>>>>> come
> > > > > > > > >>>>>>>> up
> > > > > > > > >>>>>>>>>> with
> > > > > > > > >>>>>>>>>>> a good one. I agree that InvalidTxnStateException
> > is
> > > > ideal
> > > > > > > except
> > > > > > > > >>>>>>> for
> > > > > > > > >>>>>>>>> the
> > > > > > > > >>>>>>>>>>> fact that it hasn't been returned on Produce
> > requests
> > > > > > > before. The
> > > > > > > > >>>>>>>> error
> > > > > > > > >>>>>>>>>>> handling for clients is a bit vague (which is
> why I
> > > > opened
> > > > > > > > >>>>>>>> KAFKA-14439
> > > > > > > > >>>>>>>>>>> <
> https://issues.apache.org/jira/browse/KAFKA-14439
> > > >),
> > > > but
> > > > > > > the
> > > > > > > > >>>>>>>> decision
> > > > > > > > >>>>>>>>>> we
> > > > > > > > >>>>>>>>>>> made here was to only return errors that have
> been
> > > > > > previously
> > > > > > > > >>>>>>>> returned
> > > > > > > > >>>>>>>>> to
> > > > > > > > >>>>>>>>>>> producers. As for not being fatal, I think part
> of
> > > the
> > > > > > > theory was
> > > > > > > > >>>>>>>> that
> > > > > > > > >>>>>>>>> in
> > > > > > > > >>>>>>>>>>> many cases, the producer would be disconnected.
> > (See
> > > > point
> > > > > > > 1) and
> > > > > > > > >>>>>>>> this
> > > > > > > > >>>>>>>>>>> would just be an error to return from the
> server. I
> > > did
> > > > > > plan
> > > > > > > to
> > > > > > > > >>>>>>> think
> > > > > > > > >>>>>>>>>> about
> > > > > > > > >>>>>>>>>>> other cases, so let me know if you think of any
> as
> > > > well!
> > > > > > > > >>>>>>>>>>>
> > > > > > > > >>>>>>>>>>> Lots to say! Let me know if you have further
> > > thoughts!
> > > > > > > > >>>>>>>>>>> Justine
> > > > > > > > >>>>>>>>>>>
> > > > > > > > >>>>>>>>>>> On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang <
> > > > > > > > >>>>>>>>>> guozhang.wang.us@gmail.com>
> > > > > > > > >>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> Hello Justine,
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> Thanks for the great write-up! I made a quick
> pass
> > > > through
> > > > > > > it
> > > > > > > > >>>>>> and
> > > > > > > > >>>>>>>>> here
> > > > > > > > >>>>>>>>>>>> are some thoughts (I have not been able to read
> > > > through
> > > > > > this
> > > > > > > > >>>>>>> thread
> > > > > > > > >>>>>>>>> so
> > > > > > > > >>>>>>>>>>>> pardon me if they have overlapped or subsumed by
> > > > previous
> > > > > > > > >>>>>>>> comments):
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> First are some meta ones:
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> 1. I think we need to also improve the client's
> > > > experience
> > > > > > > once
> > > > > > > > >>>>>>> we
> > > > > > > > >>>>>>>>>>>> have this defence in place. More concretely,
> say a
> > > > user's
> > > > > > > > >>>>>>> producer
> > > > > > > > >>>>>>>>>>>> code is like following:
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> future = producer.send();
> > > > > > > > >>>>>>>>>>>> // producer.flush();
> > > > > > > > >>>>>>>>>>>> producer.commitTransaction();
> > > > > > > > >>>>>>>>>>>> future.get();
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> Which resulted in the order of a)
> produce-request
> > > > sent by
> > > > > > > > >>>>>>> producer,
> > > > > > > > >>>>>>>>> b)
> > > > > > > > >>>>>>>>>>>> end-txn-request sent by producer, c)
> > > end-txn-response
> > > > sent
> > > > > > > > >>>>>> back,
> > > > > > > > >>>>>>> d)
> > > > > > > > >>>>>>>>>>>> txn-marker-request sent from coordinator to
> > > partition
> > > > > > > leader,
> > > > > > > > >>>>>> e)
> > > > > > > > >>>>>>>>>>>> produce-request finally received by the
> partition
> > > > leader,
> > > > > > > > >>>>>> before
> > > > > > > > >>>>>>>> this
> > > > > > > > >>>>>>>>>>>> KIP e) step would be accepted causing a dangling
> > > txn;
> > > > now
> > > > > > it
> > > > > > > > >>>>>>> would
> > > > > > > > >>>>>>>> be
> > > > > > > > >>>>>>>>>>>> rejected in step e) which is good. But from the
> > > > client's
> > > > > > > point
> > > > > > > > >>>>>> of
> > > > > > > > >>>>>>>>> view
> > > > > > > > >>>>>>>>>>>> now it becomes confusing since the
> > > > `commitTransaction()`
> > > > > > > > >>>>>> returns
> > > > > > > > >>>>>>>>>>>> successfully, but the "future" throws an
> > > invalid-epoch
> > > > > > > error,
> > > > > > > > >>>>>> and
> > > > > > > > >>>>>>>>> they
> > > > > > > > >>>>>>>>>>>> are not sure if the transaction did succeed or
> > not.
> > > In
> > > > > > > fact, it
> > > > > > > > >>>>>>>>>>>> "partially succeeded" with some msgs being
> > rejected
> > > > but
> > > > > > > others
> > > > > > > > >>>>>>>>>>>> committed successfully.
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> Of course the easy way to avoid this is, always
> > call
> > > > > > > > >>>>>>>>>>>> "producer.flush()" before commitTxn and that's
> > what
> > > > we do
> > > > > > > > >>>>>>>> ourselves,
> > > > > > > > >>>>>>>>>>>> and what we recommend users do. But I suspect
> not
> > > > everyone
> > > > > > > does
> > > > > > > > >>>>>>> it.
> > > > > > > > >>>>>>>>> In
> > > > > > > > >>>>>>>>>>>> fact I just checked the javadoc in KafkaProducer
> > and
> > > > our
> > > > > > > code
> > > > > > > > >>>>>>>> snippet
> > > > > > > > >>>>>>>>>>>> does not include a `flush()` call. So I'm
> thinking
> > > > maybe
> > > > > > we
> > > > > > > can
> > > > > > > > >>>>>>> in
> > > > > > > > >>>>>>>>>>>> side the `commitTxn` code to enforce flushing
> > before
> > > > > > sending
> > > > > > > > >>>>>> the
> > > > > > > > >>>>>>>>>>>> end-txn request.
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> 2. I'd like to clarify a bit details on "just
> add
> > > > > > > partitions to
> > > > > > > > >>>>>>> the
> > > > > > > > >>>>>>>>>>>> transaction on the first produce request during
> a
> > > > > > > transaction".
> > > > > > > > >>>>>>> My
> > > > > > > > >>>>>>>>>>>> understanding is that the partition leader's
> cache
> > > > has the
> > > > > > > > >>>>>>> producer
> > > > > > > > >>>>>>>>> id
> > > > > > > > >>>>>>>>>>>> / sequence / epoch for the latest txn, either
> > > > on-going or
> > > > > > is
> > > > > > > > >>>>>>>>> completed
> > > > > > > > >>>>>>>>>>>> (upon receiving the marker request from
> > > coordinator).
> > > > > > When a
> > > > > > > > >>>>>>>> produce
> > > > > > > > >>>>>>>>>>>> request is received, if
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> * producer's epoch < cached epoch, or producer's
> > > > epoch ==
> > > > > > > > >>>>>> cached
> > > > > > > > >>>>>>>>> epoch
> > > > > > > > >>>>>>>>>>>> but the latest txn is completed, leader directly
> > > > reject
> > > > > > with
> > > > > > > > >>>>>>>>>>>> invalid-epoch.
> > > > > > > > >>>>>>>>>>>> * producer's epoch > cached epoch, park the the
> > > > request
> > > > > > and
> > > > > > > > >>>>>> send
> > > > > > > > >>>>>>>>>>>> add-partitions request to coordinator.
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> In order to do it, does the coordinator need to
> > bump
> > > > the
> > > > > > > > >>>>>> sequence
> > > > > > > > >>>>>>>> and
> > > > > > > > >>>>>>>>>>>> reset epoch to 0 when the next epoch is going to
> > > > overflow?
> > > > > > > If
> > > > > > > > >>>>>> no
> > > > > > > > >>>>>>>> need
> > > > > > > > >>>>>>>>>>>> to do so, then how we handle the (admittedly
> rare,
> > > but
> > > > > > still
> > > > > > > > >>>>>> may
> > > > > > > > >>>>>>>>>>>> happen) epoch overflow situation?
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> 3. I'm a bit concerned about adding a generic
> > > > > > > "ABORTABLE_ERROR"
> > > > > > > > >>>>>>>> given
> > > > > > > > >>>>>>>>>>>> we already have a pretty messy error
> > classification
> > > > and
> > > > > > > error
> > > > > > > > >>>>>>>>> handling
> > > > > > > > >>>>>>>>>>>> on the producer clients side --- I have a
> summary
> > > > about
> > > > > > the
> > > > > > > > >>>>>>> issues
> > > > > > > > >>>>>>>>> and
> > > > > > > > >>>>>>>>>>>> a proposal to address this in
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>
> > > > > > > > >>>>
> > > > > > > > >>
> > > > > > >
> > > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
> > > > > > > > >>>>>>>>>>>> -- I understand we do not want to use
> > > > > > "UNKNOWN_PRODUCER_ID"
> > > > > > > > >>>>>>> anymore
> > > > > > > > >>>>>>>>>>>> and in fact we intend to deprecate it in KIP-360
> > and
> > > > > > > eventually
> > > > > > > > >>>>>>>>> remove
> > > > > > > > >>>>>>>>>>>> it; but I'm wondering can we still use specific
> > > error
> > > > > > codes.
> > > > > > > > >>>>>> E.g.
> > > > > > > > >>>>>>>>> what
> > > > > > > > >>>>>>>>>>>> about "InvalidProducerEpochException" since for
> > new
> > > > > > clients,
> > > > > > > > >>>>>> the
> > > > > > > > >>>>>>>>>>>> actual reason this would actually be rejected is
> > > > indeed
> > > > > > > because
> > > > > > > > >>>>>>> the
> > > > > > > > >>>>>>>>>>>> epoch on the coordinator caused the
> > > > add-partitions-request
> > > > > > > from
> > > > > > > > >>>>>>> the
> > > > > > > > >>>>>>>>>>>> brokers to be rejected anyways?
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> 4. It seems we put the producer request into
> > > purgatory
> > > > > > > before
> > > > > > > > >>>>>> we
> > > > > > > > >>>>>>>> ever
> > > > > > > > >>>>>>>>>>>> append the records, while other producer's
> records
> > > may
> > > > > > > still be
> > > > > > > > >>>>>>>>>>>> appended during the time; and that potentially
> may
> > > > result
> > > > > > in
> > > > > > > > >>>>>> some
> > > > > > > > >>>>>>>>>>>> re-ordering compared with reception order. I'm
> not
> > > > super
> > > > > > > > >>>>>>> concerned
> > > > > > > > >>>>>>>>>>>> about it since Kafka does not guarantee
> reception
> > > > ordering
> > > > > > > > >>>>>> across
> > > > > > > > >>>>>>>>>>>> producers anyways, but it may make the
> timestamps
> > of
> > > > > > records
> > > > > > > > >>>>>>>> inside a
> > > > > > > > >>>>>>>>>>>> partition to be more out-of-ordered. Are we
> aware
> > of
> > > > any
> > > > > > > > >>>>>>> scenarios
> > > > > > > > >>>>>>>>>>>> such as future enhancements on log compactions
> > that
> > > > may be
> > > > > > > > >>>>>>> affected
> > > > > > > > >>>>>>>>> by
> > > > > > > > >>>>>>>>>>>> this effect?
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> Below are just minor comments:
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> 5. In "AddPartitionsToTxnTransaction" field of
> > > > > > > > >>>>>>>>>>>> "AddPartitionsToTxnRequest" RPC, the versions of
> > > those
> > > > > > inner
> > > > > > > > >>>>>>> fields
> > > > > > > > >>>>>>>>>>>> are "0-3" while I thought they should be "0+"
> > still?
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> 6. Regarding "we can place the request in a
> > > purgatory
> > > > of
> > > > > > > sorts
> > > > > > > > >>>>>>> and
> > > > > > > > >>>>>>>>>>>> check if there is any state for the transaction
> on
> > > the
> > > > > > > > >>>>>> broker": i
> > > > > > > > >>>>>>>>>>>> think at this time when we just do the checks
> > > against
> > > > the
> > > > > > > > >>>>>> cached
> > > > > > > > >>>>>>>>>>>> state, we do not need to put the request to
> > > purgatory
> > > > yet?
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> 7. This is related to 3) above. I feel using
> > > > > > > > >>>>>>>> "InvalidRecordException"
> > > > > > > > >>>>>>>>>>>> for older clients may also be a bit confusing,
> and
> > > > also it
> > > > > > > is
> > > > > > > > >>>>>> not
> > > > > > > > >>>>>>>>>>>> fatal -- for old clients, it better to be fatal
> > > since
> > > > this
> > > > > > > > >>>>>>>> indicates
> > > > > > > > >>>>>>>>>>>> the clients is doing something wrong and hence
> it
> > > > should
> > > > > > be
> > > > > > > > >>>>>>> closed.
> > > > > > > > >>>>>>>>>>>> And in general I'd prefer to use slightly more
> > > > specific
> > > > > > > meaning
> > > > > > > > >>>>>>>> error
> > > > > > > > >>>>>>>>>>>> codes for clients. That being said, I also feel
> > > > > > > > >>>>>>>>>>>> "InvalidProducerEpochException" is not suitable
> > for
> > > > old
> > > > > > > > >>>>>> versioned
> > > > > > > > >>>>>>>>>>>> clients, and we'd have to pick one that old
> > clients
> > > > > > > recognize.
> > > > > > > > >>>>>>> I'd
> > > > > > > > >>>>>>>>>>>> prefer "InvalidTxnStateException" but that one
> is
> > > > supposed
> > > > > > > to
> > > > > > > > >>>>>> be
> > > > > > > > >>>>>>>>>>>> returned from txn coordinators only today. I'd
> > > > suggest we
> > > > > > > do a
> > > > > > > > >>>>>>>> quick
> > > > > > > > >>>>>>>>>>>> check in the current client's code path and see
> if
> > > > that
> > > > > > one
> > > > > > > > >>>>>> would
> > > > > > > > >>>>>>>> be
> > > > > > > > >>>>>>>>>>>> handled if it's from a produce-response, and if
> > yes,
> > > > use
> > > > > > > this
> > > > > > > > >>>>>>> one;
> > > > > > > > >>>>>>>>>>>> otherwise, use "ProducerFencedException" which
> is
> > > much
> > > > > > less
> > > > > > > > >>>>>>>>> meaningful
> > > > > > > > >>>>>>>>>>>> but it's still a fatal error.
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> Thanks,
> > > > > > > > >>>>>>>>>>>> Guozhang
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
> > > > > > > > >>>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > > > > > > > >>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>> Yeah -- looks like we already have code to
> handle
> > > > bumping
> > > > > > > the
> > > > > > > > >>>>>>>> epoch
> > > > > > > > >>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>> when the epoch is Short.MAX_VALUE, we get a new
> > > > producer
> > > > > > > ID.
> > > > > > > > >>>>>>>> Since
> > > > > > > > >>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>> already the behavior, do we want to change it
> > > > further?
> > > > > > > > >>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>> Justine
> > > > > > > > >>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>> On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan
> <
> > > > > > > > >>>>>>>>> jolshan@confluent.io
> > > > > > > > >>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>> Hey all, just wanted to quickly update and say
> > > I've
> > > > > > > > >>>>>> modified
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>> KIP to
> > > > > > > > >>>>>>>>>>>>>> explicitly mention that
> > > AddOffsetCommitsToTxnRequest
> > > > > > will
> > > > > > > > >>>>>> be
> > > > > > > > >>>>>>>>>> replaced
> > > > > > > > >>>>>>>>>>>> by
> > > > > > > > >>>>>>>>>>>>>> a coordinator-side (inter-broker)
> > > AddPartitionsToTxn
> > > > > > > > >>>>>> implicit
> > > > > > > > >>>>>>>>>> request.
> > > > > > > > >>>>>>>>>>>> This
> > > > > > > > >>>>>>>>>>>>>> mirrors the user partitions and will
> implicitly
> > > add
> > > > > > offset
> > > > > > > > >>>>>>>>>> partitions
> > > > > > > > >>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>> transactions when we commit offsets on them.
> We
> > > will
> > > > > > > > >>>>>>> deprecate
> > > > > > > > >>>>>>>>>>>> AddOffsetCommitsToTxnRequest
> > > > > > > > >>>>>>>>>>>>>> for new clients.
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>> Also to address Artem's comments --
> > > > > > > > >>>>>>>>>>>>>> I'm a bit unsure if the changes here will
> change
> > > the
> > > > > > > > >>>>>> previous
> > > > > > > > >>>>>>>>>> behavior
> > > > > > > > >>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>> fencing producers. In the case you mention in
> > the
> > > > first
> > > > > > > > >>>>>>>>> paragraph,
> > > > > > > > >>>>>>>>>> are
> > > > > > > > >>>>>>>>>>>> you
> > > > > > > > >>>>>>>>>>>>>> saying we bump the epoch before we try to
> abort
> > > the
> > > > > > > > >>>>>>>> transaction?
> > > > > > > > >>>>>>>>> I
> > > > > > > > >>>>>>>>>>>> think I
> > > > > > > > >>>>>>>>>>>>>> need to understand the scenarios you mention a
> > bit
> > > > > > better.
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>> As for the second part -- I think it makes
> sense
> > > to
> > > > have
> > > > > > > > >>>>>> some
> > > > > > > > >>>>>>>>> sort
> > > > > > > > >>>>>>>>>> of
> > > > > > > > >>>>>>>>>>>>>> "sentinel" epoch to signal epoch is about to
> > > > overflow (I
> > > > > > > > >>>>>>> think
> > > > > > > > >>>>>>>> we
> > > > > > > > >>>>>>>>>> sort
> > > > > > > > >>>>>>>>>>>> of
> > > > > > > > >>>>>>>>>>>>>> have this value in place in some ways) so we
> can
> > > > codify
> > > > > > it
> > > > > > > > >>>>>> in
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>> KIP.
> > > > > > > > >>>>>>>>>>>> I'll
> > > > > > > > >>>>>>>>>>>>>> look into that and try to update soon.
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>> Thanks,
> > > > > > > > >>>>>>>>>>>>>> Justine.
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>> On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
> > > > > > > > >>>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> It's good to know that KIP-588 addressed some
> > of
> > > > the
> > > > > > > > >>>>>> issues.
> > > > > > > > >>>>>>>>>> Looking
> > > > > > > > >>>>>>>>>>>> at
> > > > > > > > >>>>>>>>>>>>>>> the code, it still looks like there are some
> > > cases
> > > > that
> > > > > > > > >>>>>>> would
> > > > > > > > >>>>>>>>>> result
> > > > > > > > >>>>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>>> fatal error, e.g. PRODUCER_FENCED is issued
> by
> > > the
> > > > > > > > >>>>>>> transaction
> > > > > > > > >>>>>>>>>>>> coordinator
> > > > > > > > >>>>>>>>>>>>>>> if epoch doesn't match, and the client treats
> > it
> > > > as a
> > > > > > > > >>>>>> fatal
> > > > > > > > >>>>>>>>> error
> > > > > > > > >>>>>>>>>>>> (code in
> > > > > > > > >>>>>>>>>>>>>>> TransactionManager request handling).  If we
> > > > consider,
> > > > > > > for
> > > > > > > > >>>>>>>>>> example,
> > > > > > > > >>>>>>>>>>>>>>> committing a transaction that returns a
> > timeout,
> > > > but
> > > > > > > > >>>>>>> actually
> > > > > > > > >>>>>>>>>>>> succeeds,
> > > > > > > > >>>>>>>>>>>>>>> trying to abort it or re-commit may result in
> > > > > > > > >>>>>>> PRODUCER_FENCED
> > > > > > > > >>>>>>>>>> error
> > > > > > > > >>>>>>>>>>>>>>> (because of epoch bump).
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> For failed commits, specifically, we need to
> > know
> > > > the
> > > > > > > > >>>>>> actual
> > > > > > > > >>>>>>>>>> outcome,
> > > > > > > > >>>>>>>>>>>>>>> because if we return an error the application
> > may
> > > > think
> > > > > > > > >>>>>> that
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>> transaction is aborted and redo the work,
> > leading
> > > > to
> > > > > > > > >>>>>>>> duplicates.
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> Re: overflowing epoch.  We could either do it
> > on
> > > > the TC
> > > > > > > > >>>>>> and
> > > > > > > > >>>>>>>>> return
> > > > > > > > >>>>>>>>>>>> both
> > > > > > > > >>>>>>>>>>>>>>> producer id and epoch (e.g. change the
> > protocol),
> > > > or
> > > > > > > > >>>>>> signal
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>> client
> > > > > > > > >>>>>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>> it needs to get a new producer id.  Checking
> > for
> > > > max
> > > > > > > epoch
> > > > > > > > >>>>>>>> could
> > > > > > > > >>>>>>>>>> be a
> > > > > > > > >>>>>>>>>>>>>>> reasonable signal, the value to check should
> > > > probably
> > > > > > be
> > > > > > > > >>>>>>>> present
> > > > > > > > >>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>> KIP
> > > > > > > > >>>>>>>>>>>>>>> as this is effectively a part of the
> contract.
> > > > Also,
> > > > > > the
> > > > > > > > >>>>>> TC
> > > > > > > > >>>>>>>>>> should
> > > > > > > > >>>>>>>>>>>>>>> probably return an error if the client didn't
> > > > change
> > > > > > > > >>>>>>> producer
> > > > > > > > >>>>>>>> id
> > > > > > > > >>>>>>>>>> after
> > > > > > > > >>>>>>>>>>>>>>> hitting max epoch.
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> -Artem
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> On Thu, Jan 12, 2023 at 10:31 AM Justine
> Olshan
> > > > > > > > >>>>>>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>> Thanks for the discussion Artem.
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>> With respect to the handling of fenced
> > > producers,
> > > > we
> > > > > > > > >>>>>> have
> > > > > > > > >>>>>>>> some
> > > > > > > > >>>>>>>>>>>> behavior
> > > > > > > > >>>>>>>>>>>>>>>> already in place. As of KIP-588:
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>
> > > > > > > > >>>>
> > > > > > > > >>
> > > > > > >
> > > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> > > > > > > > >>>>>>>>>>>>>>>> ,
> > > > > > > > >>>>>>>>>>>>>>>> we handle timeouts more gracefully. The
> > producer
> > > > can
> > > > > > > > >>>>>>>> recover.
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>> Produce requests can also recover from epoch
> > > > fencing
> > > > > > by
> > > > > > > > >>>>>>>>>> aborting the
> > > > > > > > >>>>>>>>>>>>>>>> transaction and starting over.
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>> What other cases were you considering that
> > would
> > > > cause
> > > > > > > > >>>>>> us
> > > > > > > > >>>>>>> to
> > > > > > > > >>>>>>>>>> have a
> > > > > > > > >>>>>>>>>>>>>>> fenced
> > > > > > > > >>>>>>>>>>>>>>>> epoch but we'd want to recover?
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>> The first point about handling epoch
> overflows
> > > is
> > > > > > fair.
> > > > > > > > >>>>>> I
> > > > > > > > >>>>>>>>> think
> > > > > > > > >>>>>>>>>>>> there is
> > > > > > > > >>>>>>>>>>>>>>>> some logic we'd need to consider. (ie, if we
> > are
> > > > one
> > > > > > > > >>>>>> away
> > > > > > > > >>>>>>>> from
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>> max
> > > > > > > > >>>>>>>>>>>>>>>> epoch, we need to reset the producer ID.)
> I'm
> > > > still
> > > > > > > > >>>>>>>> wondering
> > > > > > > > >>>>>>>>> if
> > > > > > > > >>>>>>>>>>>> there
> > > > > > > > >>>>>>>>>>>>>>> is a
> > > > > > > > >>>>>>>>>>>>>>>> way to direct this from the response, or if
> > > > everything
> > > > > > > > >>>>>>>> should
> > > > > > > > >>>>>>>>> be
> > > > > > > > >>>>>>>>>>>> done on
> > > > > > > > >>>>>>>>>>>>>>>> the client side. Let me know if you have any
> > > > thoughts
> > > > > > > > >>>>>>> here.
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>> Thanks,
> > > > > > > > >>>>>>>>>>>>>>>> Justine
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>> On Tue, Jan 10, 2023 at 4:06 PM Artem
> Livshits
> > > > > > > > >>>>>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>> There are some workflows in the client that
> > are
> > > > > > > > >>>>>> implied
> > > > > > > > >>>>>>> by
> > > > > > > > >>>>>>>>>>>> protocol
> > > > > > > > >>>>>>>>>>>>>>>>> changes, e.g.:
> > > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>> - for new clients, epoch changes with every
> > > > > > > > >>>>>> transaction
> > > > > > > > >>>>>>>> and
> > > > > > > > >>>>>>>>>> can
> > > > > > > > >>>>>>>>>>>>>>> overflow,
> > > > > > > > >>>>>>>>>>>>>>>>> in old clients this condition was handled
> > > > > > > > >>>>>> transparently,
> > > > > > > > >>>>>>>>>> because
> > > > > > > > >>>>>>>>>>>> epoch
> > > > > > > > >>>>>>>>>>>>>>>> was
> > > > > > > > >>>>>>>>>>>>>>>>> bumped in InitProducerId and it would
> return
> > a
> > > > new
> > > > > > > > >>>>>>>> producer
> > > > > > > > >>>>>>>>>> id if
> > > > > > > > >>>>>>>>>>>>>>> epoch
> > > > > > > > >>>>>>>>>>>>>>>>> overflows, the new clients would need to
> > > > implement
> > > > > > > > >>>>>> some
> > > > > > > > >>>>>>>>>> workflow
> > > > > > > > >>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>> refresh
> > > > > > > > >>>>>>>>>>>>>>>>> producer id
> > > > > > > > >>>>>>>>>>>>>>>>> - how to handle fenced producers, for new
> > > clients
> > > > > > > > >>>>>> epoch
> > > > > > > > >>>>>>>>>> changes
> > > > > > > > >>>>>>>>>>>> with
> > > > > > > > >>>>>>>>>>>>>>>> every
> > > > > > > > >>>>>>>>>>>>>>>>> transaction, so in presence of failures
> > during
> > > > > > > > >>>>>> commits /
> > > > > > > > >>>>>>>>>> aborts,
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>> producer could get easily fenced, old
> clients
> > > > would
> > > > > > > > >>>>>>> pretty
> > > > > > > > >>>>>>>>>> much
> > > > > > > > >>>>>>>>>>>> would
> > > > > > > > >>>>>>>>>>>>>>> get
> > > > > > > > >>>>>>>>>>>>>>>>> fenced when a new incarnation of the
> producer
> > > was
> > > > > > > > >>>>>>>>> initialized
> > > > > > > > >>>>>>>>>> with
> > > > > > > > >>>>>>>>>>>>>>>>> InitProducerId so it's ok to treat as a
> fatal
> > > > error,
> > > > > > > > >>>>>> the
> > > > > > > > >>>>>>>> new
> > > > > > > > >>>>>>>>>>>> clients
> > > > > > > > >>>>>>>>>>>>>>>> would
> > > > > > > > >>>>>>>>>>>>>>>>> need to implement some workflow to handle
> > that
> > > > error,
> > > > > > > > >>>>>>>>>> otherwise
> > > > > > > > >>>>>>>>>>>> they
> > > > > > > > >>>>>>>>>>>>>>>> could
> > > > > > > > >>>>>>>>>>>>>>>>> get fenced by themselves
> > > > > > > > >>>>>>>>>>>>>>>>> - in particular (as a subset of the
> previous
> > > > issue),
> > > > > > > > >>>>>>> what
> > > > > > > > >>>>>>>>>> would
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>> client
> > > > > > > > >>>>>>>>>>>>>>>>> do if it got a timeout during commit?
> commit
> > > > > > could've
> > > > > > > > >>>>>>>>>> succeeded
> > > > > > > > >>>>>>>>>>>> or
> > > > > > > > >>>>>>>>>>>>>>>> failed
> > > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>> Not sure if this has to be defined in the
> KIP
> > > as
> > > > > > > > >>>>>>>>> implementing
> > > > > > > > >>>>>>>>>>>> those
> > > > > > > > >>>>>>>>>>>>>>>>> probably wouldn't require protocol changes,
> > but
> > > > we
> > > > > > > > >>>>>> have
> > > > > > > > >>>>>>>>>> multiple
> > > > > > > > >>>>>>>>>>>>>>>>> implementations of Kafka clients, so
> probably
> > > > would
> > > > > > be
> > > > > > > > >>>>>>>> good
> > > > > > > > >>>>>>>>> to
> > > > > > > > >>>>>>>>>>>> have
> > > > > > > > >>>>>>>>>>>>>>> some
> > > > > > > > >>>>>>>>>>>>>>>>> client implementation guidance.  Could also
> > be
> > > > done
> > > > > > > > >>>>>> as a
> > > > > > > > >>>>>>>>>> separate
> > > > > > > > >>>>>>>>>>>> doc.
> > > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>> -Artem
> > > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>> On Mon, Jan 9, 2023 at 3:38 PM Justine
> Olshan
> > > > > > > > >>>>>>>>>>>>>>>> <jolshan@confluent.io.invalid
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>> Hey all, I've updated the KIP to
> incorporate
> > > > Jason's
> > > > > > > > >>>>>>>>>>>> suggestions.
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>
> > > > > > > > >>>>
> > > > > > > > >>
> > > > > > >
> > > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>> 1. Use AddPartitionsToTxn + verify flag to
> > > > check on
> > > > > > > > >>>>>>> old
> > > > > > > > >>>>>>>>>> clients
> > > > > > > > >>>>>>>>>>>>>>>>>> 2. Updated AddPartitionsToTxn API to
> support
> > > > > > > > >>>>>>> transaction
> > > > > > > > >>>>>>>>>>>> batching
> > > > > > > > >>>>>>>>>>>>>>>>>> 3. Mention IBP bump
> > > > > > > > >>>>>>>>>>>>>>>>>> 4. Mention auth change on new
> > > AddPartitionsToTxn
> > > > > > > > >>>>>>>> version.
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>> I'm planning on opening a vote soon.
> > > > > > > > >>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > > >>>>>>>>>>>>>>>>>> Justine
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:32 PM Justine
> > Olshan
> > > <
> > > > > > > > >>>>>>>>>>>> jolshan@confluent.io
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>> Thanks Jason. Those changes make sense to
> > > me. I
> > > > > > > > >>>>>> will
> > > > > > > > >>>>>>>>>> update
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>> KIP.
> > > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:31 PM Jason
> > > Gustafson
> > > > > > > > >>>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
> > > > > > > > >>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>> Hey Justine,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> I was wondering about compatibility
> here.
> > > > When
> > > > > > > > >>>>>> we
> > > > > > > > >>>>>>>>> send
> > > > > > > > >>>>>>>>>>>> requests
> > > > > > > > >>>>>>>>>>>>>>>>>>>> between brokers, we want to ensure that
> > the
> > > > > > > > >>>>>>> receiving
> > > > > > > > >>>>>>>>>> broker
> > > > > > > > >>>>>>>>>>>>>>>>> understands
> > > > > > > > >>>>>>>>>>>>>>>>>>>> the request (specifically the new
> fields).
> > > > > > > > >>>>>>> Typically
> > > > > > > > >>>>>>>>>> this is
> > > > > > > > >>>>>>>>>>>> done
> > > > > > > > >>>>>>>>>>>>>>>> via
> > > > > > > > >>>>>>>>>>>>>>>>>>>> IBP/metadata version.
> > > > > > > > >>>>>>>>>>>>>>>>>>>> I'm trying to think if there is a way
> > around
> > > > it
> > > > > > > > >>>>>> but
> > > > > > > > >>>>>>>> I'm
> > > > > > > > >>>>>>>>>> not
> > > > > > > > >>>>>>>>>>>> sure
> > > > > > > > >>>>>>>>>>>>>>>> there
> > > > > > > > >>>>>>>>>>>>>>>>>> is.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>> Yes. I think we would gate usage of this
> > > > behind
> > > > > > > > >>>>>> an
> > > > > > > > >>>>>>>> IBP
> > > > > > > > >>>>>>>>>> bump.
> > > > > > > > >>>>>>>>>>>> Does
> > > > > > > > >>>>>>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>>>>>>> seem
> > > > > > > > >>>>>>>>>>>>>>>>>>>> reasonable?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> As for the improvements -- can you
> > clarify
> > > > how
> > > > > > > > >>>>>>> the
> > > > > > > > >>>>>>>>>> multiple
> > > > > > > > >>>>>>>>>>>>>>>>>>>> transactional
> > > > > > > > >>>>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking
> of
> > a
> > > > case
> > > > > > > > >>>>>>>> where
> > > > > > > > >>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>> wait/batch
> > > > > > > > >>>>>>>>>>>>>>>>>>>> multiple produce requests together? My
> > > > > > > > >>>>>>> understanding
> > > > > > > > >>>>>>>>> for
> > > > > > > > >>>>>>>>>> now
> > > > > > > > >>>>>>>>>>>> was
> > > > > > > > >>>>>>>>>>>>>>> 1
> > > > > > > > >>>>>>>>>>>>>>>>>>>> transactional ID and one validation per
> 1
> > > > produce
> > > > > > > > >>>>>>>>>> request.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>> Each call to `AddPartitionsToTxn` is
> > > > essentially
> > > > > > > > >>>>>> a
> > > > > > > > >>>>>>>>> write
> > > > > > > > >>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>>>>>>> log and must block on replication. The
> > more
> > > we
> > > > > > > > >>>>>> can
> > > > > > > > >>>>>>>> fit
> > > > > > > > >>>>>>>>>> into a
> > > > > > > > >>>>>>>>>>>>>>> single
> > > > > > > > >>>>>>>>>>>>>>>>>>>> request, the more writes we can do in
> > > > parallel.
> > > > > > > > >>>>>> The
> > > > > > > > >>>>>>>>>>>> alternative
> > > > > > > > >>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>> make
> > > > > > > > >>>>>>>>>>>>>>>>>>>> use of more connections, but usually we
> > > prefer
> > > > > > > > >>>>>>>> batching
> > > > > > > > >>>>>>>>>>>> since the
> > > > > > > > >>>>>>>>>>>>>>>>>> network
> > > > > > > > >>>>>>>>>>>>>>>>>>>> stack is not really optimized for high
> > > > > > > > >>>>>>>>> connection/request
> > > > > > > > >>>>>>>>>>>> loads.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> Finally with respect to the
> > > authorizations, I
> > > > > > > > >>>>>>> think
> > > > > > > > >>>>>>>>> it
> > > > > > > > >>>>>>>>>>>> makes
> > > > > > > > >>>>>>>>>>>>>>> sense
> > > > > > > > >>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>> skip
> > > > > > > > >>>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit
> > confused
> > > > by
> > > > > > > > >>>>>> the
> > > > > > > > >>>>>>>>>> "leader
> > > > > > > > >>>>>>>>>>>> ID"
> > > > > > > > >>>>>>>>>>>>>>>> field.
> > > > > > > > >>>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the
> request
> > as
> > > > > > > > >>>>>> from a
> > > > > > > > >>>>>>>>>> broker
> > > > > > > > >>>>>>>>>>>> (does
> > > > > > > > >>>>>>>>>>>>>>> it
> > > > > > > > >>>>>>>>>>>>>>>>>> matter
> > > > > > > > >>>>>>>>>>>>>>>>>>>> which one?).
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>> We could also make it version-based. For
> > the
> > > > next
> > > > > > > > >>>>>>>>>> version, we
> > > > > > > > >>>>>>>>>>>>>>> could
> > > > > > > > >>>>>>>>>>>>>>>>>>>> require
> > > > > > > > >>>>>>>>>>>>>>>>>>>> CLUSTER auth. So clients would not be
> able
> > > to
> > > > use
> > > > > > > > >>>>>>> the
> > > > > > > > >>>>>>>>> API
> > > > > > > > >>>>>>>>>>>>>>> anymore,
> > > > > > > > >>>>>>>>>>>>>>>>> which
> > > > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>> probably what we want.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>> -Jason
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:43 AM Justine
> > > Olshan
> > > > > > > > >>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> As a follow up, I was just thinking
> about
> > > the
> > > > > > > > >>>>>>>>> batching
> > > > > > > > >>>>>>>>>> a
> > > > > > > > >>>>>>>>>>>> bit
> > > > > > > > >>>>>>>>>>>>>>> more.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> I suppose if we have one request in
> > flight
> > > > and
> > > > > > > > >>>>>> we
> > > > > > > > >>>>>>>>>> queue up
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>> other
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> produce requests in some sort of
> > purgatory,
> > > > we
> > > > > > > > >>>>>>>> could
> > > > > > > > >>>>>>>>>> send
> > > > > > > > >>>>>>>>>>>>>>>>> information
> > > > > > > > >>>>>>>>>>>>>>>>>>>> out
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> for all of them rather than one by one.
> > So
> > > > that
> > > > > > > > >>>>>>>> would
> > > > > > > > >>>>>>>>>> be a
> > > > > > > > >>>>>>>>>>>>>>> benefit
> > > > > > > > >>>>>>>>>>>>>>>>> of
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> batching partitions to add per
> > transaction.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> I'll need to think a bit more on the
> > design
> > > > of
> > > > > > > > >>>>>>> this
> > > > > > > > >>>>>>>>>> part
> > > > > > > > >>>>>>>>>>>> of the
> > > > > > > > >>>>>>>>>>>>>>>> KIP,
> > > > > > > > >>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> will update the KIP in the next few
> days.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:22 AM Justine
> > > > Olshan
> > > > > > > > >>>>>> <
> > > > > > > > >>>>>>>>>>>>>>>>> jolshan@confluent.io>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> Hey Jason -- thanks for the input -- I
> > was
> > > > > > > > >>>>>> just
> > > > > > > > >>>>>>>>>> digging
> > > > > > > > >>>>>>>>>>>> a bit
> > > > > > > > >>>>>>>>>>>>>>>>> deeper
> > > > > > > > >>>>>>>>>>>>>>>>>>>> into
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> the design + implementation of the
> > > > validation
> > > > > > > > >>>>>>>> calls
> > > > > > > > >>>>>>>>>> here
> > > > > > > > >>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>> what
> > > > > > > > >>>>>>>>>>>>>>>>>> you
> > > > > > > > >>>>>>>>>>>>>>>>>>>> say
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> makes sense.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> I was wondering about compatibility
> > here.
> > > > > > > > >>>>>> When
> > > > > > > > >>>>>>> we
> > > > > > > > >>>>>>>>>> send
> > > > > > > > >>>>>>>>>>>>>>> requests
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> between brokers, we want to ensure
> that
> > > the
> > > > > > > > >>>>>>>>> receiving
> > > > > > > > >>>>>>>>>>>> broker
> > > > > > > > >>>>>>>>>>>>>>>>>>>> understands
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> the request (specifically the new
> > fields).
> > > > > > > > >>>>>>>>> Typically
> > > > > > > > >>>>>>>>>>>> this is
> > > > > > > > >>>>>>>>>>>>>>>> done
> > > > > > > > >>>>>>>>>>>>>>>>>> via
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> IBP/metadata version.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> I'm trying to think if there is a way
> > > around
> > > > > > > > >>>>>> it
> > > > > > > > >>>>>>>> but
> > > > > > > > >>>>>>>>>> I'm
> > > > > > > > >>>>>>>>>>>> not
> > > > > > > > >>>>>>>>>>>>>>> sure
> > > > > > > > >>>>>>>>>>>>>>>>>> there
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> is.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> As for the improvements -- can you
> > clarify
> > > > > > > > >>>>>> how
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>>>> multiple
> > > > > > > > >>>>>>>>>>>>>>>>>>>> transactional
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking
> > of
> > > a
> > > > > > > > >>>>>>> case
> > > > > > > > >>>>>>>>>> where we
> > > > > > > > >>>>>>>>>>>>>>>>> wait/batch
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> multiple produce requests together? My
> > > > > > > > >>>>>>>>> understanding
> > > > > > > > >>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>> now
> > > > > > > > >>>>>>>>>>>>>>>> was 1
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> transactional ID and one validation
> per
> > 1
> > > > > > > > >>>>>>> produce
> > > > > > > > >>>>>>>>>>>> request.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> Finally with respect to the
> > > authorizations,
> > > > I
> > > > > > > > >>>>>>>> think
> > > > > > > > >>>>>>>>>> it
> > > > > > > > >>>>>>>>>>>> makes
> > > > > > > > >>>>>>>>>>>>>>>> sense
> > > > > > > > >>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> skip
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit
> > > confused
> > > > > > > > >>>>>> by
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>>>> "leader
> > > > > > > > >>>>>>>>>>>>>>> ID"
> > > > > > > > >>>>>>>>>>>>>>>>>> field.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the
> > request
> > > as
> > > > > > > > >>>>>>>> from a
> > > > > > > > >>>>>>>>>>>> broker
> > > > > > > > >>>>>>>>>>>>>>> (does
> > > > > > > > >>>>>>>>>>>>>>>>> it
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> matter
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> which one?).
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> I think I want to adopt these
> > suggestions,
> > > > > > > > >>>>>> just
> > > > > > > > >>>>>>>> had
> > > > > > > > >>>>>>>>>> a few
> > > > > > > > >>>>>>>>>>>>>>>>> questions
> > > > > > > > >>>>>>>>>>>>>>>>>> on
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> details.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> On Thu, Jan 5, 2023 at 5:05 PM Jason
> > > > > > > > >>>>>> Gustafson
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Thanks for the proposal.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> I was thinking about the
> > implementation a
> > > > > > > > >>>>>>> little
> > > > > > > > >>>>>>>>>> bit.
> > > > > > > > >>>>>>>>>>>> In the
> > > > > > > > >>>>>>>>>>>>>>>>>> current
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> proposal, the behavior depends on
> > whether
> > > > we
> > > > > > > > >>>>>>>> have
> > > > > > > > >>>>>>>>> an
> > > > > > > > >>>>>>>>>>>> old or
> > > > > > > > >>>>>>>>>>>>>>> new
> > > > > > > > >>>>>>>>>>>>>>>>>>>> client.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> For
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> old clients, we send
> > > `DescribeTransactions`
> > > > > > > > >>>>>>> and
> > > > > > > > >>>>>>>>>> verify
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>> result
> > > > > > > > >>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> new clients, we send
> > > `AddPartitionsToTxn`.
> > > > > > > > >>>>>> We
> > > > > > > > >>>>>>>>> might
> > > > > > > > >>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>> able
> > > > > > > > >>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>> simplify
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> implementation if we can use the same
> > > > > > > > >>>>>> request
> > > > > > > > >>>>>>>>> type.
> > > > > > > > >>>>>>>>>> For
> > > > > > > > >>>>>>>>>>>>>>>> example,
> > > > > > > > >>>>>>>>>>>>>>>>>>>> what if
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> bump the protocol version for
> > > > > > > > >>>>>>>> `AddPartitionsToTxn`
> > > > > > > > >>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>> add a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> `validateOnly`
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> flag? For older versions, we can set
> > > > > > > > >>>>>>>>>>>> `validateOnly=true` so
> > > > > > > > >>>>>>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> request only returns successfully if
> > the
> > > > > > > > >>>>>>>> partition
> > > > > > > > >>>>>>>>>> had
> > > > > > > > >>>>>>>>>>>>>>> already
> > > > > > > > >>>>>>>>>>>>>>>>> been
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> added.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> For new versions, we can set
> > > > > > > > >>>>>>>> `validateOnly=false`
> > > > > > > > >>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>> partition
> > > > > > > > >>>>>>>>>>>>>>>>>>>> will
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> added to the transaction. The other
> > > > slightly
> > > > > > > > >>>>>>>>>> annoying
> > > > > > > > >>>>>>>>>>>> thing
> > > > > > > > >>>>>>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> get around is the need to collect the
> > > > > > > > >>>>>>>> transaction
> > > > > > > > >>>>>>>>>> state
> > > > > > > > >>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>> all
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> partitions
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> even when we only care about a
> subset.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Some additional improvements to
> > consider:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> - We can give `AddPartitionsToTxn`
> > better
> > > > > > > > >>>>>>> batch
> > > > > > > > >>>>>>>>>> support
> > > > > > > > >>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>>>> inter-broker
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> usage. Currently we only allow one
> > > > > > > > >>>>>>>>>> `TransactionalId` to
> > > > > > > > >>>>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>>>>> specified,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> but
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the broker may get some benefit being
> > > able
> > > > > > > > >>>>>> to
> > > > > > > > >>>>>>>>> batch
> > > > > > > > >>>>>>>>>>>> across
> > > > > > > > >>>>>>>>>>>>>>>>> multiple
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> transactions.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> - Another small improvement is
> skipping
> > > > > > > > >>>>>> topic
> > > > > > > > >>>>>>>>>>>> authorization
> > > > > > > > >>>>>>>>>>>>>>>>> checks
> > > > > > > > >>>>>>>>>>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> `AddPartitionsToTxn` when the request
> > is
> > > > > > > > >>>>>> from
> > > > > > > > >>>>>>> a
> > > > > > > > >>>>>>>>>> broker.
> > > > > > > > >>>>>>>>>>>>>>> Perhaps
> > > > > > > > >>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>> can
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> add
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> a field for the `LeaderId` or
> something
> > > > like
> > > > > > > > >>>>>>>> that
> > > > > > > > >>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>> require
> > > > > > > > >>>>>>>>>>>>>>>>>> CLUSTER
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> permission when set.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Best,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Jason
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 3:56 PM Jun
> Rao
> > > > > > > > >>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation. It makes
> > > sense
> > > > > > > > >>>>>>> to
> > > > > > > > >>>>>>>> me
> > > > > > > > >>>>>>>>>> now.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 1:42 PM
> > Justine
> > > > > > > > >>>>>>> Olshan
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> My understanding of the mechanism
> is
> > > > > > > > >>>>>> that
> > > > > > > > >>>>>>>> when
> > > > > > > > >>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>> get to
> > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>> last
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> epoch,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> increment to the fencing/last epoch
> > and
> > > > > > > > >>>>>> if
> > > > > > > > >>>>>>>> any
> > > > > > > > >>>>>>>>>>>> further
> > > > > > > > >>>>>>>>>>>>>>>>> requests
> > > > > > > > >>>>>>>>>>>>>>>>>>>> come
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> this producer ID they are fenced.
> > Then
> > > > > > > > >>>>>> the
> > > > > > > > >>>>>>>>>> producer
> > > > > > > > >>>>>>>>>>>>>>> gets a
> > > > > > > > >>>>>>>>>>>>>>>>> new
> > > > > > > > >>>>>>>>>>>>>>>>>> ID
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> restarts with epoch/sequence 0. The
> > > > > > > > >>>>>> fenced
> > > > > > > > >>>>>>>>> epoch
> > > > > > > > >>>>>>>>>>>> sticks
> > > > > > > > >>>>>>>>>>>>>>>>> around
> > > > > > > > >>>>>>>>>>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> duration of
> > producer.id.expiration.ms
> > > > > > > > >>>>>> and
> > > > > > > > >>>>>>>>>> blocks
> > > > > > > > >>>>>>>>>>>> any
> > > > > > > > >>>>>>>>>>>>>>> late
> > > > > > > > >>>>>>>>>>>>>>>>>>>> messages
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> there.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> The new ID will get to take
> advantage
> > > of
> > > > > > > > >>>>>>> the
> > > > > > > > >>>>>>>>>>>> improved
> > > > > > > > >>>>>>>>>>>>>>>>> semantics
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> around
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> non-zero start sequences. So I
> think
> > we
> > > > > > > > >>>>>>> are
> > > > > > > > >>>>>>>>>> covered.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> The only potential issue is
> > overloading
> > > > > > > > >>>>>>> the
> > > > > > > > >>>>>>>>>> cache,
> > > > > > > > >>>>>>>>>>>> but
> > > > > > > > >>>>>>>>>>>>>>>>>> hopefully
> > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> improvements (lowered
> > > > > > > > >>>>>>>>> producer.id.expiration.ms
> > > > > > > > >>>>>>>>>> )
> > > > > > > > >>>>>>>>>>>> will
> > > > > > > > >>>>>>>>>>>>>>> help
> > > > > > > > >>>>>>>>>>>>>>>>>> with
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> that.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Let
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> me know if you still have concerns.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 10:24 AM
> Jun
> > > Rao
> > > > > > > > >>>>>>>>>>>>>>>>>>>> <ju...@confluent.io.invalid>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> 70. The proposed fencing logic
> > doesn't
> > > > > > > > >>>>>>>> apply
> > > > > > > > >>>>>>>>>> when
> > > > > > > > >>>>>>>>>>>> pid
> > > > > > > > >>>>>>>>>>>>>>>>>> changes,
> > > > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> right? If so, I am not sure how
> > > > > > > > >>>>>> complete
> > > > > > > > >>>>>>>> we
> > > > > > > > >>>>>>>>>> are
> > > > > > > > >>>>>>>>>>>>>>>> addressing
> > > > > > > > >>>>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> issue
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the pid changes more frequently.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 16, 2022 at 9:16 AM
> > > > > > > > >>>>>> Justine
> > > > > > > > >>>>>>>>> Olshan
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for replying!
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> 70.We already do the overflow
> > > > > > > > >>>>>>> mechanism,
> > > > > > > > >>>>>>>>> so
> > > > > > > > >>>>>>>>>> my
> > > > > > > > >>>>>>>>>>>>>>> change
> > > > > > > > >>>>>>>>>>>>>>>>> would
> > > > > > > > >>>>>>>>>>>>>>>>>>>> just
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> make
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> happen more often.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I was also not suggesting a new
> > > > > > > > >>>>>> field
> > > > > > > > >>>>>>> in
> > > > > > > > >>>>>>>>> the
> > > > > > > > >>>>>>>>>>>> log,
> > > > > > > > >>>>>>>>>>>>>>> but
> > > > > > > > >>>>>>>>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> response,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> which would be gated by the
> client
> > > > > > > > >>>>>>>>> version.
> > > > > > > > >>>>>>>>>>>> Sorry if
> > > > > > > > >>>>>>>>>>>>>>>>>>>> something
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> there
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> unclear. I think we are starting
> to
> > > > > > > > >>>>>>>>> diverge.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> The goal of this KIP is to not
> > > > > > > > >>>>>> change
> > > > > > > > >>>>>>> to
> > > > > > > > >>>>>>>>> the
> > > > > > > > >>>>>>>>>>>> marker
> > > > > > > > >>>>>>>>>>>>>>>>> format
> > > > > > > > >>>>>>>>>>>>>>>>>> at
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> all.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> 71. Yes, I guess I was going
> under
> > > > > > > > >>>>>> the
> > > > > > > > >>>>>>>>>>>> assumption
> > > > > > > > >>>>>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>> log
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> just
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> look at its last epoch and treat
> it
> > > > > > > > >>>>>> as
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>>>> current
> > > > > > > > >>>>>>>>>>>>>>>>> epoch. I
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> suppose
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> have some special logic that if
> the
> > > > > > > > >>>>>>> last
> > > > > > > > >>>>>>>>>> epoch
> > > > > > > > >>>>>>>>>>>> was
> > > > > > > > >>>>>>>>>>>>>>> on a
> > > > > > > > >>>>>>>>>>>>>>>>>>>> marker
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> actually
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> expect the next epoch or
> something
> > > > > > > > >>>>>>> like
> > > > > > > > >>>>>>>>>> that. We
> > > > > > > > >>>>>>>>>>>>>>> just
> > > > > > > > >>>>>>>>>>>>>>>>> need
> > > > > > > > >>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> distinguish
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> based on whether we had a
> > > > > > > > >>>>>> commit/abort
> > > > > > > > >>>>>>>>>> marker.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> 72.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> if the producer epoch hasn't
> been
> > > > > > > > >>>>>>>> bumped
> > > > > > > > >>>>>>>>>> on
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
> > > > > > > > >>>>>>>> message
> > > > > > > > >>>>>>>>>> will
> > > > > > > > >>>>>>>>>>>> fail
> > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>> sequence
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> validation
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the
> > producer
> > > > > > > > >>>>>>>> epoch
> > > > > > > > >>>>>>>>>> has
> > > > > > > > >>>>>>>>>>>> been
> > > > > > > > >>>>>>>>>>>>>>>>> bumped,
> > > > > > > > >>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> ignore
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck
> > message
> > > > > > > > >>>>>>>> could
> > > > > > > > >>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>> appended
> > > > > > > > >>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> log.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> So,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
> > > > > > > > >>>>>> guard?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I follow that "the
> > > > > > > > >>>>>>> message
> > > > > > > > >>>>>>>>> will
> > > > > > > > >>>>>>>>>>>> fail
> > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>> sequence
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> validation".
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> In some of these cases, we had an
> > > > > > > > >>>>>>> abort
> > > > > > > > >>>>>>>>>> marker
> > > > > > > > >>>>>>>>>>>> (due
> > > > > > > > >>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>> an
> > > > > > > > >>>>>>>>>>>>>>>>>>>> error)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> then
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the late message comes in with
> the
> > > > > > > > >>>>>>>> correct
> > > > > > > > >>>>>>>>>>>> sequence
> > > > > > > > >>>>>>>>>>>>>>>>> number.
> > > > > > > > >>>>>>>>>>>>>>>>>>>> This
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> is a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> case
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> covered by the KIP.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> The latter case is actually not
> > > > > > > > >>>>>>>> something
> > > > > > > > >>>>>>>>>> we've
> > > > > > > > >>>>>>>>>>>>>>>>> considered
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> here. I
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> think
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> generally when we bump the epoch,
> > we
> > > > > > > > >>>>>>> are
> > > > > > > > >>>>>>>>>>>> accepting
> > > > > > > > >>>>>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> sequence
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> does
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> not need to be checked anymore.
> My
> > > > > > > > >>>>>>>>>>>> understanding is
> > > > > > > > >>>>>>>>>>>>>>>> also
> > > > > > > > >>>>>>>>>>>>>>>>>>>> that we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> typically bump epoch mid
> > transaction
> > > > > > > > >>>>>>>>> (based
> > > > > > > > >>>>>>>>>> on a
> > > > > > > > >>>>>>>>>>>>>>> quick
> > > > > > > > >>>>>>>>>>>>>>>>> look
> > > > > > > > >>>>>>>>>>>>>>>>>>>> at
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> code)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but let me know if that is the
> > case.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 15, 2022 at 12:23 PM
> > Jun
> > > > > > > > >>>>>>> Rao
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the reply.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Assigning a new pid on int
> > > > > > > > >>>>>>>> overflow
> > > > > > > > >>>>>>>>>> seems
> > > > > > > > >>>>>>>>>>>> a
> > > > > > > > >>>>>>>>>>>>>>> bit
> > > > > > > > >>>>>>>>>>>>>>>>>> hacky.
> > > > > > > > >>>>>>>>>>>>>>>>>>>> If
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> need a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> level id, it will be better to
> > > > > > > > >>>>>> model
> > > > > > > > >>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>> explicitly.
> > > > > > > > >>>>>>>>>>>>>>>>>>>> Adding a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> field
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> would require a bit more work
> > > > > > > > >>>>>> since
> > > > > > > > >>>>>>> it
> > > > > > > > >>>>>>>>>>>> requires a
> > > > > > > > >>>>>>>>>>>>>>> new
> > > > > > > > >>>>>>>>>>>>>>>>> txn
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> marker
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> format
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the log. So, we probably need to
> > > > > > > > >>>>>>> guard
> > > > > > > > >>>>>>>>> it
> > > > > > > > >>>>>>>>>>>> with an
> > > > > > > > >>>>>>>>>>>>>>> IBP
> > > > > > > > >>>>>>>>>>>>>>>>> or
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> metadata
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> version
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and document the impact on
> > > > > > > > >>>>>> downgrade
> > > > > > > > >>>>>>>>> once
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>> new
> > > > > > > > >>>>>>>>>>>>>>>>> format
> > > > > > > > >>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> written
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 71. Hmm, once the marker is
> > > > > > > > >>>>>> written,
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>>>> partition
> > > > > > > > >>>>>>>>>>>>>>>> will
> > > > > > > > >>>>>>>>>>>>>>>>>>>> expect
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> next
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> append to be on the next epoch.
> > > > > > > > >>>>>> Does
> > > > > > > > >>>>>>>>> that
> > > > > > > > >>>>>>>>>>>> cover
> > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>> case
> > > > > > > > >>>>>>>>>>>>>>>>>>>> you
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> mentioned?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 72. Also, just to be clear on
> the
> > > > > > > > >>>>>>>>> stucked
> > > > > > > > >>>>>>>>>>>> message
> > > > > > > > >>>>>>>>>>>>>>>> issue
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> described
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> motivation. With EoS, we also
> > > > > > > > >>>>>>> validate
> > > > > > > > >>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>> sequence
> > > > > > > > >>>>>>>>>>>>>>>> id
> > > > > > > > >>>>>>>>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> idempotency.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> So,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> with the current logic, if the
> > > > > > > > >>>>>>>> producer
> > > > > > > > >>>>>>>>>> epoch
> > > > > > > > >>>>>>>>>>>>>>> hasn't
> > > > > > > > >>>>>>>>>>>>>>>>> been
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> bumped on
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the
> stucked
> > > > > > > > >>>>>>>>> message
> > > > > > > > >>>>>>>>>> will
> > > > > > > > >>>>>>>>>>>>>>> fail
> > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> sequence
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> validation
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the
> > > > > > > > >>>>>> producer
> > > > > > > > >>>>>>>>>> epoch has
> > > > > > > > >>>>>>>>>>>>>>> been
> > > > > > > > >>>>>>>>>>>>>>>>>>>> bumped, we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> ignore
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck
> > > > > > > > >>>>>> message
> > > > > > > > >>>>>>>>>> could be
> > > > > > > > >>>>>>>>>>>>>>>> appended
> > > > > > > > >>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> log.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> So,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
> > > > > > > > >>>>>>> guard?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 10:44 AM
> > > > > > > > >>>>>>>> Justine
> > > > > > > > >>>>>>>>>>>> Olshan
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > > > > >>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias — thanks again for
> > > > > > > > >>>>>> taking
> > > > > > > > >>>>>>>>> time
> > > > > > > > >>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>> look
> > > > > > > > >>>>>>>>>>>>>>> a
> > > > > > > > >>>>>>>>>>>>>>>>> this.
> > > > > > > > >>>>>>>>>>>>>>>>>>>> You
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> said:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My proposal was only focusing
> > > > > > > > >>>>>> to
> > > > > > > > >>>>>>>>> avoid
> > > > > > > > >>>>>>>>>>>>>>> dangling
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
> > > > > > > > >>>>>> added
> > > > > > > > >>>>>>>>>> without
> > > > > > > > >>>>>>>>>>>>>>>> registered
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> partition.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> --
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more details
> > > > > > > > >>>>>> to
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>> KIP
> > > > > > > > >>>>>>>>>>>> about
> > > > > > > > >>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> scenario
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> better
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I understand what
> > > > > > > > >>>>>> you
> > > > > > > > >>>>>>>>> mean
> > > > > > > > >>>>>>>>>>>> here.
> > > > > > > > >>>>>>>>>>>>>>> The
> > > > > > > > >>>>>>>>>>>>>>>>>>>> motivation
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> section
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> describes two scenarios about
> > > > > > > > >>>>>> how
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>> record
> > > > > > > > >>>>>>>>>>>>>>> can be
> > > > > > > > >>>>>>>>>>>>>>>>>> added
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> without a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> registered partition:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Another way hanging
> > > > > > > > >>>>>> transactions
> > > > > > > > >>>>>>>> can
> > > > > > > > >>>>>>>>>>>> occur is
> > > > > > > > >>>>>>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>>>> a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> client
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> buggy
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> may somehow try to write to a
> > > > > > > > >>>>>>>>> partition
> > > > > > > > >>>>>>>>>>>> before
> > > > > > > > >>>>>>>>>>>>>>> it
> > > > > > > > >>>>>>>>>>>>>>>>> adds
> > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> partition
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> For the first example of this
> > > > > > > > >>>>>>> would
> > > > > > > > >>>>>>>> it
> > > > > > > > >>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>> helpful
> > > > > > > > >>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>> say
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> comes in after the abort, but
> > > > > > > > >>>>>>> before
> > > > > > > > >>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>> partition
> > > > > > > > >>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>> added
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> next
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction so it becomes
> > > > > > > > >>>>>>> "hanging."
> > > > > > > > >>>>>>>>>>>> Perhaps the
> > > > > > > > >>>>>>>>>>>>>>>> next
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> sentence
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> describing
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the message becoming part of
> the
> > > > > > > > >>>>>>>> next
> > > > > > > > >>>>>>>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>>> (a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> different
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> case)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> was
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not properly differentiated.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun — thanks for reading the
> > > > > > > > >>>>>> KIP.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. The int typing was a
> > > > > > > > >>>>>> concern.
> > > > > > > > >>>>>>>>>> Currently
> > > > > > > > >>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>> have a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> mechanism
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> place
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> fence the final epoch when the
> > > > > > > > >>>>>>> epoch
> > > > > > > > >>>>>>>>> is
> > > > > > > > >>>>>>>>>>>> about to
> > > > > > > > >>>>>>>>>>>>>>>>>> overflow
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> assign
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer ID with epoch 0. Of
> > > > > > > > >>>>>>> course,
> > > > > > > > >>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>> is a
> > > > > > > > >>>>>>>>>>>>>>> bit
> > > > > > > > >>>>>>>>>>>>>>>>>> tricky
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> when it
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> comes
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response back to the
> client.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Making this a long could be
> > > > > > > > >>>>>>> another
> > > > > > > > >>>>>>>>>> option,
> > > > > > > > >>>>>>>>>>>> but
> > > > > > > > >>>>>>>>>>>>>>> I
> > > > > > > > >>>>>>>>>>>>>>>>>> wonder
> > > > > > > > >>>>>>>>>>>>>>>>>>>> are
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> there
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> implications on changing this
> > > > > > > > >>>>>>> field
> > > > > > > > >>>>>>>> if
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>> epoch is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> persisted
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> disk?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to check the usages.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 71.This was something Matthias
> > > > > > > > >>>>>>> asked
> > > > > > > > >>>>>>>>>> about
> > > > > > > > >>>>>>>>>>>> as
> > > > > > > > >>>>>>>>>>>>>>>> well. I
> > > > > > > > >>>>>>>>>>>>>>>>>> was
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> considering a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible edge case where a
> > > > > > > > >>>>>> produce
> > > > > > > > >>>>>>>>>> request
> > > > > > > > >>>>>>>>>>>> from
> > > > > > > > >>>>>>>>>>>>>>> a
> > > > > > > > >>>>>>>>>>>>>>>> new
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> somehow
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> gets sent right after the
> marker
> > > > > > > > >>>>>>> is
> > > > > > > > >>>>>>>>>>>> written, but
> > > > > > > > >>>>>>>>>>>>>>>>> before
> > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> alerted of the newly bumped
> > > > > > > > >>>>>> epoch.
> > > > > > > > >>>>>>>> In
> > > > > > > > >>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>> case, we
> > > > > > > > >>>>>>>>>>>>>>>>> may
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> include
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> record
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we don't want to. I
> suppose
> > > > > > > > >>>>>>> we
> > > > > > > > >>>>>>>>>> could
> > > > > > > > >>>>>>>>>>>> try
> > > > > > > > >>>>>>>>>>>>>>> to do
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> something
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> client
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> side
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to bump the epoch after sending
> > > > > > > > >>>>>> an
> > > > > > > > >>>>>>>>>> endTxn as
> > > > > > > > >>>>>>>>>>>>>>> well
> > > > > > > > >>>>>>>>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> scenario
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> —
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> I
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wonder how it would work when
> > > > > > > > >>>>>> the
> > > > > > > > >>>>>>>>>> server is
> > > > > > > > >>>>>>>>>>>>>>>> aborting
> > > > > > > > >>>>>>>>>>>>>>>>>>>> based
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> on
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> error. I could also be missing
> > > > > > > > >>>>>>>>>> something and
> > > > > > > > >>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>> scenario
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> actually
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again to everyone
> reading
> > > > > > > > >>>>>>> and
> > > > > > > > >>>>>>>>>>>> commenting.
> > > > > > > > >>>>>>>>>>>>>>>> Let
> > > > > > > > >>>>>>>>>>>>>>>>> me
> > > > > > > > >>>>>>>>>>>>>>>>>>>> know
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> about
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> further questions or comments.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 9:41 AM
> > > > > > > > >>>>>>> Jun
> > > > > > > > >>>>>>>>> Rao
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP. A couple
> > > > > > > > >>>>>> of
> > > > > > > > >>>>>>>>>> comments.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Currently, the producer
> > > > > > > > >>>>>>> epoch
> > > > > > > > >>>>>>>> is
> > > > > > > > >>>>>>>>>> an
> > > > > > > > >>>>>>>>>>>> int.
> > > > > > > > >>>>>>>>>>>>>>> I am
> > > > > > > > >>>>>>>>>>>>>>>>> not
> > > > > > > > >>>>>>>>>>>>>>>>>>>> sure
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> it's
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> enough
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to accommodate all
> > > > > > > > >>>>>> transactions
> > > > > > > > >>>>>>> in
> > > > > > > > >>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>> lifetime
> > > > > > > > >>>>>>>>>>>>>>>> of
> > > > > > > > >>>>>>>>>>>>>>>>> a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> producer.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Should
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> change that to a long or add a
> > > > > > > > >>>>>>> new
> > > > > > > > >>>>>>>>>> long
> > > > > > > > >>>>>>>>>>>> field
> > > > > > > > >>>>>>>>>>>>>>>> like
> > > > > > > > >>>>>>>>>>>>>>>>>>>> txnId?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 71. "it will write the prepare
> > > > > > > > >>>>>>>>> commit
> > > > > > > > >>>>>>>>>>>> message
> > > > > > > > >>>>>>>>>>>>>>>> with
> > > > > > > > >>>>>>>>>>>>>>>>> a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> bumped
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> send
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WriteTxnMarkerRequests with
> > > > > > > > >>>>>> the
> > > > > > > > >>>>>>>>> bumped
> > > > > > > > >>>>>>>>>>>> epoch."
> > > > > > > > >>>>>>>>>>>>>>>> Hmm,
> > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> associated
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the current txn right?
> > > > > > > > >>>>>> So,
> > > > > > > > >>>>>>> it
> > > > > > > > >>>>>>>>>> seems
> > > > > > > > >>>>>>>>>>>>>>> weird to
> > > > > > > > >>>>>>>>>>>>>>>>>>>> write a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> commit
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with a bumped epoch. Should we
> > > > > > > > >>>>>>>> only
> > > > > > > > >>>>>>>>>> bump
> > > > > > > > >>>>>>>>>>>> up
> > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>> epoch
> > > > > > > > >>>>>>>>>>>>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> EndTxnResponse
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rename the field to sth like
> > > > > > > > >>>>>>>>>>>>>>> nextProducerEpoch?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 12, 2022 at 8:54
> > > > > > > > >>>>>> PM
> > > > > > > > >>>>>>>>>> Matthias
> > > > > > > > >>>>>>>>>>>> J.
> > > > > > > > >>>>>>>>>>>>>>> Sax <
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the background.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30: SGTM. My proposal was
> > > > > > > > >>>>>>>> only
> > > > > > > > >>>>>>>>>>>> focusing
> > > > > > > > >>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>> avoid
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> dangling
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
> > > > > > > > >>>>>>>> added
> > > > > > > > >>>>>>>>>>>> without
> > > > > > > > >>>>>>>>>>>>>>>>>> registered
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> partition.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> --
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more
> > > > > > > > >>>>>> details
> > > > > > > > >>>>>>>> to
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>> KIP
> > > > > > > > >>>>>>>>>>>>>>> about
> > > > > > > > >>>>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> scenario
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40: I think you hit a fair
> > > > > > > > >>>>>>> point
> > > > > > > > >>>>>>>>>> about
> > > > > > > > >>>>>>>>>>>> race
> > > > > > > > >>>>>>>>>>>>>>>>>>>> conditions
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> or
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> client
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> bugs
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (incorrectly not bumping the
> > > > > > > > >>>>>>>>>> epoch). The
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> complexity/confusion
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> using
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the bumped epoch I see, is
> > > > > > > > >>>>>>>> mainly
> > > > > > > > >>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>> internal
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> debugging,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> ie,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> inspecting
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log segment dumps -- it
> > > > > > > > >>>>>> seems
> > > > > > > > >>>>>>>>>> harder to
> > > > > > > > >>>>>>>>>>>>>>> reason
> > > > > > > > >>>>>>>>>>>>>>>>>> about
> > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> system
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> us
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> humans. But if we get better
> > > > > > > > >>>>>>>>>>>> guarantees, it
> > > > > > > > >>>>>>>>>>>>>>>> would
> > > > > > > > >>>>>>>>>>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> worth to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> use
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped epoch.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60: as I mentioned already,
> > > > > > > > >>>>>> I
> > > > > > > > >>>>>>>>> don't
> > > > > > > > >>>>>>>>>>>> know the
> > > > > > > > >>>>>>>>>>>>>>>>> broker
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> internals
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> provide
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more input. So if nobody
> > > > > > > > >>>>>> else
> > > > > > > > >>>>>>>>> chimes
> > > > > > > > >>>>>>>>>>>> in, we
> > > > > > > > >>>>>>>>>>>>>>>>> should
> > > > > > > > >>>>>>>>>>>>>>>>>>>> just
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> move
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> forward
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with your proposal.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 12/6/22 4:22 PM, Justine
> > > > > > > > >>>>>>>> Olshan
> > > > > > > > >>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi all,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> After Artem's questions
> > > > > > > > >>>>>>> about
> > > > > > > > >>>>>>>>>> error
> > > > > > > > >>>>>>>>>>>>>>> behavior,
> > > > > > > > >>>>>>>>>>>>>>>>>> I've
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> re-evaluated
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unknown producer ID
> > > > > > > > >>>>>>> exception
> > > > > > > > >>>>>>>>> and
> > > > > > > > >>>>>>>>>> had
> > > > > > > > >>>>>>>>>>>> some
> > > > > > > > >>>>>>>>>>>>>>>>>>>> discussions
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> offline.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think generally it makes
> > > > > > > > >>>>>>>> sense
> > > > > > > > >>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>> simplify
> > > > > > > > >>>>>>>>>>>>>>>>>> error
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> handling
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> cases
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this and the
> > > > > > > > >>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > >>>>>>>>>> error
> > > > > > > > >>>>>>>>>>>>>>> has a
> > > > > > > > >>>>>>>>>>>>>>>>>> pretty
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> long
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complicated
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> history. Because of this,
> > > > > > > > >>>>>> I
> > > > > > > > >>>>>>>>>> propose
> > > > > > > > >>>>>>>>>>>>>>> adding a
> > > > > > > > >>>>>>>>>>>>>>>>> new
> > > > > > > > >>>>>>>>>>>>>>>>>>>> error
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> code
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ABORTABLE_ERROR
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that when encountered by
> > > > > > > > >>>>>> new
> > > > > > > > >>>>>>>>>> clients
> > > > > > > > >>>>>>>>>>>>>>> (gated
> > > > > > > > >>>>>>>>>>>>>>>> by
> > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> produce
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will simply abort the
> > > > > > > > >>>>>>>>> transaction.
> > > > > > > > >>>>>>>>>>>> This
> > > > > > > > >>>>>>>>>>>>>>>> allows
> > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> server
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> some
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> say
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in whether the client
> > > > > > > > >>>>>> aborts
> > > > > > > > >>>>>>>> and
> > > > > > > > >>>>>>>>>> makes
> > > > > > > > >>>>>>>>>>>>>>>> handling
> > > > > > > > >>>>>>>>>>>>>>>>>>>> much
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> simpler.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> future, we can also use
> > > > > > > > >>>>>> this
> > > > > > > > >>>>>>>>>> error in
> > > > > > > > >>>>>>>>>>>>>>> other
> > > > > > > > >>>>>>>>>>>>>>>>>>>> situations
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> where
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> want
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> abort the transactions. We
> > > > > > > > >>>>>>> can
> > > > > > > > >>>>>>>>>> even
> > > > > > > > >>>>>>>>>>>> use on
> > > > > > > > >>>>>>>>>>>>>>>>> other
> > > > > > > > >>>>>>>>>>>>>>>>>>>> apis.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've added this to the
> > > > > > > > >>>>>> KIP.
> > > > > > > > >>>>>>>> Let
> > > > > > > > >>>>>>>>> me
> > > > > > > > >>>>>>>>>>>> know if
> > > > > > > > >>>>>>>>>>>>>>>>> there
> > > > > > > > >>>>>>>>>>>>>>>>>>>> are
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> any
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> questions
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> or
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 2, 2022 at
> > > > > > > > >>>>>> 10:22
> > > > > > > > >>>>>>>> AM
> > > > > > > > >>>>>>>>>>>> Justine
> > > > > > > > >>>>>>>>>>>>>>>> Olshan
> > > > > > > > >>>>>>>>>>>>>>>>> <
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> jolshan@confluent.io
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey Matthias,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30 — Maybe I also
> > > > > > > > >>>>>> didn't
> > > > > > > > >>>>>>>>>> express
> > > > > > > > >>>>>>>>>>>>>>> myself
> > > > > > > > >>>>>>>>>>>>>>>>>>>> clearly.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> For
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't have a way to
> > > > > > > > >>>>>>>> distinguish
> > > > > > > > >>>>>>>>>>>> between a
> > > > > > > > >>>>>>>>>>>>>>>>>> previous
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> current
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction since we
> > > > > > > > >>>>>> don't
> > > > > > > > >>>>>>>> have
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>> epoch
> > > > > > > > >>>>>>>>>>>>>>>>> bump.
> > > > > > > > >>>>>>>>>>>>>>>>>>>> This
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> means
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message from the previous
> > > > > > > > >>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>> may be
> > > > > > > > >>>>>>>>>>>>>>>>>>>> added to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> one.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> With
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older clients — we can't
> > > > > > > > >>>>>>>>>> guarantee
> > > > > > > > >>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>> won't
> > > > > > > > >>>>>>>>>>>>>>>>>>>> happen
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> if we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> already
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call
> > > > > > > > >>>>>>> (why
> > > > > > > > >>>>>>>> we
> > > > > > > > >>>>>>>>>> make
> > > > > > > > >>>>>>>>>>>>>>> changes
> > > > > > > > >>>>>>>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> newer
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> client)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can at least gate some by
> > > > > > > > >>>>>>>>>> ensuring
> > > > > > > > >>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>> partition
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> has
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> been
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> added
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. The
> > > > > > > > >>>>>> rationale
> > > > > > > > >>>>>>>> here
> > > > > > > > >>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>>> there
> > > > > > > > >>>>>>>>>>>>>>>>>> are
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> likely
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> LESS
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrivals
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as time goes on, so
> > > > > > > > >>>>>>> hopefully
> > > > > > > > >>>>>>>>>> most
> > > > > > > > >>>>>>>>>>>> late
> > > > > > > > >>>>>>>>>>>>>>>>> arrivals
> > > > > > > > >>>>>>>>>>>>>>>>>>>> will
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> come
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> BEFORE
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call.
> > > > > > > > >>>>>>>> Those
> > > > > > > > >>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>> arrive
> > > > > > > > >>>>>>>>>>>>>>>>>> before
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> will
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> properly
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> gated
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> > > > > > > > >>>>>>> describeTransactions
> > > > > > > > >>>>>>>>>>>> approach.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we take the approach
> > > > > > > > >>>>>> you
> > > > > > > > >>>>>>>>>>>> suggested,
> > > > > > > > >>>>>>>>>>>>>>> ANY
> > > > > > > > >>>>>>>>>>>>>>>>> late
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> arrival
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> from a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> previous
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction will be
> > > > > > > > >>>>>> added.
> > > > > > > > >>>>>>>> And
> > > > > > > > >>>>>>>>> we
> > > > > > > > >>>>>>>>>>>> don't
> > > > > > > > >>>>>>>>>>>>>>> want
> > > > > > > > >>>>>>>>>>>>>>>>>>>> that. I
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> also
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> see
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> benefit in sending
> > > > > > > > >>>>>>>>>> addPartitionsToTxn
> > > > > > > > >>>>>>>>>>>>>>> over
> > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> describeTxns
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> They
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> both be one extra RPC to
> > > > > > > > >>>>>>> the
> > > > > > > > >>>>>>>>> Txn
> > > > > > > > >>>>>>>>>>>>>>>> coordinator.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To be clear — newer
> > > > > > > > >>>>>> clients
> > > > > > > > >>>>>>>>> will
> > > > > > > > >>>>>>>>>> use
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> instead
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTxns.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that if we
> > > > > > > > >>>>>>> have
> > > > > > > > >>>>>>>>>> some
> > > > > > > > >>>>>>>>>>>> delay
> > > > > > > > >>>>>>>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> client
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> bump
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it could continue to send
> > > > > > > > >>>>>>>> epoch
> > > > > > > > >>>>>>>>>> 73
> > > > > > > > >>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>> those
> > > > > > > > >>>>>>>>>>>>>>>>>>>> records
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fenced.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Perhaps this is not an
> > > > > > > > >>>>>>> issue
> > > > > > > > >>>>>>>> if
> > > > > > > > >>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>> don't
> > > > > > > > >>>>>>>>>>>>>>>> allow
> > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> next
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> produce
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> go
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through before the EndTxn
> > > > > > > > >>>>>>>>> request
> > > > > > > > >>>>>>>>>>>>>>> returns.
> > > > > > > > >>>>>>>>>>>>>>>> I'm
> > > > > > > > >>>>>>>>>>>>>>>>>>>> also
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> thinking
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> about
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cases of
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure. I will need to
> > > > > > > > >>>>>>> think
> > > > > > > > >>>>>>>>> on
> > > > > > > > >>>>>>>>>>>> this a
> > > > > > > > >>>>>>>>>>>>>>> bit.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I wasn't sure if it was
> > > > > > > > >>>>>>> that
> > > > > > > > >>>>>>>>>>>> confusing.
> > > > > > > > >>>>>>>>>>>>>>> But
> > > > > > > > >>>>>>>>>>>>>>>> if
> > > > > > > > >>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> think it
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> is,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> investigate other ways.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure these are
> > > > > > > > >>>>>> the
> > > > > > > > >>>>>>>> same
> > > > > > > > >>>>>>>>>>>>>>> purgatories
> > > > > > > > >>>>>>>>>>>>>>>>>> since
> > > > > > > > >>>>>>>>>>>>>>>>>>>> one
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> is a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> produce
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory (I was planning
> > > > > > > > >>>>>>> on
> > > > > > > > >>>>>>>>>> using a
> > > > > > > > >>>>>>>>>>>>>>>> callback
> > > > > > > > >>>>>>>>>>>>>>>>>>>> rather
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> than
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the other is simply a
> > > > > > > > >>>>>>> request
> > > > > > > > >>>>>>>>> to
> > > > > > > > >>>>>>>>>>>> append
> > > > > > > > >>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>> log.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> Not
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> sure
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> structure here for
> > > > > > > > >>>>>>> ordering,
> > > > > > > > >>>>>>>>> but
> > > > > > > > >>>>>>>>>> my
> > > > > > > > >>>>>>>>>>>>>>>>>> understanding
> > > > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handle the write request
> > > > > > > > >>>>>>>> before
> > > > > > > > >>>>>>>>>> it
> > > > > > > > >>>>>>>>>>>> hears
> > > > > > > > >>>>>>>>>>>>>>>> back
> > > > > > > > >>>>>>>>>>>>>>>>>> from
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Txn
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Coordinator.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if I
> > > > > > > > >>>>>>>> misunderstood
> > > > > > > > >>>>>>>>>>>> something
> > > > > > > > >>>>>>>>>>>>>>> or
> > > > > > > > >>>>>>>>>>>>>>>>>>>> something
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> was
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> unclear.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 1, 2022 at
> > > > > > > > >>>>>>> 12:15
> > > > > > > > >>>>>>>> PM
> > > > > > > > >>>>>>>>>>>> Matthias
> > > > > > > > >>>>>>>>>>>>>>> J.
> > > > > > > > >>>>>>>>>>>>>>>>> Sax
> > > > > > > > >>>>>>>>>>>>>>>>>> <
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the details
> > > > > > > > >>>>>>>>> Justine!
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side change
> > > > > > > > >>>>>>> for
> > > > > > > > >>>>>>>> 2
> > > > > > > > >>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>> removing
> > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> addPartitions
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need to
> > > > > > > > >>>>>>> make
> > > > > > > > >>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>> from
> > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>> producer
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think I did not
> > > > > > > > >>>>>> express
> > > > > > > > >>>>>>>>> myself
> > > > > > > > >>>>>>>>>>>>>>> clearly. I
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> understand
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should) change the
> > > > > > > > >>>>>>> producer
> > > > > > > > >>>>>>>> to
> > > > > > > > >>>>>>>>>> not
> > > > > > > > >>>>>>>>>>>> send
> > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> longer. But I don't
> > > > > > > > >>>>>> thinks
> > > > > > > > >>>>>>>>> it's
> > > > > > > > >>>>>>>>>>>>>>> requirement
> > > > > > > > >>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> change
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> broker?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What I am trying to say
> > > > > > > > >>>>>>> is:
> > > > > > > > >>>>>>>>> as a
> > > > > > > > >>>>>>>>>>>>>>> safe-guard
> > > > > > > > >>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> improvement
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producers, the partition
> > > > > > > > >>>>>>>>> leader
> > > > > > > > >>>>>>>>>> can
> > > > > > > > >>>>>>>>>>>> just
> > > > > > > > >>>>>>>>>>>>>>>> send
> > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request to the
> > > > > > > > >>>>>>>> TX-coordinator
> > > > > > > > >>>>>>>>>> in any
> > > > > > > > >>>>>>>>>>>>>>> case
> > > > > > > > >>>>>>>>>>>>>>>> --
> > > > > > > > >>>>>>>>>>>>>>>>> if
> > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> old
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> correctly did send the
> > > > > > > > >>>>>>>>>>>> `addPartition`
> > > > > > > > >>>>>>>>>>>>>>>> request
> > > > > > > > >>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> already, the
> > > > > > > > >>>>>>> TX-coordinator
> > > > > > > > >>>>>>>>> can
> > > > > > > > >>>>>>>>>> just
> > > > > > > > >>>>>>>>>>>>>>>> "ignore"
> > > > > > > > >>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>> as
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> idempotent.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if the old producer has
> > > > > > > > >>>>>> a
> > > > > > > > >>>>>>>> bug
> > > > > > > > >>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>> did
> > > > > > > > >>>>>>>>>>>>>>>> forget
> > > > > > > > >>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>> sent
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `addPartition`
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request, we would now
> > > > > > > > >>>>>>> ensure
> > > > > > > > >>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>> partition
> > > > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> indeed
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> added
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX and thus fix a
> > > > > > > > >>>>>>> potential
> > > > > > > > >>>>>>>>>>>> producer bug
> > > > > > > > >>>>>>>>>>>>>>>>> (even
> > > > > > > > >>>>>>>>>>>>>>>>>>>> if we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> get
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fencing via the bump
> > > > > > > > >>>>>>> epoch).
> > > > > > > > >>>>>>>>> --
> > > > > > > > >>>>>>>>>> It
> > > > > > > > >>>>>>>>>>>>>>> seems to
> > > > > > > > >>>>>>>>>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>>>>> a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> good
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Or
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there a reason to not do
> > > > > > > > >>>>>>>> this?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is ongoing
> > > > > > > > >>>>>> =
> > > > > > > > >>>>>>>>>> partition
> > > > > > > > >>>>>>>>>>>> was
> > > > > > > > >>>>>>>>>>>>>>>> added
> > > > > > > > >>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> via
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn. We
> > > > > > > > >>>>>>>> check
> > > > > > > > >>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>> with
> > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this wasn't
> > > > > > > > >>>>>>>>>> sufficiently
> > > > > > > > >>>>>>>>>>>>>>>>> explained
> > > > > > > > >>>>>>>>>>>>>>>>>>>> here:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do what I propose
> > > > > > > > >>>>>> in
> > > > > > > > >>>>>>>>>> (20), we
> > > > > > > > >>>>>>>>>>>>>>> don't
> > > > > > > > >>>>>>>>>>>>>>>>>> really
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> need
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> make
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `DescribeTransaction`
> > > > > > > > >>>>>>> call,
> > > > > > > > >>>>>>>> as
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>> partition
> > > > > > > > >>>>>>>>>>>>>>>>>>>> leader
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> adds
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for older clients and we
> > > > > > > > >>>>>>> get
> > > > > > > > >>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>> check
> > > > > > > > >>>>>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>> free.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is that
> > > > > > > > >>>>>> if
> > > > > > > > >>>>>>>> any
> > > > > > > > >>>>>>>>>>>> messages
> > > > > > > > >>>>>>>>>>>>>>>>> somehow
> > > > > > > > >>>>>>>>>>>>>>>>>>>> come
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> before
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the producer,
> > > > > > > > >>>>>>> they
> > > > > > > > >>>>>>>>>> will be
> > > > > > > > >>>>>>>>>>>>>>>> fenced.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> However,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it can be
> > > > > > > > >>>>>>>>>> discussed
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree that we should
> > > > > > > > >>>>>>> have
> > > > > > > > >>>>>>>>>> epoch
> > > > > > > > >>>>>>>>>>>>>>> fencing.
> > > > > > > > >>>>>>>>>>>>>>>> My
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> question is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> different:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Assume we are at epoch
> > > > > > > > >>>>>> 73,
> > > > > > > > >>>>>>>> and
> > > > > > > > >>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>> have
> > > > > > > > >>>>>>>>>>>>>>> an
> > > > > > > > >>>>>>>>>>>>>>>>>> ongoing
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed. It seems
> > > > > > > > >>>>>>> natural
> > > > > > > > >>>>>>>> to
> > > > > > > > >>>>>>>>>>>> write the
> > > > > > > > >>>>>>>>>>>>>>>>>> "prepare
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> commit"
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> marker
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
> > > > > > > > >>>>>>> both
> > > > > > > > >>>>>>>>> with
> > > > > > > > >>>>>>>>>>>> epoch
> > > > > > > > >>>>>>>>>>>>>>> 73,
> > > > > > > > >>>>>>>>>>>>>>>>> too,
> > > > > > > > >>>>>>>>>>>>>>>>>>>> as
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> it
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> belongs
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current transaction. Of
> > > > > > > > >>>>>>>>> course,
> > > > > > > > >>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>> now
> > > > > > > > >>>>>>>>>>>>>>> also
> > > > > > > > >>>>>>>>>>>>>>>>>> bump
> > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> expect
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the next requests to
> > > > > > > > >>>>>> have
> > > > > > > > >>>>>>>>> epoch
> > > > > > > > >>>>>>>>>> 74,
> > > > > > > > >>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>> would
> > > > > > > > >>>>>>>>>>>>>>>>>>>> reject
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> an
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch 73, as the
> > > > > > > > >>>>>>>> corresponding
> > > > > > > > >>>>>>>>>> TX
> > > > > > > > >>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>> epoch
> > > > > > > > >>>>>>>>>>>>>>>>> 73
> > > > > > > > >>>>>>>>>>>>>>>>>>>> was
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> already
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems you propose to
> > > > > > > > >>>>>>>> write
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>> "prepare
> > > > > > > > >>>>>>>>>>>>>>>>>> commit
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> marker"
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
> > > > > > > > >>>>>>> with
> > > > > > > > >>>>>>>>>> epoch 74
> > > > > > > > >>>>>>>>>>>>>>>> though,
> > > > > > > > >>>>>>>>>>>>>>>>>> what
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> work,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems confusing. Is
> > > > > > > > >>>>>> there
> > > > > > > > >>>>>>> a
> > > > > > > > >>>>>>>>>> reason
> > > > > > > > >>>>>>>>>>>> why
> > > > > > > > >>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>> would
> > > > > > > > >>>>>>>>>>>>>>>>>>>> use
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> bumped
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 74
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of the current
> > > > > > > > >>>>>>> epoch
> > > > > > > > >>>>>>>>> 73?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are checking if
> > > > > > > > >>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>> transaction is
> > > > > > > > >>>>>>>>>>>>>>>>>>>> ongoing,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the leader
> > > > > > > > >>>>>>>>> partition
> > > > > > > > >>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for this
> > > > > > > > >>>>>>>>>> message to
> > > > > > > > >>>>>>>>>>>> come
> > > > > > > > >>>>>>>>>>>>>>>>> back,
> > > > > > > > >>>>>>>>>>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> theory
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
> > > > > > > > >>>>>> that
> > > > > > > > >>>>>>>>> would
> > > > > > > > >>>>>>>>>>>> make the
> > > > > > > > >>>>>>>>>>>>>>>>>> original
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> result
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out of
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why we
> > > > > > > > >>>>>> can
> > > > > > > > >>>>>>>>> check
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>> leader
> > > > > > > > >>>>>>>>>>>>>>>>>> state
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> before
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks. Got it.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, is this really
> > > > > > > > >>>>>> an
> > > > > > > > >>>>>>>>>> issue?
> > > > > > > > >>>>>>>>>>>> We put
> > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>> produce
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory, so how could
> > > > > > > > >>>>>> we
> > > > > > > > >>>>>>>>>> process
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> first?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Don't we need to put the
> > > > > > > > >>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> into
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> too,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for this case, and
> > > > > > > > >>>>>> process
> > > > > > > > >>>>>>>>> both
> > > > > > > > >>>>>>>>>>>> request
> > > > > > > > >>>>>>>>>>>>>>>>>> in-order?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> (Again,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> my
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> knowledge is limited and
> > > > > > > > >>>>>>>> maybe
> > > > > > > > >>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>> don't
> > > > > > > > >>>>>>>>>>>>>>>>>> maintain
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> order
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case, what seems to be
> > > > > > > > >>>>>> an
> > > > > > > > >>>>>>>>> issue
> > > > > > > > >>>>>>>>>>>> IMHO,
> > > > > > > > >>>>>>>>>>>>>>> and I
> > > > > > > > >>>>>>>>>>>>>>>>> am
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> wondering
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> changing
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request handling to
> > > > > > > > >>>>>>> preserve
> > > > > > > > >>>>>>>>>> order
> > > > > > > > >>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>> case
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> might be
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> cleaner
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution?)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/30/22 3:28 PM,
> > > > > > > > >>>>>> Artem
> > > > > > > > >>>>>>>>>> Livshits
> > > > > > > > >>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think the interesting
> > > > > > > > >>>>>>>> part
> > > > > > > > >>>>>>>>> is
> > > > > > > > >>>>>>>>>>>> not in
> > > > > > > > >>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>> logic
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> (because
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> tries
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> figure out when
> > > > > > > > >>>>>>>>>>>> UNKNOWN_PRODUCER_ID is
> > > > > > > > >>>>>>>>>>>>>>>>>> retriable
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> it's
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retryable,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's definitely not
> > > > > > > > >>>>>>> fatal),
> > > > > > > > >>>>>>>>> but
> > > > > > > > >>>>>>>>>>>> what
> > > > > > > > >>>>>>>>>>>>>>>> happens
> > > > > > > > >>>>>>>>>>>>>>>>>>>> when
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> logic
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'true' and falls
> > > > > > > > >>>>>> through.
> > > > > > > > >>>>>>>> In
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>> old
> > > > > > > > >>>>>>>>>>>>>>>>> clients
> > > > > > > > >>>>>>>>>>>>>>>>>> it
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> seems
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> keep the behavior in
> > > > > > > > >>>>>> the
> > > > > > > > >>>>>>>> new
> > > > > > > > >>>>>>>>>>>> clients,
> > > > > > > > >>>>>>>>>>>>>>> I'd
> > > > > > > > >>>>>>>>>>>>>>>>>>>> expect it
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> well.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 29, 2022 at
> > > > > > > > >>>>>>>> 11:57
> > > > > > > > >>>>>>>>>> AM
> > > > > > > > >>>>>>>>>>>>>>> Justine
> > > > > > > > >>>>>>>>>>>>>>>>>> Olshan
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>> <jolshan@confluent.io.invalid
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Artem and Jeff,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
> > > > > > > > >>>>>> look
> > > > > > > > >>>>>>>> and
> > > > > > > > >>>>>>>>>>>> sorry for
> > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>> slow
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> response.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You both mentioned the
> > > > > > > > >>>>>>>>> change
> > > > > > > > >>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>> handle
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear — this error
> > > > > > > > >>>>>> code
> > > > > > > > >>>>>>>> will
> > > > > > > > >>>>>>>>>> only
> > > > > > > > >>>>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>> sent
> > > > > > > > >>>>>>>>>>>>>>>>>> again
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> when
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> client's
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version is high enough
> > > > > > > > >>>>>>> to
> > > > > > > > >>>>>>>>>> ensure
> > > > > > > > >>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>> handle
> > > > > > > > >>>>>>>>>>>>>>>>> it
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> correctly.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The current (Java)
> > > > > > > > >>>>>>> client
> > > > > > > > >>>>>>>>>> handles
> > > > > > > > >>>>>>>>>>>>>>> this by
> > > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> following
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> (somewhat
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> long)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code snippet:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // An
> > > > > > > > >>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > >>>>>>>>>> means
> > > > > > > > >>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>> have
> > > > > > > > >>>>>>>>>>>>>>>>>>>> lost
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker. Depending on
> > > > > > > > >>>>>> the
> > > > > > > > >>>>>>>> log
> > > > > > > > >>>>>>>>>> start
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // offset, we may want
> > > > > > > > >>>>>>> to
> > > > > > > > >>>>>>>>>> retry
> > > > > > > > >>>>>>>>>>>>>>> these, as
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> described
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> each
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> case
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below. If
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> none of those apply,
> > > > > > > > >>>>>>> then
> > > > > > > > >>>>>>>>> for
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // idempotent
> > > > > > > > >>>>>> producer,
> > > > > > > > >>>>>>> we
> > > > > > > > >>>>>>>>>> will
> > > > > > > > >>>>>>>>>>>>>>> locally
> > > > > > > > >>>>>>>>>>>>>>>>> bump
> > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> reset
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence numbers of
> > > > > > > > >>>>>>>>> in-flight
> > > > > > > > >>>>>>>>>>>> batches
> > > > > > > > >>>>>>>>>>>>>>>> from
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // sequence 0, then
> > > > > > > > >>>>>>> retry
> > > > > > > > >>>>>>>>> the
> > > > > > > > >>>>>>>>>>>> failed
> > > > > > > > >>>>>>>>>>>>>>>> batch,
> > > > > > > > >>>>>>>>>>>>>>>>>>>> which
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> should
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> now
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> succeed.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the transactional
> > > > > > > > >>>>>>>> producer,
> > > > > > > > >>>>>>>>>> allow
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // batch to fail. When
> > > > > > > > >>>>>>>>>> processing
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>> failed
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> batch,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transition
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an abortable error and
> > > > > > > > >>>>>>>> set a
> > > > > > > > >>>>>>>>>> flag
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // indicating that we
> > > > > > > > >>>>>>> need
> > > > > > > > >>>>>>>>> to
> > > > > > > > >>>>>>>>>>>> bump the
> > > > > > > > >>>>>>>>>>>>>>>>> epoch
> > > > > > > > >>>>>>>>>>>>>>>>>>>> (if
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> supported
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> by
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker).
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if (error ==
> > > > > > > > >>>>>>>>>>>>>>>> Errors.*UNKNOWN_PRODUCER_ID*)
> > > > > > > > >>>>>>>>>>>>>>>>> {
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > > > > > > > >>>>>>>>>> (response.logStartOffset
> > > > > > > > >>>>>>>>>>>> ==
> > > > > > > > >>>>>>>>>>>>>>> -1)
> > > > > > > > >>>>>>>>>>>>>>>> {
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // We don't
> > > > > > > > >>>>>>> know
> > > > > > > > >>>>>>>>>> the log
> > > > > > > > >>>>>>>>>>>>>>> start
> > > > > > > > >>>>>>>>>>>>>>>>>> offset
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> with
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> response.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the request
> > > > > > > > >>>>>>>> until
> > > > > > > > >>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>> get
> > > > > > > > >>>>>>>>>>>>>>> it.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // The
> > > > > > > > >>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > >>>>>>>>>>>>>>>> error
> > > > > > > > >>>>>>>>>>>>>>>>>> code
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> was
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> added
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> along
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProduceResponse which
> > > > > > > > >>>>>>>>>> includes the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              //
> > > > > > > > >>>>>>>> logStartOffset.
> > > > > > > > >>>>>>>>>> So
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>> '-1'
> > > > > > > > >>>>>>>>>>>>>>>>>>>> sentinel
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> backward
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatibility.
> > > > > > > > >>>>>> Instead,
> > > > > > > > >>>>>>> it
> > > > > > > > >>>>>>>>> is
> > > > > > > > >>>>>>>>>>>> possible
> > > > > > > > >>>>>>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // a broker
> > > > > > > > >>>>>> to
> > > > > > > > >>>>>>>> not
> > > > > > > > >>>>>>>>>> know
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> logStartOffset at
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> when
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returning
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response because
> > > > > > > > >>>>>> the
> > > > > > > > >>>>>>>>>> partition
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // may have
> > > > > > > > >>>>>>>> moved
> > > > > > > > >>>>>>>>>> away
> > > > > > > > >>>>>>>>>>>> from
> > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>> broker
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> from
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error was
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initially raised to
> > > > > > > > >>>>>> the
> > > > > > > > >>>>>>>> time
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // response
> > > > > > > > >>>>>>> was
> > > > > > > > >>>>>>>>>> being
> > > > > > > > >>>>>>>>>>>>>>>>> constructed.
> > > > > > > > >>>>>>>>>>>>>>>>>> In
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> these
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> cases,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retry the request: we
> > > > > > > > >>>>>>> are
> > > > > > > > >>>>>>>>>>>> guaranteed
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // to
> > > > > > > > >>>>>>> eventually
> > > > > > > > >>>>>>>>>> get a
> > > > > > > > >>>>>>>>>>>>>>>>>> logStartOffset
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> once
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> things
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> settle
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> down.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return
> true;
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > > > > > > > >>>>>>>>>>>>>>> (batch.sequenceHasBeenReset()) {
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // When the
> > > > > > > > >>>>>>>> first
> > > > > > > > >>>>>>>>>>>> inflight
> > > > > > > > >>>>>>>>>>>>>>>> batch
> > > > > > > > >>>>>>>>>>>>>>>>>>>> fails
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> due to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> truncation
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> then the sequences of
> > > > > > > > >>>>>>> all
> > > > > > > > >>>>>>>>> the
> > > > > > > > >>>>>>>>>>>> other
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // in
> flight
> > > > > > > > >>>>>>>>> batches
> > > > > > > > >>>>>>>>>>>> would
> > > > > > > > >>>>>>>>>>>>>>> have
> > > > > > > > >>>>>>>>>>>>>>>>>> been
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> restarted
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> from
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> beginning.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, when those
> > > > > > > > >>>>>>>>> responses
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // come
> back
> > > > > > > > >>>>>>>> from
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>> broker,
> > > > > > > > >>>>>>>>>>>>>>>>> they
> > > > > > > > >>>>>>>>>>>>>>>>>>>> would
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> also
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> come
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > >>>>>>> error.
> > > > > > > > >>>>>>>>> In
> > > > > > > > >>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>> case,
> > > > > > > > >>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>> should
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // reset
> the
> > > > > > > > >>>>>>>>>> sequence
> > > > > > > > >>>>>>>>>>>>>>> numbers
> > > > > > > > >>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> beginning.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return
> true;
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          } else if
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>> (lastAckedOffset(batch.topicPartition).orElse(
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> response.logStartOffset) {
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // The head
> > > > > > > > >>>>>> of
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>> log
> > > > > > > > >>>>>>>>>>>> has
> > > > > > > > >>>>>>>>>>>>>>> been
> > > > > > > > >>>>>>>>>>>>>>>>>>>> removed,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> probably
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> due
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retention time
> > > > > > > > >>>>>> elapsing.
> > > > > > > > >>>>>>>> In
> > > > > > > > >>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>> case,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // we
> expect
> > > > > > > > >>>>>>> to
> > > > > > > > >>>>>>>>>> lose the
> > > > > > > > >>>>>>>>>>>>>>>> producer
> > > > > > > > >>>>>>>>>>>>>>>>>>>> state.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> For
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer, reset the
> > > > > > > > >>>>>>>>> sequences
> > > > > > > > >>>>>>>>>> of
> > > > > > > > >>>>>>>>>>>> all
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // inflight
> > > > > > > > >>>>>>>>> batches
> > > > > > > > >>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>> from
> > > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> beginning
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> retry
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the transaction
> > > > > > > > >>>>>>> does
> > > > > > > > >>>>>>>>> not
> > > > > > > > >>>>>>>>>>>> need to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // be
> > > > > > > > >>>>>> aborted.
> > > > > > > > >>>>>>>> For
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>> idempotent
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> producer,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> bump
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> avoid
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reusing (sequence,
> > > > > > > > >>>>>>> epoch)
> > > > > > > > >>>>>>>>>> pairs
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              if
> > > > > > > > >>>>>>>>>> (isTransactional()) {
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>
> > > > > > > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>> this.producerIdAndEpoch);
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              } else {
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>
> > > requestEpochBumpForPartition(batch.topicPartition);
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              }
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return
> true;
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > > > > > > > >>>>>>>>> (!isTransactional())
> > > > > > > > >>>>>>>>>> {
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // For the
> > > > > > > > >>>>>>>>>> idempotent
> > > > > > > > >>>>>>>>>>>>>>> producer,
> > > > > > > > >>>>>>>>>>>>>>>>>>>> always
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> retry
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors. If the batch
> > > > > > > > >>>>>> has
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>>>> current
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // producer
> > > > > > > > >>>>>> ID
> > > > > > > > >>>>>>>> and
> > > > > > > > >>>>>>>>>>>> epoch,
> > > > > > > > >>>>>>>>>>>>>>>>> request a
> > > > > > > > >>>>>>>>>>>>>>>>>>>> bump
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Otherwise
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the
> > > > > > > > >>>>>> produce.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>
> > requestEpochBumpForPartition(batch.topicPartition);
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return
> true;
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I was considering
> > > > > > > > >>>>>>> keeping
> > > > > > > > >>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>> behavior —
> > > > > > > > >>>>>>>>>>>>>>>>> but
> > > > > > > > >>>>>>>>>>>>>>>>>>>> am
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> open
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simplifying
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We are leaving changes
> > > > > > > > >>>>>>> to
> > > > > > > > >>>>>>>>>> older
> > > > > > > > >>>>>>>>>>>>>>> clients
> > > > > > > > >>>>>>>>>>>>>>>> off
> > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> table
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> here
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> since
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> caused
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many issues for
> > > > > > > > >>>>>> clients
> > > > > > > > >>>>>>> in
> > > > > > > > >>>>>>>>> the
> > > > > > > > >>>>>>>>>>>> past.
> > > > > > > > >>>>>>>>>>>>>>>>>> Previously
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> was
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we didn't have the
> > > > > > > > >>>>>>>>> mechanisms
> > > > > > > > >>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>>> place to
> > > > > > > > >>>>>>>>>>>>>>>>>>>> detect
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> when
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> was
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> legitimate
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case vs some bug or
> > > > > > > > >>>>>> gap
> > > > > > > > >>>>>>> in
> > > > > > > > >>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>> protocol.
> > > > > > > > >>>>>>>>>>>>>>>>>>>> Ensuring
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> each
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> its
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> own epoch should close
> > > > > > > > >>>>>>>> this
> > > > > > > > >>>>>>>>>> gap.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> And to address Jeff's
> > > > > > > > >>>>>>>> second
> > > > > > > > >>>>>>>>>>>> point:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *does the typical
> > > > > > > > >>>>>>> produce
> > > > > > > > >>>>>>>>>> request
> > > > > > > > >>>>>>>>>>>> path
> > > > > > > > >>>>>>>>>>>>>>>>> append
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> records
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> local
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> log
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along*
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *with the
> > > > > > > > >>>>>>>>>> currentTxnFirstOffset
> > > > > > > > >>>>>>>>>>>>>>>>> information?
> > > > > > > > >>>>>>>>>>>>>>>>>> I
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> would
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> like
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand*
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *when the field is
> > > > > > > > >>>>>>> written
> > > > > > > > >>>>>>>>> to
> > > > > > > > >>>>>>>>>>>> disk.*
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, the first produce
> > > > > > > > >>>>>>>>> request
> > > > > > > > >>>>>>>>>>>>>>> populates
> > > > > > > > >>>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> field
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> writes
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> offset
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as part of the record
> > > > > > > > >>>>>>>> batch
> > > > > > > > >>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>> also
> > > > > > > > >>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> producer
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> state
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> snapshot.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we reload the records
> > > > > > > > >>>>>> on
> > > > > > > > >>>>>>>>>> restart
> > > > > > > > >>>>>>>>>>>>>>> and/or
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> reassignment,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> repopulate
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> field with the
> > > > > > > > >>>>>> snapshot
> > > > > > > > >>>>>>>> from
> > > > > > > > >>>>>>>>>> disk
> > > > > > > > >>>>>>>>>>>>>>> along
> > > > > > > > >>>>>>>>>>>>>>>>> with
> > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> rest
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if there
> > > > > > > > >>>>>> are
> > > > > > > > >>>>>>>>>> further
> > > > > > > > >>>>>>>>>>>>>>> comments
> > > > > > > > >>>>>>>>>>>>>>>>>>>> and/or
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> questions.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > > > > > > > >>>>>> at
> > > > > > > > >>>>>>>> 9:00
> > > > > > > > >>>>>>>>>> PM
> > > > > > > > >>>>>>>>>>>> Jeff
> > > > > > > > >>>>>>>>>>>>>>> Kim
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>> <jeff.kim@confluent.io.invalid
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP! I
> > > > > > > > >>>>>>>> have
> > > > > > > > >>>>>>>>>> two
> > > > > > > > >>>>>>>>>>>>>>>> questions:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1) For new clients,
> > > > > > > > >>>>>> we
> > > > > > > > >>>>>>>> can
> > > > > > > > >>>>>>>>>> once
> > > > > > > > >>>>>>>>>>>> again
> > > > > > > > >>>>>>>>>>>>>>>>> return
> > > > > > > > >>>>>>>>>>>>>>>>>>>> an
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> error
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for sequences
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that are non-zero
> > > > > > > > >>>>>> when
> > > > > > > > >>>>>>>>> there
> > > > > > > > >>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>> no
> > > > > > > > >>>>>>>>>>>>>>>>> producer
> > > > > > > > >>>>>>>>>>>>>>>>>>>> state
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> present
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> on
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This will indicate we
> > > > > > > > >>>>>>>>> missed
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>> 0
> > > > > > > > >>>>>>>>>>>>>>>>> sequence
> > > > > > > > >>>>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> yet
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> want
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the log.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would like to
> > > > > > > > >>>>>>>> understand
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>> current
> > > > > > > > >>>>>>>>>>>>>>>>>>>> behavior
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> handle
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and if there are any
> > > > > > > > >>>>>>>>> changes
> > > > > > > > >>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>> are
> > > > > > > > >>>>>>>>>>>>>>>>> making.
> > > > > > > > >>>>>>>>>>>>>>>>>>>> Maybe
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> I'm
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> missing
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but we would want to
> > > > > > > > >>>>>>>>> identify
> > > > > > > > >>>>>>>>>>>>>>> whether we
> > > > > > > > >>>>>>>>>>>>>>>>>>>> missed
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the 0
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> sequence
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients, no?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2) Upon returning
> > > > > > > > >>>>>> from
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> coordinator, we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> set
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as ongoing on the
> > > > > > > > >>>>>>> leader
> > > > > > > > >>>>>>>> by
> > > > > > > > >>>>>>>>>>>>>>> populating
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> currentTxnFirstOffset
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through the typical
> > > > > > > > >>>>>>>> produce
> > > > > > > > >>>>>>>>>>>> request
> > > > > > > > >>>>>>>>>>>>>>>>>> handling.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does the typical
> > > > > > > > >>>>>>> produce
> > > > > > > > >>>>>>>>>> request
> > > > > > > > >>>>>>>>>>>> path
> > > > > > > > >>>>>>>>>>>>>>>>> append
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> records
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> local
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> log
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> > > > > > > > >>>>>>>>>> currentTxnFirstOffset
> > > > > > > > >>>>>>>>>>>>>>>>> information?
> > > > > > > > >>>>>>>>>>>>>>>>>> I
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> would
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> like
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when the field is
> > > > > > > > >>>>>>> written
> > > > > > > > >>>>>>>>> to
> > > > > > > > >>>>>>>>>>>> disk.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeff
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > > > > > > > >>>>>> at
> > > > > > > > >>>>>>>>> 4:44
> > > > > > > > >>>>>>>>>> PM
> > > > > > > > >>>>>>>>>>>> Artem
> > > > > > > > >>>>>>>>>>>>>>>>>> Livshits
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
> > > > > > > > >>>>>>> alivshits@confluent.io
> > > > > > > > >>>>>>>>>> .invalid>
> > > > > > > > >>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for the
> > > > > > > > >>>>>> KIP.
> > > > > > > > >>>>>>>> I
> > > > > > > > >>>>>>>>>> have
> > > > > > > > >>>>>>>>>>>> one
> > > > > > > > >>>>>>>>>>>>>>>>>> question.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 5) For new clients,
> > > > > > > > >>>>>> we
> > > > > > > > >>>>>>>> can
> > > > > > > > >>>>>>>>>> once
> > > > > > > > >>>>>>>>>>>>>>> again
> > > > > > > > >>>>>>>>>>>>>>>>>> return
> > > > > > > > >>>>>>>>>>>>>>>>>>>> an
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> error
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe we had
> > > > > > > > >>>>>>>> problems
> > > > > > > > >>>>>>>>>> in the
> > > > > > > > >>>>>>>>>>>>>>> past
> > > > > > > > >>>>>>>>>>>>>>>>> with
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> returning
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because it was
> > > > > > > > >>>>>>>> considered
> > > > > > > > >>>>>>>>>> fatal
> > > > > > > > >>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>> required
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> client
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> restart.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> It
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good to spell out
> > > > > > > > >>>>>> the
> > > > > > > > >>>>>>>> new
> > > > > > > > >>>>>>>>>> client
> > > > > > > > >>>>>>>>>>>>>>>> behavior
> > > > > > > > >>>>>>>>>>>>>>>>>>>> when
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> it
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> receives
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > > > > > > > >>>>>>> at
> > > > > > > > >>>>>>>>>> 10:00 AM
> > > > > > > > >>>>>>>>>>>>>>>> Justine
> > > > > > > > >>>>>>>>>>>>>>>>>>>> Olshan
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > > > > >>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
> > > > > > > > >>>>>>>> look
> > > > > > > > >>>>>>>>>>>> Matthias.
> > > > > > > > >>>>>>>>>>>>>>>> I've
> > > > > > > > >>>>>>>>>>>>>>>>>>>> tried
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> answer
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> your
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> questions
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 10)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Right — so the
> > > > > > > > >>>>>>> hanging
> > > > > > > > >>>>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>>> only
> > > > > > > > >>>>>>>>>>>>>>>>>>>> occurs
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> when
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> come in and the
> > > > > > > > >>>>>>>> partition
> > > > > > > > >>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>> never
> > > > > > > > >>>>>>>>>>>>>>>> added
> > > > > > > > >>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>> a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> again.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never add the
> > > > > > > > >>>>>>> partition
> > > > > > > > >>>>>>>>> to
> > > > > > > > >>>>>>>>>> a
> > > > > > > > >>>>>>>>>>>>>>>>> transaction,
> > > > > > > > >>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> will
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> never
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> marker
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never advance the
> > > > > > > > >>>>>>> LSO.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do end up
> > > > > > > > >>>>>>> adding
> > > > > > > > >>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>> partition
> > > > > > > > >>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (I
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suppose
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happen before or
> > > > > > > > >>>>>>> after
> > > > > > > > >>>>>>>>> the
> > > > > > > > >>>>>>>>>> late
> > > > > > > > >>>>>>>>>>>>>>>> message
> > > > > > > > >>>>>>>>>>>>>>>>>>>> comes
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> in)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> then
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> include
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late message in the
> > > > > > > > >>>>>>>> next
> > > > > > > > >>>>>>>>>>>>>>> (incorrect)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> transaction.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> So perhaps it is
> > > > > > > > >>>>>>>> clearer
> > > > > > > > >>>>>>>>> to
> > > > > > > > >>>>>>>>>>>> make
> > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> distinction
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> between
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> eventually get
> > > > > > > > >>>>>> added
> > > > > > > > >>>>>>> to
> > > > > > > > >>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>>>>> (but
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> wrong
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> one)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> or
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that never get
> > > > > > > > >>>>>> added
> > > > > > > > >>>>>>>> and
> > > > > > > > >>>>>>>>>> become
> > > > > > > > >>>>>>>>>>>>>>>> hanging.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side
> > > > > > > > >>>>>>> change
> > > > > > > > >>>>>>>>> for
> > > > > > > > >>>>>>>>>> 2 is
> > > > > > > > >>>>>>>>>>>>>>>> removing
> > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> addPartitions
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need
> > > > > > > > >>>>>>> to
> > > > > > > > >>>>>>>>> make
> > > > > > > > >>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>> from
> > > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In my opinion, the
> > > > > > > > >>>>>>>> issue
> > > > > > > > >>>>>>>>>> with
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> call
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is that we don't
> > > > > > > > >>>>>> have
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>> epoch
> > > > > > > > >>>>>>>>>>>>>>> bump,
> > > > > > > > >>>>>>>>>>>>>>>> so
> > > > > > > > >>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> don't
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> know
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> belongs to the
> > > > > > > > >>>>>>> previous
> > > > > > > > >>>>>>>>>>>>>>> transaction or
> > > > > > > > >>>>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> one.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> We
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition has been
> > > > > > > > >>>>>>>> added
> > > > > > > > >>>>>>>>> to
> > > > > > > > >>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>> transaction.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> Of
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> course,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> means
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't completely
> > > > > > > > >>>>>>> cover
> > > > > > > > >>>>>>>>> the
> > > > > > > > >>>>>>>>>> case
> > > > > > > > >>>>>>>>>>>>>>> where
> > > > > > > > >>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>> have a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> really
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have added the
> > > > > > > > >>>>>>>> partition
> > > > > > > > >>>>>>>>> to
> > > > > > > > >>>>>>>>>>>> the new
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> transaction,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> that's
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something we will
> > > > > > > > >>>>>>> need
> > > > > > > > >>>>>>>>> the
> > > > > > > > >>>>>>>>>> new
> > > > > > > > >>>>>>>>>>>>>>> clients
> > > > > > > > >>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> cover.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is
> > > > > > > > >>>>>>> ongoing
> > > > > > > > >>>>>>>> =
> > > > > > > > >>>>>>>>>>>> partition
> > > > > > > > >>>>>>>>>>>>>>> was
> > > > > > > > >>>>>>>>>>>>>>>>>>>> added to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> via
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn.
> > > > > > > > >>>>>>> We
> > > > > > > > >>>>>>>>>> check
> > > > > > > > >>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>> with
> > > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this
> > > > > > > > >>>>>>> wasn't
> > > > > > > > >>>>>>>>>>>> sufficiently
> > > > > > > > >>>>>>>>>>>>>>>>>>>> explained
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> here:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>
> > > > > > > > >>>>
> > > > > > > > >>
> > > > > > >
> > > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is
> > > > > > > > >>>>>> that
> > > > > > > > >>>>>>>> if
> > > > > > > > >>>>>>>>>> any
> > > > > > > > >>>>>>>>>>>>>>> messages
> > > > > > > > >>>>>>>>>>>>>>>>>>>> somehow
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> come
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> before
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the
> > > > > > > > >>>>>>> producer,
> > > > > > > > >>>>>>>>> they
> > > > > > > > >>>>>>>>>>>> will be
> > > > > > > > >>>>>>>>>>>>>>>>>> fenced.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> However,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it
> > > > > > > > >>>>>> can
> > > > > > > > >>>>>>> be
> > > > > > > > >>>>>>>>>>>> discussed
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 50)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It should be
> > > > > > > > >>>>>>>> synchronous
> > > > > > > > >>>>>>>>>>>> because
> > > > > > > > >>>>>>>>>>>>>>> if we
> > > > > > > > >>>>>>>>>>>>>>>>>> have
> > > > > > > > >>>>>>>>>>>>>>>>>>>> an
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> event
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (ie,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> an
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes us to need
> > > > > > > > >>>>>> to
> > > > > > > > >>>>>>>>> abort
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>> transaction,
> > > > > > > > >>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> need
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions to send
> > > > > > > > >>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>> markers
> > > > > > > > >>>>>>>>>>>>>>>>> to.
> > > > > > > > >>>>>>>>>>>>>>>>>>>> We
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> know
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we added them to
> > > > > > > > >>>>>> the
> > > > > > > > >>>>>>>>>>>> coordinator
> > > > > > > > >>>>>>>>>>>>>>> via
> > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Previously we have
> > > > > > > > >>>>>>> had
> > > > > > > > >>>>>>>>>>>> asynchronous
> > > > > > > > >>>>>>>>>>>>>>>>> calls
> > > > > > > > >>>>>>>>>>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> past
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (ie,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit markers when
> > > > > > > > >>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>> transaction is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> completed)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> often
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes confusion as
> > > > > > > > >>>>>>> we
> > > > > > > > >>>>>>>>>> need to
> > > > > > > > >>>>>>>>>>>> wait
> > > > > > > > >>>>>>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>> some
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> operations
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complete.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing commit
> > > > > > > > >>>>>>> markers
> > > > > > > > >>>>>>>>>> case,
> > > > > > > > >>>>>>>>>>>>>>> clients
> > > > > > > > >>>>>>>>>>>>>>>>> often
> > > > > > > > >>>>>>>>>>>>>>>>>>>> see
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>> CONCURRENT_TRANSACTIONs
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error messages and
> > > > > > > > >>>>>>> that
> > > > > > > > >>>>>>>>>> can be
> > > > > > > > >>>>>>>>>>>>>>>>> confusing.
> > > > > > > > >>>>>>>>>>>>>>>>>>>> For
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> reason,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> may
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simpler to just
> > > > > > > > >>>>>> have
> > > > > > > > >>>>>>>>>>>> synchronous
> > > > > > > > >>>>>>>>>>>>>>>> calls —
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> especially
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> block
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some operation's
> > > > > > > > >>>>>>>>> completion
> > > > > > > > >>>>>>>>>>>> anyway
> > > > > > > > >>>>>>>>>>>>>>>>> before
> > > > > > > > >>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> can
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> start
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> next
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. And
> > > > > > > > >>>>>>> yes, I
> > > > > > > > >>>>>>>>>> meant
> > > > > > > > >>>>>>>>>>>>>>>>>> coordinator. I
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> will
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> fix
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> that.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are
> > > > > > > > >>>>>> checking
> > > > > > > > >>>>>>> if
> > > > > > > > >>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> ongoing,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the
> > > > > > > > >>>>>> leader
> > > > > > > > >>>>>>>>>> partition
> > > > > > > > >>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for
> > > > > > > > >>>>>>> this
> > > > > > > > >>>>>>>>>>>> message to
> > > > > > > > >>>>>>>>>>>>>>>> come
> > > > > > > > >>>>>>>>>>>>>>>>>>>> back,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> theory
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
> > > > > > > > >>>>>>>> that
> > > > > > > > >>>>>>>>>> would
> > > > > > > > >>>>>>>>>>>> make
> > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> original
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> result
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why
> > > > > > > > >>>>>> we
> > > > > > > > >>>>>>>> can
> > > > > > > > >>>>>>>>>> check
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>> leader
> > > > > > > > >>>>>>>>>>>>>>>>>>>> state
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> before
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm happy to update
> > > > > > > > >>>>>>> the
> > > > > > > > >>>>>>>>>> KIP if
> > > > > > > > >>>>>>>>>>>>>>> some of
> > > > > > > > >>>>>>>>>>>>>>>>>> these
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> things
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> were
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Nov 21,
> > > > > > > > >>>>>> 2022
> > > > > > > > >>>>>>> at
> > > > > > > > >>>>>>>>>> 7:11 PM
> > > > > > > > >>>>>>>>>>>>>>>> Matthias
> > > > > > > > >>>>>>>>>>>>>>>>>> J.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> Sax <
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the
> > > > > > > > >>>>>> KIP.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Couple of
> > > > > > > > >>>>>>>> clarification
> > > > > > > > >>>>>>>>>>>> questions
> > > > > > > > >>>>>>>>>>>>>>> (I
> > > > > > > > >>>>>>>>>>>>>>>> am
> > > > > > > > >>>>>>>>>>>>>>>>>>>> not a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> broker
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> expert
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> do
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some question are
> > > > > > > > >>>>>>>>> obvious
> > > > > > > > >>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>> others,
> > > > > > > > >>>>>>>>>>>>>>>>> but
> > > > > > > > >>>>>>>>>>>>>>>>>>>> not
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> me
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> my
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lack
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker knowledge).
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (10)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What happens if
> > > > > > > > >>>>>> the
> > > > > > > > >>>>>>>>>> message
> > > > > > > > >>>>>>>>>>>> come
> > > > > > > > >>>>>>>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>>>>>> before
> > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> next
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request? It seems
> > > > > > > > >>>>>>> the
> > > > > > > > >>>>>>>>>> broker
> > > > > > > > >>>>>>>>>>>>>>> hosting
> > > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>> data
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anything about it
> > > > > > > > >>>>>>> and
> > > > > > > > >>>>>>>>>> append
> > > > > > > > >>>>>>>>>>>> it to
> > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> partition,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> too?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> What
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> difference between
> > > > > > > > >>>>>>>> both
> > > > > > > > >>>>>>>>>> cases?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, it seems a
> > > > > > > > >>>>>> TX
> > > > > > > > >>>>>>>>> would
> > > > > > > > >>>>>>>>>> only
> > > > > > > > >>>>>>>>>>>>>>> hang,
> > > > > > > > >>>>>>>>>>>>>>>> if
> > > > > > > > >>>>>>>>>>>>>>>>>>>> there
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> is no
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> following
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> either committer
> > > > > > > > >>>>>> or
> > > > > > > > >>>>>>>>>> aborted?
> > > > > > > > >>>>>>>>>>>> Thus,
> > > > > > > > >>>>>>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> case
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> above,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> TX
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> might
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually not hang
> > > > > > > > >>>>>>> (of
> > > > > > > > >>>>>>>>>> course,
> > > > > > > > >>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>> might
> > > > > > > > >>>>>>>>>>>>>>>>>> get
> > > > > > > > >>>>>>>>>>>>>>>>>>>> an
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> EOS
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> violation
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX was aborted and
> > > > > > > > >>>>>>> the
> > > > > > > > >>>>>>>>>> second
> > > > > > > > >>>>>>>>>>>>>>>>> committed,
> > > > > > > > >>>>>>>>>>>>>>>>>> or
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> other
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> way
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around).
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (20)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Of course, 1 and
> > > > > > > > >>>>>> 2
> > > > > > > > >>>>>>>>>> require
> > > > > > > > >>>>>>>>>>>>>>>> client-side
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> changes, so
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> those approaches
> > > > > > > > >>>>>>> won’t
> > > > > > > > >>>>>>>>>> apply.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For (1) I
> > > > > > > > >>>>>> understand
> > > > > > > > >>>>>>>>> why a
> > > > > > > > >>>>>>>>>>>> client
> > > > > > > > >>>>>>>>>>>>>>>>> change
> > > > > > > > >>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> necessary,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we need a client
> > > > > > > > >>>>>>>> change
> > > > > > > > >>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>> (2).
> > > > > > > > >>>>>>>>>>>>>>> Can
> > > > > > > > >>>>>>>>>>>>>>>>> you
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> --
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Later
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> explain
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that we should
> > > > > > > > >>>>>> send
> > > > > > > > >>>>>>> a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> DescribeTransactionRequest,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> but I
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> am
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Can't we not just
> > > > > > > > >>>>>> do
> > > > > > > > >>>>>>>> an
> > > > > > > > >>>>>>>>>>>> implicit
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> AddPartiitonToTx,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> too?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> If
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> old
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer correctly
> > > > > > > > >>>>>>>>>> registered
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>> partition
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> already,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can just ignore it
> > > > > > > > >>>>>>> as
> > > > > > > > >>>>>>>>>> it's an
> > > > > > > > >>>>>>>>>>>>>>>>> idempotent
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> operation?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (30)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To cover older
> > > > > > > > >>>>>>>> clients,
> > > > > > > > >>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>> will
> > > > > > > > >>>>>>>>>>>>>>>>> ensure a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ongoing
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> before
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we write to a
> > > > > > > > >>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Not sure what you
> > > > > > > > >>>>>>> mean
> > > > > > > > >>>>>>>>> by
> > > > > > > > >>>>>>>>>>>> this?
> > > > > > > > >>>>>>>>>>>>>>> Can
> > > > > > > > >>>>>>>>>>>>>>>> you
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (40)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [the
> > > > > > > > >>>>>>> TX-coordinator]
> > > > > > > > >>>>>>>>> will
> > > > > > > > >>>>>>>>>>>> write
> > > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>> prepare
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> commit
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch and send
> > > > > > > > >>>>>>>>>>>>>>> WriteTxnMarkerRequests
> > > > > > > > >>>>>>>>>>>>>>>>>> with
> > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> bumped
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why do we use the
> > > > > > > > >>>>>>>> bumped
> > > > > > > > >>>>>>>>>>>> epoch for
> > > > > > > > >>>>>>>>>>>>>>>>> both?
> > > > > > > > >>>>>>>>>>>>>>>>>> It
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> seems
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> more
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> intuitive
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the current epoch,
> > > > > > > > >>>>>>> and
> > > > > > > > >>>>>>>>>> only
> > > > > > > > >>>>>>>>>>>> return
> > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>> bumped
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (50) "Implicit
> > > > > > > > >>>>>>>>>>>>>>>>> AddPartitionToTransaction"
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why does the
> > > > > > > > >>>>>>>> implicitly
> > > > > > > > >>>>>>>>>> sent
> > > > > > > > >>>>>>>>>>>>>>> request
> > > > > > > > >>>>>>>>>>>>>>>>> need
> > > > > > > > >>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> synchronous?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also says
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in case we need
> > > > > > > > >>>>>> to
> > > > > > > > >>>>>>>>> abort
> > > > > > > > >>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>> need to
> > > > > > > > >>>>>>>>>>>>>>>>>> know
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> which
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What do you mean
> > > > > > > > >>>>>> by
> > > > > > > > >>>>>>>>> this?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we don’t want to
> > > > > > > > >>>>>>>> write
> > > > > > > > >>>>>>>>>> to it
> > > > > > > > >>>>>>>>>>>>>>> before
> > > > > > > > >>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>> store
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> manager
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Do you mean
> > > > > > > > >>>>>>>>> TX-coordinator
> > > > > > > > >>>>>>>>>>>>>>> instead of
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> "manager"?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (60)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For older clients
> > > > > > > > >>>>>>> and
> > > > > > > > >>>>>>>>>> ensuring
> > > > > > > > >>>>>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>> TX
> > > > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> ongoing,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> describe a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> race condition. I
> > > > > > > > >>>>>> am
> > > > > > > > >>>>>>>> not
> > > > > > > > >>>>>>>>>> sure
> > > > > > > > >>>>>>>>>>>> if I
> > > > > > > > >>>>>>>>>>>>>>>> can
> > > > > > > > >>>>>>>>>>>>>>>>>>>> follow
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> here.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> Can
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/18/22 1:21
> > > > > > > > >>>>>> PM,
> > > > > > > > >>>>>>>>>> Justine
> > > > > > > > >>>>>>>>>>>>>>> Olshan
> > > > > > > > >>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey all!
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to
> > > > > > > > >>>>>> start a
> > > > > > > > >>>>>>>>>>>> discussion
> > > > > > > > >>>>>>>>>>>>>>> on my
> > > > > > > > >>>>>>>>>>>>>>>>>>>> proposal
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> add
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> some
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checks on
> > > > > > > > >>>>>>>> transactions
> > > > > > > > >>>>>>>>> to
> > > > > > > > >>>>>>>>>>>> avoid
> > > > > > > > >>>>>>>>>>>>>>>>> hanging
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> transactions.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> been
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an issue for some
> > > > > > > > >>>>>>>> time,
> > > > > > > > >>>>>>>>>> so I
> > > > > > > > >>>>>>>>>>>>>>> really
> > > > > > > > >>>>>>>>>>>>>>>>> hope
> > > > > > > > >>>>>>>>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> KIP
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> helpful
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users of EOS.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The KIP includes
> > > > > > > > >>>>>>>>> changes
> > > > > > > > >>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>> will
> > > > > > > > >>>>>>>>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> compatible
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> old
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> changes to
> > > > > > > > >>>>>> improve
> > > > > > > > >>>>>>>>>>>> performance
> > > > > > > > >>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> correctness
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> on
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Please take a
> > > > > > > > >>>>>> look
> > > > > > > > >>>>>>>> and
> > > > > > > > >>>>>>>>>> leave
> > > > > > > > >>>>>>>>>>>> any
> > > > > > > > >>>>>>>>>>>>>>>>>> comments
> > > > > > > > >>>>>>>>>>>>>>>>>>>> you
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> may
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> have!
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>
> > > > > > > > >>>>
> > > > > > > > >>
> > > > > > >
> > > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> JIRA:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-14402
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>
> > > > > > > > >>>>>
> > > > > > > > >>>>
> > > > > > > > >>>
> > > > > > > > >>
> > > > > > > > >
> > > > > > >
> > > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
Hi Daniel,

You are correct. We need to slightly adjust the response from how it
appeared in the KIP. I've been focusing on part 1 (old clients) but when we
get into part 2, I will address this issue. I will also update this thread
when we do.
Thanks for bringing it up.

Justine.

On Tue, Apr 18, 2023 at 12:28 AM Dániel Urbán <ur...@gmail.com> wrote:

> Hi,
>
> Sorry for being super-late to this discussion, but a question came to my
> mind related to the bump-on-txn-end part of the KIP.
> I see that the END_TXN_RESPONSE will be updated to also include the new
> epoch to be used by the producer - but what about epoch exhaustion? What
> will happen when the last ongoing transaction uses the max available epoch,
> and we try to end it? The producer would need a new producer ID after the
> epoch bump. Because of this, I think the response should also contain the
> producer-id-to-use.
>
> Thanks,
> Daniel
>
> Artem Livshits <al...@confluent.io.invalid> ezt írta (időpont: 2023.
> jan. 28., Szo, 0:17):
>
> > >  I'd like to check that, under this KIP's circumstances, if we could
> ever
> > have "partial committed" txns
> >
> > The goal of this KIP is to harden transaction implementation to never
> have
> > partially committed txns.
> >
> > -Artem
> >
> > On Fri, Jan 27, 2023 at 10:04 AM Guozhang Wang <
> guozhang.wang.us@gmail.com
> > >
> > wrote:
> >
> > > Thanks folks, I agree with all of your points.
> > >
> > > I think in the past we did have some cases where abortable txns are
> > > handled as fatal ones which we are fixing, but maybe there are some
> > > vice versa as well. I'd like to check that, under this KIP's
> > > circumstances, if we could ever have "partial committed" txns where
> > > some of the data are considered as committed while some others are
> > > aborted later. And if there were, could we recover from it. If the
> > > answer is "yes and yes", then I think we can bite the complexity
> > > bullet inside the client and leave users in comfort and peace :)
> > >
> > > Guozhang
> > >
> > > On Fri, Jan 27, 2023 at 9:42 AM Artem Livshits
> > > <al...@confluent.io.invalid> wrote:
> > > >
> > > > I agree.  I also think we should go a little further and be more
> > > > prescriptive with our recommendations for error handling (i.e. not
> just
> > > > think if there is a chance, but propose design patterns for typical
> > > > cases).  For example, if a transaction is aborted, the application
> must
> > > > either keep enough state to be able to redo the operation from the
> > > > beginning, or reset all state and fully recover from Kafka (i.e.
> > > re-consume
> > > > data from the last committed offsets).
> > > >
> > > > For failed commits, we need a way for the application to either learn
> > the
> > > > result (if commit went through, then continue, otherwise redo
> > operation)
> > > or
> > > > just reset all state and fully recover from Kafka (i.e. re-consume
> data
> > > > from the last committed offsets).
> > > >
> > > > -Artem
> > > >
> > > > On Fri, Jan 27, 2023 at 9:31 AM Justine Olshan
> > > <jo...@confluent.io.invalid>
> > > > wrote:
> > > >
> > > > > Thanks for clarifying Guozhang,
> > > > >
> > > > > I like the framing:
> > > > > * “if we did abort the txn and continue, could the app ever has a
> > > chance to
> > > > > recover and not have any side effects violating EOS guarantees”.*
> > > > >
> > > > > With this principle in mind, if we could convince ourselves that we
> > > won't
> > > > > have side effects, then we could move forward with a non-fatal
> > error. I
> > > > > appreciate laying out the cases we've discussed. I think I also
> want
> > to
> > > > > take a look in very close detail and convince myself of the right
> > > option.
> > > > > I'll get back to this thread with the final conclusion as well as
> > > update
> > > > > the KIP.
> > > > >
> > > > > Thanks for also bringing up KIP-691. I should take a look there as
> > > well. :)
> > > > >
> > > > > Justine
> > > > >
> > > > > On Thu, Jan 26, 2023 at 5:28 AM Guozhang Wang <
> > > guozhang.wang.us@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hello Justine,
> > > > > >
> > > > > > Regarding which errors should be fatal v.s. abortable, I think
> the
> > > > > > principle would be “if we did abort the txn and continue, could
> the
> > > > > > app ever has a chance to recover and not have any side effects
> > > > > > violating EOS guarantees”.Of course we can discuss whether
> that’s a
> > > > > > good principle  e.g. one can argue that maybe it’s okay to
> violate
> > > EOS
> > > > > > some times rather than killing the whole app, whereas my
> rationale
> > is
> > > > > > that since we want to make EOS the default config, it’s crucial
> > that
> > > > > > we try to guarantee it as much as possible and whenever we
> detected
> > > if
> > > > > > there’s a chance that it maybe broken, we should let users know.
> > And
> > > > > > if we just log an error in app and try to abort and continue,
> it’s
> > > > > > very likely that users would not be notified and probably only
> > > realize
> > > > > > that after a while, which may cause an even harder scenario to
> > > > > > debug.If people agree on that principle, we can go back and check
> > > this
> > > > > > KIP’s scenario: there are several scenarios causing a partition
> > > leader
> > > > > > detects an out of date epoch from a produce requests:
> > > > > >
> > > > > > * The old produce requests arrive late after a network partition,
> > and
> > > > > > the old producer is already gone.
> > > > > > * The old produce requests arrive late after a network partition,
> > but
> > > > > > the old producer is still around.
> > > > > > * A buggy producer client that did not follow the protocol.
> > > > > >
> > > > > > For the third case, we should try to let user know asap, and as
> > clear
> > > > > > as possible, and hence it’s best to just stop the client app; for
> > the
> > > > > > first case, it does not matter since the producer is already
> gone;
> > > > > > only for the second, probably the least likely case, we need to
> > > > > > ponder, and there my rationale again is that at that time, we may
> > > have
> > > > > > already violated the EOS guarantees since there are some partial
> > txn
> > > > > > records that should be rejected while the txn itself has been
> > > > > > committed. In this case I think it’s better to let clients/users
> > know
> > > > > > as soon as possible as well.
> > > > > >
> > > > > > Regarding the past scenarios where a fatal error killing the
> whole
> > > > > > apps, I believe part of that reason is that we were doing an
> > > > > > sub-optimal job on clients side handling various error cases and
> > > > > > that’s what KIP-691 is trying to resolve, and hence personally I
> > > would
> > > > > > suggest we do not weight in too much on that if we can trust that
> > > > > > KIP-691 will be successfully eliminate those
> > not-necessary-hard-fail
> > > > > > scenarios.
> > > > > >
> > > > > >
> > > > > > Guozhang
> > > > > >
> > > > > > On Wed, Jan 25, 2023 at 5:51 PM Matthias J. Sax <
> mjsax@apache.org>
> > > > > wrote:
> > > > > > >
> > > > > > > So the timestamp would be set when the write happens and thus
> no
> > > > > > > out-of-order data (base in time) can be introduced with
> > > "append_time"
> > > > > > > config even if a request sits in purgatory first while we check
> > > the TX
> > > > > > > status.
> > > > > > >
> > > > > > > That does make sense. Thanks for confirming, that there is no
> > > > > > > out-of-order issue for this case.
> > > > > > >
> > > > > > >
> > > > > > > -Matthias
> > > > > > >
> > > > > > > On 1/25/23 5:04 PM, Justine Olshan wrote:
> > > > > > > > Hey Matthias,
> > > > > > > > Let me put it this way, if a producer is checking if a
> > > transaction is
> > > > > > > > ongoing, then no writes to the partition from the producer
> will
> > > go
> > > > > > through
> > > > > > > > until the transaction is confirmed ongoing.
> > > > > > > >  From then, I think I can apply the writes in the order they
> > > came in.
> > > > > > Does
> > > > > > > > that make sense?
> > > > > > > >
> > > > > > > > Let me know if I'm missing something.
> > > > > > > > Justine
> > > > > > > >
> > > > > > > > On Wed, Jan 25, 2023 at 4:57 PM Matthias J. Sax <
> > > mjsax@apache.org>
> > > > > > wrote:
> > > > > > > >
> > > > > > > >>> would it build an offset map with just the latest timestamp
> > > for a
> > > > > > key?
> > > > > > > >>
> > > > > > > >> Cannot remember the details without reading the KIP, but
> yes,
> > > > > > something
> > > > > > > >> like this (I believe it actually needs to track both, offset
> > and
> > > > > > > >> timestamp per key).
> > > > > > > >>
> > > > > > > >>> I wonder if ordering assumptions are baked in there, why
> not
> > > use
> > > > > > > >> offset-based compaction.
> > > > > > > >>
> > > > > > > >> The use case is a compacted topic that does contain
> > out-of-order
> > > > > data.
> > > > > > > >> If you set key k1=v1 @ 5 offset 100 and later key1 = v0 @ 3
> at
> > > > > offset
> > > > > > > >> 200 we want to cleanup v0 with higher offset because it's
> > > > > out-of-order
> > > > > > > >> based on time, but keep v1 what is the actual latest version
> > of
> > > k1.
> > > > > > > >>
> > > > > > > >>
> > > > > > > >>> I was also not aware of this "guarantee" with regards to
> > broker
> > > > > side
> > > > > > > >> time.
> > > > > > > >>
> > > > > > > >> As already said: I am not sure if it's a public contract,
> but
> > > based
> > > > > on
> > > > > > > >> my experience, people might reply on it as "implicit
> > contract".
> > > --
> > > > > > Maybe
> > > > > > > >> somebody else knows if it's public or not, and if it would
> be
> > > ok to
> > > > > > > >> "break" it.
> > > > > > > >>
> > > > > > > >>> Let me know if you have any concerns here.
> > > > > > > >>
> > > > > > > >> My understanding is: While we cannot make an offset-order
> > > guarantee
> > > > > > for
> > > > > > > >> interleaved writes of different producer, if the topic is
> > > configures
> > > > > > > >> with "append_time", we "guarantee" (cf. my comment above")
> > > timestamp
> > > > > > > >> order... If that's the case, it would be an issue if we
> break
> > > this
> > > > > > > >> "guarantee".
> > > > > > > >>
> > > > > > > >> I am not sure when the broker sets the timestamp for
> > > "append_time"
> > > > > > > >> config? If we do it before putting the request into
> purgatory,
> > > we
> > > > > > have a
> > > > > > > >> problem. However, if we set the timestamp when we actually
> > > process
> > > > > the
> > > > > > > >> request and do the actual append, it seems there is no
> issue,
> > > as the
> > > > > > > >> request that was waiting in purgatory get the "newest"
> > > timestamp and
> > > > > > > >> thus cannot introduce out-of-order data.
> > > > > > > >>
> > > > > > > >>
> > > > > > > >> -Matthias
> > > > > > > >>
> > > > > > > >>
> > > > > > > >> On 1/24/23 10:44 AM, Justine Olshan wrote:
> > > > > > > >>> Hey Matthias,
> > > > > > > >>>
> > > > > > > >>> I have actually never heard of KIP-280 so thanks for
> bringing
> > > it
> > > > > up.
> > > > > > That
> > > > > > > >>> seems interesting. I wonder how it would work though --
> would
> > > it
> > > > > > build an
> > > > > > > >>> offset map with just the latest timestamp for a key? I
> wonder
> > > if
> > > > > > ordering
> > > > > > > >>> assumptions are baked in there, why not use offset-based
> > > > > compaction.
> > > > > > > >>>
> > > > > > > >>> I was also not aware of this "guarantee" with regards to
> > broker
> > > > > side
> > > > > > > >> time.
> > > > > > > >>> I think that we can do in order handling for a given
> > producer,
> > > but
> > > > > > not
> > > > > > > >>> across all producers. However, we can't guarantee that
> > anyway.
> > > > > > > >>>
> > > > > > > >>> Let me know if you have any concerns here.
> > > > > > > >>>
> > > > > > > >>> Thanks,
> > > > > > > >>> Justine
> > > > > > > >>>
> > > > > > > >>> On Mon, Jan 23, 2023 at 6:33 PM Matthias J. Sax <
> > > mjsax@apache.org>
> > > > > > > >> wrote:
> > > > > > > >>>
> > > > > > > >>>> Just a side note about Guozhang comments about timestamps.
> > > > > > > >>>>
> > > > > > > >>>> If the producer sets the timestamp, putting the record
> into
> > > > > > purgatory
> > > > > > > >>>> seems not to be an issue (as already said: for this case
> we
> > > don't
> > > > > > > >>>> guarantee timestamp order between writes of different
> > > producers
> > > > > > anyway).
> > > > > > > >>>> However, if the broker sets the timestamp, the expectation
> > is
> > > that
> > > > > > there
> > > > > > > >>>> is no out-of-order data in the partition ever; if we would
> > > > > introduce
> > > > > > > >>>> out-of-order data for this case (for interleaved writes of
> > > > > different
> > > > > > > >>>> producers), it seems we would violate the current
> contract?
> > > (To be
> > > > > > fair:
> > > > > > > >>>> I don't know if that's an official contract, but I assume
> > > people
> > > > > > rely on
> > > > > > > >>>> this behavior -- and it "advertised" in many public
> > talks...)
> > > > > > > >>>>
> > > > > > > >>>> About compaction: there is actually KIP-280 that adds
> > > timestamp
> > > > > > based
> > > > > > > >>>> compaction what is a very useful feature for Kafka Streams
> > > with
> > > > > > regard
> > > > > > > >>>> to out-of-order data handling. So the impact if we
> introduce
> > > > > > > >>>> out-of-order data could be larger scoped.
> > > > > > > >>>>
> > > > > > > >>>>
> > > > > > > >>>> -Matthias
> > > > > > > >>>>
> > > > > > > >>>>
> > > > > > > >>>> On 1/20/23 4:48 PM, Justine Olshan wrote:
> > > > > > > >>>>> Hey Artem,
> > > > > > > >>>>>
> > > > > > > >>>>> I see there is a check for transactional producers. I'm
> > > wondering
> > > > > > if we
> > > > > > > >>>>> don't handle the epoch overflow case. I'm also not sure
> it
> > > will
> > > > > be
> > > > > > a
> > > > > > > >> huge
> > > > > > > >>>>> issue to extend to transactional producers, but maybe I'm
> > > missing
> > > > > > > >>>> something.
> > > > > > > >>>>>
> > > > > > > >>>>> As for the recovery path -- I think Guozhang's point was
> if
> > > we
> > > > > > have a
> > > > > > > >> bad
> > > > > > > >>>>> client that repeatedly tries to produce without adding to
> > the
> > > > > > > >> transaction
> > > > > > > >>>>> we would do the following:
> > > > > > > >>>>> a) if not fatal, we just fail the produce request over
> and
> > > over
> > > > > > > >>>>> b) if fatal, we fence the producer
> > > > > > > >>>>>
> > > > > > > >>>>> Here with B, the issue with the client would be made
> clear
> > > more
> > > > > > > >> quickly.
> > > > > > > >>>> I
> > > > > > > >>>>> suppose there are some intermediate cases where the issue
> > > only
> > > > > > occurs
> > > > > > > >>>>> sometimes, but I wonder if we should consider how to
> > recover
> > > with
> > > > > > > >> clients
> > > > > > > >>>>> who don't behave as expected anyway.
> > > > > > > >>>>>
> > > > > > > >>>>> I think there is a place for the abortable error that we
> > are
> > > > > > adding --
> > > > > > > >>>> just
> > > > > > > >>>>> abort and try again. But I think there are also some
> cases
> > > where
> > > > > > trying
> > > > > > > >>>> to
> > > > > > > >>>>> recover overcomplicates some logic. Especially if we are
> > > > > > considering
> > > > > > > >>>> older
> > > > > > > >>>>> clients -- there I'm not sure if there's a ton we can do
> > > besides
> > > > > > fail
> > > > > > > >> the
> > > > > > > >>>>> batch or fence the producer. With newer clients, we can
> > > consider
> > > > > > more
> > > > > > > >>>>> options for what can just be recovered after aborting.
> But
> > > epochs
> > > > > > might
> > > > > > > >>>> be
> > > > > > > >>>>> a hard one unless we also want to reset producer ID.
> > > > > > > >>>>>
> > > > > > > >>>>> Thanks,
> > > > > > > >>>>> Justine
> > > > > > > >>>>>
> > > > > > > >>>>>
> > > > > > > >>>>>
> > > > > > > >>>>> On Fri, Jan 20, 2023 at 3:59 PM Artem Livshits
> > > > > > > >>>>> <al...@confluent.io.invalid> wrote:
> > > > > > > >>>>>
> > > > > > > >>>>>>>     besides the poorly written client case
> > > > > > > >>>>>>
> > > > > > > >>>>>> A poorly written client could create a lot of grief to
> > > people
> > > > > who
> > > > > > run
> > > > > > > >>>> Kafka
> > > > > > > >>>>>> brokers :-), so when deciding to make an error fatal I
> > > would see
> > > > > > if
> > > > > > > >>>> there
> > > > > > > >>>>>> is a reasonable recovery path rather than how often it
> > could
> > > > > > happen.
> > > > > > > >>>> If we
> > > > > > > >>>>>> have solid implementation of transactions (which I hope
> > > we'll do
> > > > > > as a
> > > > > > > >>>>>> result of this KIP), it would help to recover from a
> large
> > > class
> > > > > > of
> > > > > > > >>>> errors
> > > > > > > >>>>>> by just aborting a transaction, even if the cause of
> error
> > > is a
> > > > > > race
> > > > > > > >>>>>> condition or etc.
> > > > > > > >>>>>>
> > > > > > > >>>>>> -Artem
> > > > > > > >>>>>>
> > > > > > > >>>>>> On Fri, Jan 20, 2023 at 3:26 PM Justine Olshan
> > > > > > > >>>>>> <jo...@confluent.io.invalid>
> > > > > > > >>>>>> wrote:
> > > > > > > >>>>>>
> > > > > > > >>>>>>> Artem --
> > > > > > > >>>>>>> I guess the discussion path we were going down is when
> we
> > > > > expect
> > > > > > to
> > > > > > > >> see
> > > > > > > >>>>>>> this error. I mentioned that it was hard to come up
> with
> > > cases
> > > > > > for
> > > > > > > >> when
> > > > > > > >>>>>> the
> > > > > > > >>>>>>> producer would still be around to receive the error
> > > besides the
> > > > > > > >> poorly
> > > > > > > >>>>>>> written client case.
> > > > > > > >>>>>>> If we don't expect to have a producer to receive the
> > > response,
> > > > > it
> > > > > > > >> sort
> > > > > > > >>>> of
> > > > > > > >>>>>>> makes sense for it to be fatal.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> I had some discussion with Jason offline about the
> epoch
> > > being
> > > > > > off
> > > > > > > >>>> cases
> > > > > > > >>>>>>> and I'm not sure we could find a ton (outside of
> produce
> > > > > > requests)
> > > > > > > >>>> where
> > > > > > > >>>>>> we
> > > > > > > >>>>>>> could/should recover. I'd be happy to hear some
> examples
> > > > > though,
> > > > > > > >> maybe
> > > > > > > >>>>>> I'm
> > > > > > > >>>>>>> missing something.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> Thanks,
> > > > > > > >>>>>>> Justine
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> On Fri, Jan 20, 2023 at 3:19 PM Artem Livshits
> > > > > > > >>>>>>> <al...@confluent.io.invalid> wrote:
> > > > > > > >>>>>>>
> > > > > > > >>>>>>>> In general, I'd like to avoid fatal errors as much as
> > > > > possible,
> > > > > > in
> > > > > > > >>>> some
> > > > > > > >>>>>>>> sense fatal errors just push out recovery logic to the
> > > > > > application
> > > > > > > >>>>>> which
> > > > > > > >>>>>>>> either complicates the application or leads to
> > disruption
> > > > > (we've
> > > > > > > >> seen
> > > > > > > >>>>>>> cases
> > > > > > > >>>>>>>> when a transient broker error could lead to work
> > stoppage
> > > when
> > > > > > > >>>>>>> applications
> > > > > > > >>>>>>>> need to be manually restarted).  I think we should
> > strive
> > > to
> > > > > > define
> > > > > > > >>>>>>>> recovery logic for most errors (and/or encapsulate it
> in
> > > the
> > > > > > Kafka
> > > > > > > >>>>>> client
> > > > > > > >>>>>>>> as much as possible).
> > > > > > > >>>>>>>>
> > > > > > > >>>>>>>> One benefit of transactions is that they simplify
> > recovery
> > > > > from
> > > > > > > >>>> errors,
> > > > > > > >>>>>>>> pretty much any error (that's not handled
> transparently
> > by
> > > > > > retries
> > > > > > > >> in
> > > > > > > >>>>>>> Kafka
> > > > > > > >>>>>>>> client) can be handled by the application via aborting
> > the
> > > > > > > >> transaction
> > > > > > > >>>>>>> and
> > > > > > > >>>>>>>> repeating the transactional logic again.  One tricky
> > > error is
> > > > > an
> > > > > > > >> error
> > > > > > > >>>>>>>> during commit, because we don't know the outcome.  For
> > > commit
> > > > > > > >> errors,
> > > > > > > >>>>>> the
> > > > > > > >>>>>>>> recommendation should be to retry the commit until it
> > > returns
> > > > > > the
> > > > > > > >>>>>>> specific
> > > > > > > >>>>>>>> result (committed or aborted).
> > > > > > > >>>>>>>>
> > > > > > > >>>>>>>> -Artem
> > > > > > > >>>>>>>>
> > > > > > > >>>>>>>> On Fri, Jan 20, 2023 at 2:52 PM Justine Olshan
> > > > > > > >>>>>>>> <jo...@confluent.io.invalid>
> > > > > > > >>>>>>>> wrote:
> > > > > > > >>>>>>>>
> > > > > > > >>>>>>>>> That's a fair point about other clients.
> > > > > > > >>>>>>>>>
> > > > > > > >>>>>>>>> I think the abortable error case is interesting
> because
> > > I'm
> > > > > > curious
> > > > > > > >>>>>> how
> > > > > > > >>>>>>>>> other clients would handle this. I assume they would
> > > need to
> > > > > > > >>>>>> implement
> > > > > > > >>>>>>>>> handling for the error code unless they did something
> > > like
> > > > > "any
> > > > > > > >>>>>> unknown
> > > > > > > >>>>>>>>> error codes/any codes that aren't x,y,z are
> > retriable." I
> > > > > would
> > > > > > > >> hope
> > > > > > > >>>>>>> that
> > > > > > > >>>>>>>>> unknown error codes were fatal, and if the code was
> > > > > > implemented it
> > > > > > > >>>>>>> would
> > > > > > > >>>>>>>>> abort the transaction. But I will think on this too.
> > > > > > > >>>>>>>>>
> > > > > > > >>>>>>>>> As for InvalidRecord -- you mentioned it was not
> fatal,
> > > but
> > > > > I'm
> > > > > > > >>>>>> taking
> > > > > > > >>>>>>> a
> > > > > > > >>>>>>>>> look through the code. We would see this on handling
> > the
> > > > > > produce
> > > > > > > >>>>>>>> response.
> > > > > > > >>>>>>>>> If I recall correctly, we check if errors are
> > retriable.
> > > I
> > > > > > think
> > > > > > > >> this
> > > > > > > >>>>>>>> error
> > > > > > > >>>>>>>>> would not be retriable. But I guess the concern here
> is
> > > that
> > > > > > it is
> > > > > > > >>>>>> not
> > > > > > > >>>>>>>>> enough for just that batch to fail. I guess I hadn't
> > > > > considered
> > > > > > > >> fully
> > > > > > > >>>>>>>>> fencing the old producer but there are valid
> arguments
> > > here
> > > > > > why we
> > > > > > > >>>>>>> would
> > > > > > > >>>>>>>>> want to.
> > > > > > > >>>>>>>>>
> > > > > > > >>>>>>>>> Thanks,
> > > > > > > >>>>>>>>> Justine
> > > > > > > >>>>>>>>>
> > > > > > > >>>>>>>>> On Fri, Jan 20, 2023 at 2:35 PM Guozhang Wang <
> > > > > > > >>>>>>>> guozhang.wang.us@gmail.com>
> > > > > > > >>>>>>>>> wrote:
> > > > > > > >>>>>>>>>
> > > > > > > >>>>>>>>>> Thanks Justine for the replies! I agree with most of
> > > your
> > > > > > > >> thoughts.
> > > > > > > >>>>>>>>>>
> > > > > > > >>>>>>>>>> Just for 3/7), though I agree for our own AK
> producer,
> > > since
> > > > > > we do
> > > > > > > >>>>>>>>>> "nextRequest(boolean hasIncompleteBatches)", we
> > > guarantee
> > > > > the
> > > > > > > >>>>>> end-txn
> > > > > > > >>>>>>>>>> would not be sent until we've effectively flushed,
> but
> > > I was
> > > > > > > >>>>>>> referring
> > > > > > > >>>>>>>>>> to any future bugs or other buggy clients that the
> > same
> > > > > > client may
> > > > > > > >>>>>>> get
> > > > > > > >>>>>>>>>> into this situation, in which case we should give
> the
> > > > > client a
> > > > > > > >>>>>> clear
> > > > > > > >>>>>>>>>> msg that "you did something wrong, and hence now you
> > > should
> > > > > > > >> fatally
> > > > > > > >>>>>>>>>> close yourself". What I'm concerned about is that,
> by
> > > seeing
> > > > > > an
> > > > > > > >>>>>>>>>> "abortable error" or in some rare cases an "invalid
> > > record",
> > > > > > the
> > > > > > > >>>>>>>>>> client could not realize "something that's really
> bad
> > > > > > happened".
> > > > > > > >> So
> > > > > > > >>>>>>>>>> it's not about adding a new error, it's mainly about
> > > those
> > > > > > real
> > > > > > > >>>>>> buggy
> > > > > > > >>>>>>>>>> situations causing such "should never happen" cases,
> > the
> > > > > > errors
> > > > > > > >>>>>>> return
> > > > > > > >>>>>>>>>> would not be informative enough.
> > > > > > > >>>>>>>>>>
> > > > > > > >>>>>>>>>> Thinking in other ways, if we believe that for most
> > > cases
> > > > > such
> > > > > > > >>>>>> error
> > > > > > > >>>>>>>>>> codes would not reach the original clients since
> they
> > > would
> > > > > be
> > > > > > > >>>>>>>>>> disconnected or even gone by that time, and only in
> > some
> > > > > rare
> > > > > > > >> cases
> > > > > > > >>>>>>>>>> they would still be seen by the sending clients,
> then
> > > why
> > > > > not
> > > > > > make
> > > > > > > >>>>>>>>>> them more fatal and more specific than generic.
> > > > > > > >>>>>>>>>>
> > > > > > > >>>>>>>>>> Guozhang
> > > > > > > >>>>>>>>>>
> > > > > > > >>>>>>>>>> On Fri, Jan 20, 2023 at 1:59 PM Justine Olshan
> > > > > > > >>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > > > > > > >>>>>>>>>>>
> > > > > > > >>>>>>>>>>> Hey Guozhang. Thanks for taking a look and for the
> > > detailed
> > > > > > > >>>>>>> comments!
> > > > > > > >>>>>>>>>> I'll
> > > > > > > >>>>>>>>>>> do my best to address below.
> > > > > > > >>>>>>>>>>>
> > > > > > > >>>>>>>>>>> 1. I see what you are saying here, but I think I
> need
> > > to
> > > > > look
> > > > > > > >>>>>>> through
> > > > > > > >>>>>>>>> the
> > > > > > > >>>>>>>>>>> sequence of events you mention. Typically we've
> seen
> > > this
> > > > > > issue
> > > > > > > >>>>>> in
> > > > > > > >>>>>>> a
> > > > > > > >>>>>>>>> few
> > > > > > > >>>>>>>>>>> cases.
> > > > > > > >>>>>>>>>>>
> > > > > > > >>>>>>>>>>>     One is when we have a producer disconnect when
> > > trying
> > > > > to
> > > > > > > >>>>>> produce.
> > > > > > > >>>>>>>>>>> Typically in these cases, we abort the transaction.
> > > We've
> > > > > > seen
> > > > > > > >>>>>> that
> > > > > > > >>>>>>>>> after
> > > > > > > >>>>>>>>>>> the markers are written, the disconnection can
> > > sometimes
> > > > > > cause
> > > > > > > >>>>>> the
> > > > > > > >>>>>>>>>> request
> > > > > > > >>>>>>>>>>> to get flushed to the broker. In this case, we
> don't
> > > need
> > > > > > client
> > > > > > > >>>>>>>>> handling
> > > > > > > >>>>>>>>>>> because the producer we are responding to is gone.
> We
> > > just
> > > > > > needed
> > > > > > > >>>>>>> to
> > > > > > > >>>>>>>>> make
> > > > > > > >>>>>>>>>>> sure we didn't write to the log on the broker side.
> > I'm
> > > > > > trying to
> > > > > > > >>>>>>>> think
> > > > > > > >>>>>>>>>> of
> > > > > > > >>>>>>>>>>> a case where we do have the client to return to.
> I'd
> > > think
> > > > > > the
> > > > > > > >>>>>> same
> > > > > > > >>>>>>>>>> client
> > > > > > > >>>>>>>>>>> couldn't progress to committing the transaction
> > unless
> > > the
> > > > > > > >>>>>> produce
> > > > > > > >>>>>>>>>> request
> > > > > > > >>>>>>>>>>> returned right? Of course, there is the incorrectly
> > > written
> > > > > > > >>>>>> clients
> > > > > > > >>>>>>>>> case.
> > > > > > > >>>>>>>>>>> I'll think on this a bit more and let you know if I
> > > come up
> > > > > > with
> > > > > > > >>>>>>>>> another
> > > > > > > >>>>>>>>>>> scenario when we would return to an active client
> > when
> > > the
> > > > > > > >>>>>>>> transaction
> > > > > > > >>>>>>>>> is
> > > > > > > >>>>>>>>>>> no longer ongoing.
> > > > > > > >>>>>>>>>>>
> > > > > > > >>>>>>>>>>> I was not aware that we checked the result of a
> send
> > > after
> > > > > we
> > > > > > > >>>>>>> commit
> > > > > > > >>>>>>>>>>> though. I'll need to look into that a bit more.
> > > > > > > >>>>>>>>>>>
> > > > > > > >>>>>>>>>>> 2. There were some questions about this in the
> > > discussion.
> > > > > > The
> > > > > > > >>>>>> plan
> > > > > > > >>>>>>>> is
> > > > > > > >>>>>>>>> to
> > > > > > > >>>>>>>>>>> handle overflow with the mechanism we currently
> have
> > > in the
> > > > > > > >>>>>>> producer.
> > > > > > > >>>>>>>>> If
> > > > > > > >>>>>>>>>> we
> > > > > > > >>>>>>>>>>> try to bump and the epoch will overflow, we
> actually
> > > > > > allocate a
> > > > > > > >>>>>> new
> > > > > > > >>>>>>>>>>> producer ID. I need to confirm the fencing logic on
> > the
> > > > > last
> > > > > > > >>>>>> epoch
> > > > > > > >>>>>>>> (ie,
> > > > > > > >>>>>>>>>> we
> > > > > > > >>>>>>>>>>> probably shouldn't allow any records to be produced
> > > with
> > > > > the
> > > > > > > >>>>>> final
> > > > > > > >>>>>>>>> epoch
> > > > > > > >>>>>>>>>>> since we can never properly fence that one).
> > > > > > > >>>>>>>>>>>
> > > > > > > >>>>>>>>>>> 3. I can agree with you that the current error
> > > handling is
> > > > > > > >>>>>> messy. I
> > > > > > > >>>>>>>>>> recall
> > > > > > > >>>>>>>>>>> taking a look at your KIP a while back, but I
> think I
> > > > > mostly
> > > > > > saw
> > > > > > > >>>>>>> the
> > > > > > > >>>>>>>>>>> section about how the errors were wrapped. Maybe I
> > > need to
> > > > > > take
> > > > > > > >>>>>>>> another
> > > > > > > >>>>>>>>>>> look. As for abortable error, the idea was that the
> > > > > handling
> > > > > > > >>>>>> would
> > > > > > > >>>>>>> be
> > > > > > > >>>>>>>>>>> simple -- if this error is seen, the transaction
> > > should be
> > > > > > > >>>>>> aborted
> > > > > > > >>>>>>> --
> > > > > > > >>>>>>>>> no
> > > > > > > >>>>>>>>>>> other logic about previous state or requests
> > > necessary. Is
> > > > > > your
> > > > > > > >>>>>>>> concern
> > > > > > > >>>>>>>>>>> simply about adding new errors? We were hoping to
> > have
> > > an
> > > > > > error
> > > > > > > >>>>>>> that
> > > > > > > >>>>>>>>>> would
> > > > > > > >>>>>>>>>>> have one meaning and many of the current errors
> have
> > a
> > > > > > history of
> > > > > > > >>>>>>>>> meaning
> > > > > > > >>>>>>>>>>> different things on different client versions. That
> > > was the
> > > > > > main
> > > > > > > >>>>>>>>>> motivation
> > > > > > > >>>>>>>>>>> for adding a new error.
> > > > > > > >>>>>>>>>>>
> > > > > > > >>>>>>>>>>> 4. This is a good point about record timestamp
> > > reordering.
> > > > > > > >>>>>>> Timestamps
> > > > > > > >>>>>>>>>> don't
> > > > > > > >>>>>>>>>>> affect compaction, but they do affect retention
> > > deletion.
> > > > > For
> > > > > > > >>>>>> that,
> > > > > > > >>>>>>>>> kafka
> > > > > > > >>>>>>>>>>> considers the largest timestamp in the segment, so
> I
> > > think
> > > > > a
> > > > > > > >>>>>> small
> > > > > > > >>>>>>>>> amount
> > > > > > > >>>>>>>>>>> of reordering (hopefully on the order of
> milliseconds
> > > or
> > > > > even
> > > > > > > >>>>>>>> seconds)
> > > > > > > >>>>>>>>>> will
> > > > > > > >>>>>>>>>>> be ok. We take timestamps from clients so there is
> > > already
> > > > > a
> > > > > > > >>>>>>>>> possibility
> > > > > > > >>>>>>>>>>> for some drift and non-monotonically increasing
> > > timestamps.
> > > > > > > >>>>>>>>>>>
> > > > > > > >>>>>>>>>>> 5. Thanks for catching. The error is there, but
> it's
> > > > > actually
> > > > > > > >>>>>> that
> > > > > > > >>>>>>>>> those
> > > > > > > >>>>>>>>>>> fields should be 4+! Due to how the message
> generator
> > > > > works,
> > > > > > I
> > > > > > > >>>>>>>> actually
> > > > > > > >>>>>>>>>>> have to redefine those fields inside the
> > > > > > > >>>>>>>>> `"AddPartitionsToTxnTransaction`
> > > > > > > >>>>>>>>>>> block for it to build correctly. I'll fix it to be
> > > correct.
> > > > > > > >>>>>>>>>>>
> > > > > > > >>>>>>>>>>> 6. Correct -- we will only add the request to
> > > purgatory if
> > > > > > the
> > > > > > > >>>>>>> cache
> > > > > > > >>>>>>>>> has
> > > > > > > >>>>>>>>>> no
> > > > > > > >>>>>>>>>>> ongoing transaction. I can change the wording to
> make
> > > that
> > > > > > > >>>>>> clearer
> > > > > > > >>>>>>>> that
> > > > > > > >>>>>>>>>> we
> > > > > > > >>>>>>>>>>> only place the request in purgatory if we need to
> > > contact
> > > > > the
> > > > > > > >>>>>>>>> transaction
> > > > > > > >>>>>>>>>>> coordinator.
> > > > > > > >>>>>>>>>>>
> > > > > > > >>>>>>>>>>> 7. We did take a look at some of the errors and it
> > was
> > > hard
> > > > > > to
> > > > > > > >>>>>> come
> > > > > > > >>>>>>>> up
> > > > > > > >>>>>>>>>> with
> > > > > > > >>>>>>>>>>> a good one. I agree that InvalidTxnStateException
> is
> > > ideal
> > > > > > except
> > > > > > > >>>>>>> for
> > > > > > > >>>>>>>>> the
> > > > > > > >>>>>>>>>>> fact that it hasn't been returned on Produce
> requests
> > > > > > before. The
> > > > > > > >>>>>>>> error
> > > > > > > >>>>>>>>>>> handling for clients is a bit vague (which is why I
> > > opened
> > > > > > > >>>>>>>> KAFKA-14439
> > > > > > > >>>>>>>>>>> <https://issues.apache.org/jira/browse/KAFKA-14439
> > >),
> > > but
> > > > > > the
> > > > > > > >>>>>>>> decision
> > > > > > > >>>>>>>>>> we
> > > > > > > >>>>>>>>>>> made here was to only return errors that have been
> > > > > previously
> > > > > > > >>>>>>>> returned
> > > > > > > >>>>>>>>> to
> > > > > > > >>>>>>>>>>> producers. As for not being fatal, I think part of
> > the
> > > > > > theory was
> > > > > > > >>>>>>>> that
> > > > > > > >>>>>>>>> in
> > > > > > > >>>>>>>>>>> many cases, the producer would be disconnected.
> (See
> > > point
> > > > > > 1) and
> > > > > > > >>>>>>>> this
> > > > > > > >>>>>>>>>>> would just be an error to return from the server. I
> > did
> > > > > plan
> > > > > > to
> > > > > > > >>>>>>> think
> > > > > > > >>>>>>>>>> about
> > > > > > > >>>>>>>>>>> other cases, so let me know if you think of any as
> > > well!
> > > > > > > >>>>>>>>>>>
> > > > > > > >>>>>>>>>>> Lots to say! Let me know if you have further
> > thoughts!
> > > > > > > >>>>>>>>>>> Justine
> > > > > > > >>>>>>>>>>>
> > > > > > > >>>>>>>>>>> On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang <
> > > > > > > >>>>>>>>>> guozhang.wang.us@gmail.com>
> > > > > > > >>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> Hello Justine,
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> Thanks for the great write-up! I made a quick pass
> > > through
> > > > > > it
> > > > > > > >>>>>> and
> > > > > > > >>>>>>>>> here
> > > > > > > >>>>>>>>>>>> are some thoughts (I have not been able to read
> > > through
> > > > > this
> > > > > > > >>>>>>> thread
> > > > > > > >>>>>>>>> so
> > > > > > > >>>>>>>>>>>> pardon me if they have overlapped or subsumed by
> > > previous
> > > > > > > >>>>>>>> comments):
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> First are some meta ones:
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> 1. I think we need to also improve the client's
> > > experience
> > > > > > once
> > > > > > > >>>>>>> we
> > > > > > > >>>>>>>>>>>> have this defence in place. More concretely, say a
> > > user's
> > > > > > > >>>>>>> producer
> > > > > > > >>>>>>>>>>>> code is like following:
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> future = producer.send();
> > > > > > > >>>>>>>>>>>> // producer.flush();
> > > > > > > >>>>>>>>>>>> producer.commitTransaction();
> > > > > > > >>>>>>>>>>>> future.get();
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> Which resulted in the order of a) produce-request
> > > sent by
> > > > > > > >>>>>>> producer,
> > > > > > > >>>>>>>>> b)
> > > > > > > >>>>>>>>>>>> end-txn-request sent by producer, c)
> > end-txn-response
> > > sent
> > > > > > > >>>>>> back,
> > > > > > > >>>>>>> d)
> > > > > > > >>>>>>>>>>>> txn-marker-request sent from coordinator to
> > partition
> > > > > > leader,
> > > > > > > >>>>>> e)
> > > > > > > >>>>>>>>>>>> produce-request finally received by the partition
> > > leader,
> > > > > > > >>>>>> before
> > > > > > > >>>>>>>> this
> > > > > > > >>>>>>>>>>>> KIP e) step would be accepted causing a dangling
> > txn;
> > > now
> > > > > it
> > > > > > > >>>>>>> would
> > > > > > > >>>>>>>> be
> > > > > > > >>>>>>>>>>>> rejected in step e) which is good. But from the
> > > client's
> > > > > > point
> > > > > > > >>>>>> of
> > > > > > > >>>>>>>>> view
> > > > > > > >>>>>>>>>>>> now it becomes confusing since the
> > > `commitTransaction()`
> > > > > > > >>>>>> returns
> > > > > > > >>>>>>>>>>>> successfully, but the "future" throws an
> > invalid-epoch
> > > > > > error,
> > > > > > > >>>>>> and
> > > > > > > >>>>>>>>> they
> > > > > > > >>>>>>>>>>>> are not sure if the transaction did succeed or
> not.
> > In
> > > > > > fact, it
> > > > > > > >>>>>>>>>>>> "partially succeeded" with some msgs being
> rejected
> > > but
> > > > > > others
> > > > > > > >>>>>>>>>>>> committed successfully.
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> Of course the easy way to avoid this is, always
> call
> > > > > > > >>>>>>>>>>>> "producer.flush()" before commitTxn and that's
> what
> > > we do
> > > > > > > >>>>>>>> ourselves,
> > > > > > > >>>>>>>>>>>> and what we recommend users do. But I suspect not
> > > everyone
> > > > > > does
> > > > > > > >>>>>>> it.
> > > > > > > >>>>>>>>> In
> > > > > > > >>>>>>>>>>>> fact I just checked the javadoc in KafkaProducer
> and
> > > our
> > > > > > code
> > > > > > > >>>>>>>> snippet
> > > > > > > >>>>>>>>>>>> does not include a `flush()` call. So I'm thinking
> > > maybe
> > > > > we
> > > > > > can
> > > > > > > >>>>>>> in
> > > > > > > >>>>>>>>>>>> side the `commitTxn` code to enforce flushing
> before
> > > > > sending
> > > > > > > >>>>>> the
> > > > > > > >>>>>>>>>>>> end-txn request.
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> 2. I'd like to clarify a bit details on "just add
> > > > > > partitions to
> > > > > > > >>>>>>> the
> > > > > > > >>>>>>>>>>>> transaction on the first produce request during a
> > > > > > transaction".
> > > > > > > >>>>>>> My
> > > > > > > >>>>>>>>>>>> understanding is that the partition leader's cache
> > > has the
> > > > > > > >>>>>>> producer
> > > > > > > >>>>>>>>> id
> > > > > > > >>>>>>>>>>>> / sequence / epoch for the latest txn, either
> > > on-going or
> > > > > is
> > > > > > > >>>>>>>>> completed
> > > > > > > >>>>>>>>>>>> (upon receiving the marker request from
> > coordinator).
> > > > > When a
> > > > > > > >>>>>>>> produce
> > > > > > > >>>>>>>>>>>> request is received, if
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> * producer's epoch < cached epoch, or producer's
> > > epoch ==
> > > > > > > >>>>>> cached
> > > > > > > >>>>>>>>> epoch
> > > > > > > >>>>>>>>>>>> but the latest txn is completed, leader directly
> > > reject
> > > > > with
> > > > > > > >>>>>>>>>>>> invalid-epoch.
> > > > > > > >>>>>>>>>>>> * producer's epoch > cached epoch, park the the
> > > request
> > > > > and
> > > > > > > >>>>>> send
> > > > > > > >>>>>>>>>>>> add-partitions request to coordinator.
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> In order to do it, does the coordinator need to
> bump
> > > the
> > > > > > > >>>>>> sequence
> > > > > > > >>>>>>>> and
> > > > > > > >>>>>>>>>>>> reset epoch to 0 when the next epoch is going to
> > > overflow?
> > > > > > If
> > > > > > > >>>>>> no
> > > > > > > >>>>>>>> need
> > > > > > > >>>>>>>>>>>> to do so, then how we handle the (admittedly rare,
> > but
> > > > > still
> > > > > > > >>>>>> may
> > > > > > > >>>>>>>>>>>> happen) epoch overflow situation?
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> 3. I'm a bit concerned about adding a generic
> > > > > > "ABORTABLE_ERROR"
> > > > > > > >>>>>>>> given
> > > > > > > >>>>>>>>>>>> we already have a pretty messy error
> classification
> > > and
> > > > > > error
> > > > > > > >>>>>>>>> handling
> > > > > > > >>>>>>>>>>>> on the producer clients side --- I have a summary
> > > about
> > > > > the
> > > > > > > >>>>>>> issues
> > > > > > > >>>>>>>>> and
> > > > > > > >>>>>>>>>>>> a proposal to address this in
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>
> > > > > > > >>>>>>>>>
> > > > > > > >>>>>>>>
> > > > > > > >>>>>>>
> > > > > > > >>>>>>
> > > > > > > >>>>
> > > > > > > >>
> > > > > >
> > > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
> > > > > > > >>>>>>>>>>>> -- I understand we do not want to use
> > > > > "UNKNOWN_PRODUCER_ID"
> > > > > > > >>>>>>> anymore
> > > > > > > >>>>>>>>>>>> and in fact we intend to deprecate it in KIP-360
> and
> > > > > > eventually
> > > > > > > >>>>>>>>> remove
> > > > > > > >>>>>>>>>>>> it; but I'm wondering can we still use specific
> > error
> > > > > codes.
> > > > > > > >>>>>> E.g.
> > > > > > > >>>>>>>>> what
> > > > > > > >>>>>>>>>>>> about "InvalidProducerEpochException" since for
> new
> > > > > clients,
> > > > > > > >>>>>> the
> > > > > > > >>>>>>>>>>>> actual reason this would actually be rejected is
> > > indeed
> > > > > > because
> > > > > > > >>>>>>> the
> > > > > > > >>>>>>>>>>>> epoch on the coordinator caused the
> > > add-partitions-request
> > > > > > from
> > > > > > > >>>>>>> the
> > > > > > > >>>>>>>>>>>> brokers to be rejected anyways?
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> 4. It seems we put the producer request into
> > purgatory
> > > > > > before
> > > > > > > >>>>>> we
> > > > > > > >>>>>>>> ever
> > > > > > > >>>>>>>>>>>> append the records, while other producer's records
> > may
> > > > > > still be
> > > > > > > >>>>>>>>>>>> appended during the time; and that potentially may
> > > result
> > > > > in
> > > > > > > >>>>>> some
> > > > > > > >>>>>>>>>>>> re-ordering compared with reception order. I'm not
> > > super
> > > > > > > >>>>>>> concerned
> > > > > > > >>>>>>>>>>>> about it since Kafka does not guarantee reception
> > > ordering
> > > > > > > >>>>>> across
> > > > > > > >>>>>>>>>>>> producers anyways, but it may make the timestamps
> of
> > > > > records
> > > > > > > >>>>>>>> inside a
> > > > > > > >>>>>>>>>>>> partition to be more out-of-ordered. Are we aware
> of
> > > any
> > > > > > > >>>>>>> scenarios
> > > > > > > >>>>>>>>>>>> such as future enhancements on log compactions
> that
> > > may be
> > > > > > > >>>>>>> affected
> > > > > > > >>>>>>>>> by
> > > > > > > >>>>>>>>>>>> this effect?
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> Below are just minor comments:
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> 5. In "AddPartitionsToTxnTransaction" field of
> > > > > > > >>>>>>>>>>>> "AddPartitionsToTxnRequest" RPC, the versions of
> > those
> > > > > inner
> > > > > > > >>>>>>> fields
> > > > > > > >>>>>>>>>>>> are "0-3" while I thought they should be "0+"
> still?
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> 6. Regarding "we can place the request in a
> > purgatory
> > > of
> > > > > > sorts
> > > > > > > >>>>>>> and
> > > > > > > >>>>>>>>>>>> check if there is any state for the transaction on
> > the
> > > > > > > >>>>>> broker": i
> > > > > > > >>>>>>>>>>>> think at this time when we just do the checks
> > against
> > > the
> > > > > > > >>>>>> cached
> > > > > > > >>>>>>>>>>>> state, we do not need to put the request to
> > purgatory
> > > yet?
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> 7. This is related to 3) above. I feel using
> > > > > > > >>>>>>>> "InvalidRecordException"
> > > > > > > >>>>>>>>>>>> for older clients may also be a bit confusing, and
> > > also it
> > > > > > is
> > > > > > > >>>>>> not
> > > > > > > >>>>>>>>>>>> fatal -- for old clients, it better to be fatal
> > since
> > > this
> > > > > > > >>>>>>>> indicates
> > > > > > > >>>>>>>>>>>> the clients is doing something wrong and hence it
> > > should
> > > > > be
> > > > > > > >>>>>>> closed.
> > > > > > > >>>>>>>>>>>> And in general I'd prefer to use slightly more
> > > specific
> > > > > > meaning
> > > > > > > >>>>>>>> error
> > > > > > > >>>>>>>>>>>> codes for clients. That being said, I also feel
> > > > > > > >>>>>>>>>>>> "InvalidProducerEpochException" is not suitable
> for
> > > old
> > > > > > > >>>>>> versioned
> > > > > > > >>>>>>>>>>>> clients, and we'd have to pick one that old
> clients
> > > > > > recognize.
> > > > > > > >>>>>>> I'd
> > > > > > > >>>>>>>>>>>> prefer "InvalidTxnStateException" but that one is
> > > supposed
> > > > > > to
> > > > > > > >>>>>> be
> > > > > > > >>>>>>>>>>>> returned from txn coordinators only today. I'd
> > > suggest we
> > > > > > do a
> > > > > > > >>>>>>>> quick
> > > > > > > >>>>>>>>>>>> check in the current client's code path and see if
> > > that
> > > > > one
> > > > > > > >>>>>> would
> > > > > > > >>>>>>>> be
> > > > > > > >>>>>>>>>>>> handled if it's from a produce-response, and if
> yes,
> > > use
> > > > > > this
> > > > > > > >>>>>>> one;
> > > > > > > >>>>>>>>>>>> otherwise, use "ProducerFencedException" which is
> > much
> > > > > less
> > > > > > > >>>>>>>>> meaningful
> > > > > > > >>>>>>>>>>>> but it's still a fatal error.
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> Thanks,
> > > > > > > >>>>>>>>>>>> Guozhang
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
> > > > > > > >>>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > > > > > > >>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>> Yeah -- looks like we already have code to handle
> > > bumping
> > > > > > the
> > > > > > > >>>>>>>> epoch
> > > > > > > >>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>> when the epoch is Short.MAX_VALUE, we get a new
> > > producer
> > > > > > ID.
> > > > > > > >>>>>>>> Since
> > > > > > > >>>>>>>>>> this
> > > > > > > >>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>> already the behavior, do we want to change it
> > > further?
> > > > > > > >>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>> Justine
> > > > > > > >>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>> On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <
> > > > > > > >>>>>>>>> jolshan@confluent.io
> > > > > > > >>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>> Hey all, just wanted to quickly update and say
> > I've
> > > > > > > >>>>>> modified
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>> KIP to
> > > > > > > >>>>>>>>>>>>>> explicitly mention that
> > AddOffsetCommitsToTxnRequest
> > > > > will
> > > > > > > >>>>>> be
> > > > > > > >>>>>>>>>> replaced
> > > > > > > >>>>>>>>>>>> by
> > > > > > > >>>>>>>>>>>>>> a coordinator-side (inter-broker)
> > AddPartitionsToTxn
> > > > > > > >>>>>> implicit
> > > > > > > >>>>>>>>>> request.
> > > > > > > >>>>>>>>>>>> This
> > > > > > > >>>>>>>>>>>>>> mirrors the user partitions and will implicitly
> > add
> > > > > offset
> > > > > > > >>>>>>>>>> partitions
> > > > > > > >>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>> transactions when we commit offsets on them. We
> > will
> > > > > > > >>>>>>> deprecate
> > > > > > > >>>>>>>>>>>> AddOffsetCommitsToTxnRequest
> > > > > > > >>>>>>>>>>>>>> for new clients.
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>> Also to address Artem's comments --
> > > > > > > >>>>>>>>>>>>>> I'm a bit unsure if the changes here will change
> > the
> > > > > > > >>>>>> previous
> > > > > > > >>>>>>>>>> behavior
> > > > > > > >>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>> fencing producers. In the case you mention in
> the
> > > first
> > > > > > > >>>>>>>>> paragraph,
> > > > > > > >>>>>>>>>> are
> > > > > > > >>>>>>>>>>>> you
> > > > > > > >>>>>>>>>>>>>> saying we bump the epoch before we try to abort
> > the
> > > > > > > >>>>>>>> transaction?
> > > > > > > >>>>>>>>> I
> > > > > > > >>>>>>>>>>>> think I
> > > > > > > >>>>>>>>>>>>>> need to understand the scenarios you mention a
> bit
> > > > > better.
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>> As for the second part -- I think it makes sense
> > to
> > > have
> > > > > > > >>>>>> some
> > > > > > > >>>>>>>>> sort
> > > > > > > >>>>>>>>>> of
> > > > > > > >>>>>>>>>>>>>> "sentinel" epoch to signal epoch is about to
> > > overflow (I
> > > > > > > >>>>>>> think
> > > > > > > >>>>>>>> we
> > > > > > > >>>>>>>>>> sort
> > > > > > > >>>>>>>>>>>> of
> > > > > > > >>>>>>>>>>>>>> have this value in place in some ways) so we can
> > > codify
> > > > > it
> > > > > > > >>>>>> in
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>> KIP.
> > > > > > > >>>>>>>>>>>> I'll
> > > > > > > >>>>>>>>>>>>>> look into that and try to update soon.
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>> Thanks,
> > > > > > > >>>>>>>>>>>>>> Justine.
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>> On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
> > > > > > > >>>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> It's good to know that KIP-588 addressed some
> of
> > > the
> > > > > > > >>>>>> issues.
> > > > > > > >>>>>>>>>> Looking
> > > > > > > >>>>>>>>>>>> at
> > > > > > > >>>>>>>>>>>>>>> the code, it still looks like there are some
> > cases
> > > that
> > > > > > > >>>>>>> would
> > > > > > > >>>>>>>>>> result
> > > > > > > >>>>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>>> fatal error, e.g. PRODUCER_FENCED is issued by
> > the
> > > > > > > >>>>>>> transaction
> > > > > > > >>>>>>>>>>>> coordinator
> > > > > > > >>>>>>>>>>>>>>> if epoch doesn't match, and the client treats
> it
> > > as a
> > > > > > > >>>>>> fatal
> > > > > > > >>>>>>>>> error
> > > > > > > >>>>>>>>>>>> (code in
> > > > > > > >>>>>>>>>>>>>>> TransactionManager request handling).  If we
> > > consider,
> > > > > > for
> > > > > > > >>>>>>>>>> example,
> > > > > > > >>>>>>>>>>>>>>> committing a transaction that returns a
> timeout,
> > > but
> > > > > > > >>>>>>> actually
> > > > > > > >>>>>>>>>>>> succeeds,
> > > > > > > >>>>>>>>>>>>>>> trying to abort it or re-commit may result in
> > > > > > > >>>>>>> PRODUCER_FENCED
> > > > > > > >>>>>>>>>> error
> > > > > > > >>>>>>>>>>>>>>> (because of epoch bump).
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> For failed commits, specifically, we need to
> know
> > > the
> > > > > > > >>>>>> actual
> > > > > > > >>>>>>>>>> outcome,
> > > > > > > >>>>>>>>>>>>>>> because if we return an error the application
> may
> > > think
> > > > > > > >>>>>> that
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>> transaction is aborted and redo the work,
> leading
> > > to
> > > > > > > >>>>>>>> duplicates.
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> Re: overflowing epoch.  We could either do it
> on
> > > the TC
> > > > > > > >>>>>> and
> > > > > > > >>>>>>>>> return
> > > > > > > >>>>>>>>>>>> both
> > > > > > > >>>>>>>>>>>>>>> producer id and epoch (e.g. change the
> protocol),
> > > or
> > > > > > > >>>>>> signal
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>> client
> > > > > > > >>>>>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>> it needs to get a new producer id.  Checking
> for
> > > max
> > > > > > epoch
> > > > > > > >>>>>>>> could
> > > > > > > >>>>>>>>>> be a
> > > > > > > >>>>>>>>>>>>>>> reasonable signal, the value to check should
> > > probably
> > > > > be
> > > > > > > >>>>>>>> present
> > > > > > > >>>>>>>>>> in
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>> KIP
> > > > > > > >>>>>>>>>>>>>>> as this is effectively a part of the contract.
> > > Also,
> > > > > the
> > > > > > > >>>>>> TC
> > > > > > > >>>>>>>>>> should
> > > > > > > >>>>>>>>>>>>>>> probably return an error if the client didn't
> > > change
> > > > > > > >>>>>>> producer
> > > > > > > >>>>>>>> id
> > > > > > > >>>>>>>>>> after
> > > > > > > >>>>>>>>>>>>>>> hitting max epoch.
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> -Artem
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
> > > > > > > >>>>>>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>> Thanks for the discussion Artem.
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>> With respect to the handling of fenced
> > producers,
> > > we
> > > > > > > >>>>>> have
> > > > > > > >>>>>>>> some
> > > > > > > >>>>>>>>>>>> behavior
> > > > > > > >>>>>>>>>>>>>>>> already in place. As of KIP-588:
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>
> > > > > > > >>>>>>>>>
> > > > > > > >>>>>>>>
> > > > > > > >>>>>>>
> > > > > > > >>>>>>
> > > > > > > >>>>
> > > > > > > >>
> > > > > >
> > > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> > > > > > > >>>>>>>>>>>>>>>> ,
> > > > > > > >>>>>>>>>>>>>>>> we handle timeouts more gracefully. The
> producer
> > > can
> > > > > > > >>>>>>>> recover.
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>> Produce requests can also recover from epoch
> > > fencing
> > > > > by
> > > > > > > >>>>>>>>>> aborting the
> > > > > > > >>>>>>>>>>>>>>>> transaction and starting over.
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>> What other cases were you considering that
> would
> > > cause
> > > > > > > >>>>>> us
> > > > > > > >>>>>>> to
> > > > > > > >>>>>>>>>> have a
> > > > > > > >>>>>>>>>>>>>>> fenced
> > > > > > > >>>>>>>>>>>>>>>> epoch but we'd want to recover?
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>> The first point about handling epoch overflows
> > is
> > > > > fair.
> > > > > > > >>>>>> I
> > > > > > > >>>>>>>>> think
> > > > > > > >>>>>>>>>>>> there is
> > > > > > > >>>>>>>>>>>>>>>> some logic we'd need to consider. (ie, if we
> are
> > > one
> > > > > > > >>>>>> away
> > > > > > > >>>>>>>> from
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>> max
> > > > > > > >>>>>>>>>>>>>>>> epoch, we need to reset the producer ID.) I'm
> > > still
> > > > > > > >>>>>>>> wondering
> > > > > > > >>>>>>>>> if
> > > > > > > >>>>>>>>>>>> there
> > > > > > > >>>>>>>>>>>>>>> is a
> > > > > > > >>>>>>>>>>>>>>>> way to direct this from the response, or if
> > > everything
> > > > > > > >>>>>>>> should
> > > > > > > >>>>>>>>> be
> > > > > > > >>>>>>>>>>>> done on
> > > > > > > >>>>>>>>>>>>>>>> the client side. Let me know if you have any
> > > thoughts
> > > > > > > >>>>>>> here.
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>> Thanks,
> > > > > > > >>>>>>>>>>>>>>>> Justine
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>> On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
> > > > > > > >>>>>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>> There are some workflows in the client that
> are
> > > > > > > >>>>>> implied
> > > > > > > >>>>>>> by
> > > > > > > >>>>>>>>>>>> protocol
> > > > > > > >>>>>>>>>>>>>>>>> changes, e.g.:
> > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>> - for new clients, epoch changes with every
> > > > > > > >>>>>> transaction
> > > > > > > >>>>>>>> and
> > > > > > > >>>>>>>>>> can
> > > > > > > >>>>>>>>>>>>>>> overflow,
> > > > > > > >>>>>>>>>>>>>>>>> in old clients this condition was handled
> > > > > > > >>>>>> transparently,
> > > > > > > >>>>>>>>>> because
> > > > > > > >>>>>>>>>>>> epoch
> > > > > > > >>>>>>>>>>>>>>>> was
> > > > > > > >>>>>>>>>>>>>>>>> bumped in InitProducerId and it would return
> a
> > > new
> > > > > > > >>>>>>>> producer
> > > > > > > >>>>>>>>>> id if
> > > > > > > >>>>>>>>>>>>>>> epoch
> > > > > > > >>>>>>>>>>>>>>>>> overflows, the new clients would need to
> > > implement
> > > > > > > >>>>>> some
> > > > > > > >>>>>>>>>> workflow
> > > > > > > >>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>> refresh
> > > > > > > >>>>>>>>>>>>>>>>> producer id
> > > > > > > >>>>>>>>>>>>>>>>> - how to handle fenced producers, for new
> > clients
> > > > > > > >>>>>> epoch
> > > > > > > >>>>>>>>>> changes
> > > > > > > >>>>>>>>>>>> with
> > > > > > > >>>>>>>>>>>>>>>> every
> > > > > > > >>>>>>>>>>>>>>>>> transaction, so in presence of failures
> during
> > > > > > > >>>>>> commits /
> > > > > > > >>>>>>>>>> aborts,
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>> producer could get easily fenced, old clients
> > > would
> > > > > > > >>>>>>> pretty
> > > > > > > >>>>>>>>>> much
> > > > > > > >>>>>>>>>>>> would
> > > > > > > >>>>>>>>>>>>>>> get
> > > > > > > >>>>>>>>>>>>>>>>> fenced when a new incarnation of the producer
> > was
> > > > > > > >>>>>>>>> initialized
> > > > > > > >>>>>>>>>> with
> > > > > > > >>>>>>>>>>>>>>>>> InitProducerId so it's ok to treat as a fatal
> > > error,
> > > > > > > >>>>>> the
> > > > > > > >>>>>>>> new
> > > > > > > >>>>>>>>>>>> clients
> > > > > > > >>>>>>>>>>>>>>>> would
> > > > > > > >>>>>>>>>>>>>>>>> need to implement some workflow to handle
> that
> > > error,
> > > > > > > >>>>>>>>>> otherwise
> > > > > > > >>>>>>>>>>>> they
> > > > > > > >>>>>>>>>>>>>>>> could
> > > > > > > >>>>>>>>>>>>>>>>> get fenced by themselves
> > > > > > > >>>>>>>>>>>>>>>>> - in particular (as a subset of the previous
> > > issue),
> > > > > > > >>>>>>> what
> > > > > > > >>>>>>>>>> would
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>> client
> > > > > > > >>>>>>>>>>>>>>>>> do if it got a timeout during commit?  commit
> > > > > could've
> > > > > > > >>>>>>>>>> succeeded
> > > > > > > >>>>>>>>>>>> or
> > > > > > > >>>>>>>>>>>>>>>> failed
> > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>> Not sure if this has to be defined in the KIP
> > as
> > > > > > > >>>>>>>>> implementing
> > > > > > > >>>>>>>>>>>> those
> > > > > > > >>>>>>>>>>>>>>>>> probably wouldn't require protocol changes,
> but
> > > we
> > > > > > > >>>>>> have
> > > > > > > >>>>>>>>>> multiple
> > > > > > > >>>>>>>>>>>>>>>>> implementations of Kafka clients, so probably
> > > would
> > > > > be
> > > > > > > >>>>>>>> good
> > > > > > > >>>>>>>>> to
> > > > > > > >>>>>>>>>>>> have
> > > > > > > >>>>>>>>>>>>>>> some
> > > > > > > >>>>>>>>>>>>>>>>> client implementation guidance.  Could also
> be
> > > done
> > > > > > > >>>>>> as a
> > > > > > > >>>>>>>>>> separate
> > > > > > > >>>>>>>>>>>> doc.
> > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>> -Artem
> > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>> On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
> > > > > > > >>>>>>>>>>>>>>>> <jolshan@confluent.io.invalid
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>> Hey all, I've updated the KIP to incorporate
> > > Jason's
> > > > > > > >>>>>>>>>>>> suggestions.
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>
> > > > > > > >>>>>>>>>
> > > > > > > >>>>>>>>
> > > > > > > >>>>>>>
> > > > > > > >>>>>>
> > > > > > > >>>>
> > > > > > > >>
> > > > > >
> > > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>> 1. Use AddPartitionsToTxn + verify flag to
> > > check on
> > > > > > > >>>>>>> old
> > > > > > > >>>>>>>>>> clients
> > > > > > > >>>>>>>>>>>>>>>>>> 2. Updated AddPartitionsToTxn API to support
> > > > > > > >>>>>>> transaction
> > > > > > > >>>>>>>>>>>> batching
> > > > > > > >>>>>>>>>>>>>>>>>> 3. Mention IBP bump
> > > > > > > >>>>>>>>>>>>>>>>>> 4. Mention auth change on new
> > AddPartitionsToTxn
> > > > > > > >>>>>>>> version.
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>> I'm planning on opening a vote soon.
> > > > > > > >>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > >>>>>>>>>>>>>>>>>> Justine
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:32 PM Justine
> Olshan
> > <
> > > > > > > >>>>>>>>>>>> jolshan@confluent.io
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>> Thanks Jason. Those changes make sense to
> > me. I
> > > > > > > >>>>>> will
> > > > > > > >>>>>>>>>> update
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>> KIP.
> > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:31 PM Jason
> > Gustafson
> > > > > > > >>>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
> > > > > > > >>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>> Hey Justine,
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>> I was wondering about compatibility here.
> > > When
> > > > > > > >>>>>> we
> > > > > > > >>>>>>>>> send
> > > > > > > >>>>>>>>>>>> requests
> > > > > > > >>>>>>>>>>>>>>>>>>>> between brokers, we want to ensure that
> the
> > > > > > > >>>>>>> receiving
> > > > > > > >>>>>>>>>> broker
> > > > > > > >>>>>>>>>>>>>>>>> understands
> > > > > > > >>>>>>>>>>>>>>>>>>>> the request (specifically the new fields).
> > > > > > > >>>>>>> Typically
> > > > > > > >>>>>>>>>> this is
> > > > > > > >>>>>>>>>>>> done
> > > > > > > >>>>>>>>>>>>>>>> via
> > > > > > > >>>>>>>>>>>>>>>>>>>> IBP/metadata version.
> > > > > > > >>>>>>>>>>>>>>>>>>>> I'm trying to think if there is a way
> around
> > > it
> > > > > > > >>>>>> but
> > > > > > > >>>>>>>> I'm
> > > > > > > >>>>>>>>>> not
> > > > > > > >>>>>>>>>>>> sure
> > > > > > > >>>>>>>>>>>>>>>> there
> > > > > > > >>>>>>>>>>>>>>>>>> is.
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>> Yes. I think we would gate usage of this
> > > behind
> > > > > > > >>>>>> an
> > > > > > > >>>>>>>> IBP
> > > > > > > >>>>>>>>>> bump.
> > > > > > > >>>>>>>>>>>> Does
> > > > > > > >>>>>>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>>>>>>> seem
> > > > > > > >>>>>>>>>>>>>>>>>>>> reasonable?
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>> As for the improvements -- can you
> clarify
> > > how
> > > > > > > >>>>>>> the
> > > > > > > >>>>>>>>>> multiple
> > > > > > > >>>>>>>>>>>>>>>>>>>> transactional
> > > > > > > >>>>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking of
> a
> > > case
> > > > > > > >>>>>>>> where
> > > > > > > >>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>> wait/batch
> > > > > > > >>>>>>>>>>>>>>>>>>>> multiple produce requests together? My
> > > > > > > >>>>>>> understanding
> > > > > > > >>>>>>>>> for
> > > > > > > >>>>>>>>>> now
> > > > > > > >>>>>>>>>>>> was
> > > > > > > >>>>>>>>>>>>>>> 1
> > > > > > > >>>>>>>>>>>>>>>>>>>> transactional ID and one validation per 1
> > > produce
> > > > > > > >>>>>>>>>> request.
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>> Each call to `AddPartitionsToTxn` is
> > > essentially
> > > > > > > >>>>>> a
> > > > > > > >>>>>>>>> write
> > > > > > > >>>>>>>>>> to
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>>>>>>> log and must block on replication. The
> more
> > we
> > > > > > > >>>>>> can
> > > > > > > >>>>>>>> fit
> > > > > > > >>>>>>>>>> into a
> > > > > > > >>>>>>>>>>>>>>> single
> > > > > > > >>>>>>>>>>>>>>>>>>>> request, the more writes we can do in
> > > parallel.
> > > > > > > >>>>>> The
> > > > > > > >>>>>>>>>>>> alternative
> > > > > > > >>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>> make
> > > > > > > >>>>>>>>>>>>>>>>>>>> use of more connections, but usually we
> > prefer
> > > > > > > >>>>>>>> batching
> > > > > > > >>>>>>>>>>>> since the
> > > > > > > >>>>>>>>>>>>>>>>>> network
> > > > > > > >>>>>>>>>>>>>>>>>>>> stack is not really optimized for high
> > > > > > > >>>>>>>>> connection/request
> > > > > > > >>>>>>>>>>>> loads.
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>> Finally with respect to the
> > authorizations, I
> > > > > > > >>>>>>> think
> > > > > > > >>>>>>>>> it
> > > > > > > >>>>>>>>>>>> makes
> > > > > > > >>>>>>>>>>>>>>> sense
> > > > > > > >>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>> skip
> > > > > > > >>>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit
> confused
> > > by
> > > > > > > >>>>>> the
> > > > > > > >>>>>>>>>> "leader
> > > > > > > >>>>>>>>>>>> ID"
> > > > > > > >>>>>>>>>>>>>>>> field.
> > > > > > > >>>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the request
> as
> > > > > > > >>>>>> from a
> > > > > > > >>>>>>>>>> broker
> > > > > > > >>>>>>>>>>>> (does
> > > > > > > >>>>>>>>>>>>>>> it
> > > > > > > >>>>>>>>>>>>>>>>>> matter
> > > > > > > >>>>>>>>>>>>>>>>>>>> which one?).
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>> We could also make it version-based. For
> the
> > > next
> > > > > > > >>>>>>>>>> version, we
> > > > > > > >>>>>>>>>>>>>>> could
> > > > > > > >>>>>>>>>>>>>>>>>>>> require
> > > > > > > >>>>>>>>>>>>>>>>>>>> CLUSTER auth. So clients would not be able
> > to
> > > use
> > > > > > > >>>>>>> the
> > > > > > > >>>>>>>>> API
> > > > > > > >>>>>>>>>>>>>>> anymore,
> > > > > > > >>>>>>>>>>>>>>>>> which
> > > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>> probably what we want.
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>> -Jason
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:43 AM Justine
> > Olshan
> > > > > > > >>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>> As a follow up, I was just thinking about
> > the
> > > > > > > >>>>>>>>> batching
> > > > > > > >>>>>>>>>> a
> > > > > > > >>>>>>>>>>>> bit
> > > > > > > >>>>>>>>>>>>>>> more.
> > > > > > > >>>>>>>>>>>>>>>>>>>>> I suppose if we have one request in
> flight
> > > and
> > > > > > > >>>>>> we
> > > > > > > >>>>>>>>>> queue up
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>> other
> > > > > > > >>>>>>>>>>>>>>>>>>>>> produce requests in some sort of
> purgatory,
> > > we
> > > > > > > >>>>>>>> could
> > > > > > > >>>>>>>>>> send
> > > > > > > >>>>>>>>>>>>>>>>> information
> > > > > > > >>>>>>>>>>>>>>>>>>>> out
> > > > > > > >>>>>>>>>>>>>>>>>>>>> for all of them rather than one by one.
> So
> > > that
> > > > > > > >>>>>>>> would
> > > > > > > >>>>>>>>>> be a
> > > > > > > >>>>>>>>>>>>>>> benefit
> > > > > > > >>>>>>>>>>>>>>>>> of
> > > > > > > >>>>>>>>>>>>>>>>>>>>> batching partitions to add per
> transaction.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>> I'll need to think a bit more on the
> design
> > > of
> > > > > > > >>>>>>> this
> > > > > > > >>>>>>>>>> part
> > > > > > > >>>>>>>>>>>> of the
> > > > > > > >>>>>>>>>>>>>>>> KIP,
> > > > > > > >>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>> will update the KIP in the next few days.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > >>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:22 AM Justine
> > > Olshan
> > > > > > > >>>>>> <
> > > > > > > >>>>>>>>>>>>>>>>> jolshan@confluent.io>
> > > > > > > >>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> Hey Jason -- thanks for the input -- I
> was
> > > > > > > >>>>>> just
> > > > > > > >>>>>>>>>> digging
> > > > > > > >>>>>>>>>>>> a bit
> > > > > > > >>>>>>>>>>>>>>>>> deeper
> > > > > > > >>>>>>>>>>>>>>>>>>>> into
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> the design + implementation of the
> > > validation
> > > > > > > >>>>>>>> calls
> > > > > > > >>>>>>>>>> here
> > > > > > > >>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>> what
> > > > > > > >>>>>>>>>>>>>>>>>> you
> > > > > > > >>>>>>>>>>>>>>>>>>>> say
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> makes sense.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> I was wondering about compatibility
> here.
> > > > > > > >>>>>> When
> > > > > > > >>>>>>> we
> > > > > > > >>>>>>>>>> send
> > > > > > > >>>>>>>>>>>>>>> requests
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> between brokers, we want to ensure that
> > the
> > > > > > > >>>>>>>>> receiving
> > > > > > > >>>>>>>>>>>> broker
> > > > > > > >>>>>>>>>>>>>>>>>>>> understands
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> the request (specifically the new
> fields).
> > > > > > > >>>>>>>>> Typically
> > > > > > > >>>>>>>>>>>> this is
> > > > > > > >>>>>>>>>>>>>>>> done
> > > > > > > >>>>>>>>>>>>>>>>>> via
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> IBP/metadata version.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> I'm trying to think if there is a way
> > around
> > > > > > > >>>>>> it
> > > > > > > >>>>>>>> but
> > > > > > > >>>>>>>>>> I'm
> > > > > > > >>>>>>>>>>>> not
> > > > > > > >>>>>>>>>>>>>>> sure
> > > > > > > >>>>>>>>>>>>>>>>>> there
> > > > > > > >>>>>>>>>>>>>>>>>>>>> is.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> As for the improvements -- can you
> clarify
> > > > > > > >>>>>> how
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>>>> multiple
> > > > > > > >>>>>>>>>>>>>>>>>>>> transactional
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking
> of
> > a
> > > > > > > >>>>>>> case
> > > > > > > >>>>>>>>>> where we
> > > > > > > >>>>>>>>>>>>>>>>> wait/batch
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> multiple produce requests together? My
> > > > > > > >>>>>>>>> understanding
> > > > > > > >>>>>>>>>> for
> > > > > > > >>>>>>>>>>>> now
> > > > > > > >>>>>>>>>>>>>>>> was 1
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> transactional ID and one validation per
> 1
> > > > > > > >>>>>>> produce
> > > > > > > >>>>>>>>>>>> request.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> Finally with respect to the
> > authorizations,
> > > I
> > > > > > > >>>>>>>> think
> > > > > > > >>>>>>>>>> it
> > > > > > > >>>>>>>>>>>> makes
> > > > > > > >>>>>>>>>>>>>>>> sense
> > > > > > > >>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>> skip
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit
> > confused
> > > > > > > >>>>>> by
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>>>> "leader
> > > > > > > >>>>>>>>>>>>>>> ID"
> > > > > > > >>>>>>>>>>>>>>>>>> field.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the
> request
> > as
> > > > > > > >>>>>>>> from a
> > > > > > > >>>>>>>>>>>> broker
> > > > > > > >>>>>>>>>>>>>>> (does
> > > > > > > >>>>>>>>>>>>>>>>> it
> > > > > > > >>>>>>>>>>>>>>>>>>>>> matter
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> which one?).
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> I think I want to adopt these
> suggestions,
> > > > > > > >>>>>> just
> > > > > > > >>>>>>>> had
> > > > > > > >>>>>>>>>> a few
> > > > > > > >>>>>>>>>>>>>>>>> questions
> > > > > > > >>>>>>>>>>>>>>>>>> on
> > > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> details.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> On Thu, Jan 5, 2023 at 5:05 PM Jason
> > > > > > > >>>>>> Gustafson
> > > > > > > >>>>>>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Thanks for the proposal.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> I was thinking about the
> implementation a
> > > > > > > >>>>>>> little
> > > > > > > >>>>>>>>>> bit.
> > > > > > > >>>>>>>>>>>> In the
> > > > > > > >>>>>>>>>>>>>>>>>> current
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> proposal, the behavior depends on
> whether
> > > we
> > > > > > > >>>>>>>> have
> > > > > > > >>>>>>>>> an
> > > > > > > >>>>>>>>>>>> old or
> > > > > > > >>>>>>>>>>>>>>> new
> > > > > > > >>>>>>>>>>>>>>>>>>>> client.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> For
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> old clients, we send
> > `DescribeTransactions`
> > > > > > > >>>>>>> and
> > > > > > > >>>>>>>>>> verify
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>> result
> > > > > > > >>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> new clients, we send
> > `AddPartitionsToTxn`.
> > > > > > > >>>>>> We
> > > > > > > >>>>>>>>> might
> > > > > > > >>>>>>>>>> be
> > > > > > > >>>>>>>>>>>> able
> > > > > > > >>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>> simplify
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> implementation if we can use the same
> > > > > > > >>>>>> request
> > > > > > > >>>>>>>>> type.
> > > > > > > >>>>>>>>>> For
> > > > > > > >>>>>>>>>>>>>>>> example,
> > > > > > > >>>>>>>>>>>>>>>>>>>> what if
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> bump the protocol version for
> > > > > > > >>>>>>>> `AddPartitionsToTxn`
> > > > > > > >>>>>>>>>> and
> > > > > > > >>>>>>>>>>>> add a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> `validateOnly`
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> flag? For older versions, we can set
> > > > > > > >>>>>>>>>>>> `validateOnly=true` so
> > > > > > > >>>>>>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> request only returns successfully if
> the
> > > > > > > >>>>>>>> partition
> > > > > > > >>>>>>>>>> had
> > > > > > > >>>>>>>>>>>>>>> already
> > > > > > > >>>>>>>>>>>>>>>>> been
> > > > > > > >>>>>>>>>>>>>>>>>>>>> added.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> For new versions, we can set
> > > > > > > >>>>>>>> `validateOnly=false`
> > > > > > > >>>>>>>>>> and
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>> partition
> > > > > > > >>>>>>>>>>>>>>>>>>>> will
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> added to the transaction. The other
> > > slightly
> > > > > > > >>>>>>>>>> annoying
> > > > > > > >>>>>>>>>>>> thing
> > > > > > > >>>>>>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> get around is the need to collect the
> > > > > > > >>>>>>>> transaction
> > > > > > > >>>>>>>>>> state
> > > > > > > >>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>> all
> > > > > > > >>>>>>>>>>>>>>>>>>>>> partitions
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> even when we only care about a subset.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Some additional improvements to
> consider:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> - We can give `AddPartitionsToTxn`
> better
> > > > > > > >>>>>>> batch
> > > > > > > >>>>>>>>>> support
> > > > > > > >>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>>>> inter-broker
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> usage. Currently we only allow one
> > > > > > > >>>>>>>>>> `TransactionalId` to
> > > > > > > >>>>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>>>>> specified,
> > > > > > > >>>>>>>>>>>>>>>>>>>>> but
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the broker may get some benefit being
> > able
> > > > > > > >>>>>> to
> > > > > > > >>>>>>>>> batch
> > > > > > > >>>>>>>>>>>> across
> > > > > > > >>>>>>>>>>>>>>>>> multiple
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> transactions.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> - Another small improvement is skipping
> > > > > > > >>>>>> topic
> > > > > > > >>>>>>>>>>>> authorization
> > > > > > > >>>>>>>>>>>>>>>>> checks
> > > > > > > >>>>>>>>>>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> `AddPartitionsToTxn` when the request
> is
> > > > > > > >>>>>> from
> > > > > > > >>>>>>> a
> > > > > > > >>>>>>>>>> broker.
> > > > > > > >>>>>>>>>>>>>>> Perhaps
> > > > > > > >>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>> can
> > > > > > > >>>>>>>>>>>>>>>>>>>>> add
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> a field for the `LeaderId` or something
> > > like
> > > > > > > >>>>>>>> that
> > > > > > > >>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>> require
> > > > > > > >>>>>>>>>>>>>>>>>> CLUSTER
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> permission when set.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Best,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Jason
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
> > > > > > > >>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation. It makes
> > sense
> > > > > > > >>>>>>> to
> > > > > > > >>>>>>>> me
> > > > > > > >>>>>>>>>> now.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 1:42 PM
> Justine
> > > > > > > >>>>>>> Olshan
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> My understanding of the mechanism is
> > > > > > > >>>>>> that
> > > > > > > >>>>>>>> when
> > > > > > > >>>>>>>>>> we
> > > > > > > >>>>>>>>>>>> get to
> > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>> last
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> epoch,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> increment to the fencing/last epoch
> and
> > > > > > > >>>>>> if
> > > > > > > >>>>>>>> any
> > > > > > > >>>>>>>>>>>> further
> > > > > > > >>>>>>>>>>>>>>>>> requests
> > > > > > > >>>>>>>>>>>>>>>>>>>> come
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> this producer ID they are fenced.
> Then
> > > > > > > >>>>>> the
> > > > > > > >>>>>>>>>> producer
> > > > > > > >>>>>>>>>>>>>>> gets a
> > > > > > > >>>>>>>>>>>>>>>>> new
> > > > > > > >>>>>>>>>>>>>>>>>> ID
> > > > > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> restarts with epoch/sequence 0. The
> > > > > > > >>>>>> fenced
> > > > > > > >>>>>>>>> epoch
> > > > > > > >>>>>>>>>>>> sticks
> > > > > > > >>>>>>>>>>>>>>>>> around
> > > > > > > >>>>>>>>>>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> duration of
> producer.id.expiration.ms
> > > > > > > >>>>>> and
> > > > > > > >>>>>>>>>> blocks
> > > > > > > >>>>>>>>>>>> any
> > > > > > > >>>>>>>>>>>>>>> late
> > > > > > > >>>>>>>>>>>>>>>>>>>> messages
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> there.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> The new ID will get to take advantage
> > of
> > > > > > > >>>>>>> the
> > > > > > > >>>>>>>>>>>> improved
> > > > > > > >>>>>>>>>>>>>>>>> semantics
> > > > > > > >>>>>>>>>>>>>>>>>>>>> around
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> non-zero start sequences. So I think
> we
> > > > > > > >>>>>>> are
> > > > > > > >>>>>>>>>> covered.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> The only potential issue is
> overloading
> > > > > > > >>>>>>> the
> > > > > > > >>>>>>>>>> cache,
> > > > > > > >>>>>>>>>>>> but
> > > > > > > >>>>>>>>>>>>>>>>>> hopefully
> > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> improvements (lowered
> > > > > > > >>>>>>>>> producer.id.expiration.ms
> > > > > > > >>>>>>>>>> )
> > > > > > > >>>>>>>>>>>> will
> > > > > > > >>>>>>>>>>>>>>> help
> > > > > > > >>>>>>>>>>>>>>>>>> with
> > > > > > > >>>>>>>>>>>>>>>>>>>>> that.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Let
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> me know if you still have concerns.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 10:24 AM Jun
> > Rao
> > > > > > > >>>>>>>>>>>>>>>>>>>> <ju...@confluent.io.invalid>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> 70. The proposed fencing logic
> doesn't
> > > > > > > >>>>>>>> apply
> > > > > > > >>>>>>>>>> when
> > > > > > > >>>>>>>>>>>> pid
> > > > > > > >>>>>>>>>>>>>>>>>> changes,
> > > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> right? If so, I am not sure how
> > > > > > > >>>>>> complete
> > > > > > > >>>>>>>> we
> > > > > > > >>>>>>>>>> are
> > > > > > > >>>>>>>>>>>>>>>> addressing
> > > > > > > >>>>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> issue
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the pid changes more frequently.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 16, 2022 at 9:16 AM
> > > > > > > >>>>>> Justine
> > > > > > > >>>>>>>>> Olshan
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for replying!
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> 70.We already do the overflow
> > > > > > > >>>>>>> mechanism,
> > > > > > > >>>>>>>>> so
> > > > > > > >>>>>>>>>> my
> > > > > > > >>>>>>>>>>>>>>> change
> > > > > > > >>>>>>>>>>>>>>>>> would
> > > > > > > >>>>>>>>>>>>>>>>>>>> just
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> make
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> happen more often.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I was also not suggesting a new
> > > > > > > >>>>>> field
> > > > > > > >>>>>>> in
> > > > > > > >>>>>>>>> the
> > > > > > > >>>>>>>>>>>> log,
> > > > > > > >>>>>>>>>>>>>>> but
> > > > > > > >>>>>>>>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> response,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> which would be gated by the client
> > > > > > > >>>>>>>>> version.
> > > > > > > >>>>>>>>>>>> Sorry if
> > > > > > > >>>>>>>>>>>>>>>>>>>> something
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> there
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> unclear. I think we are starting to
> > > > > > > >>>>>>>>> diverge.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> The goal of this KIP is to not
> > > > > > > >>>>>> change
> > > > > > > >>>>>>> to
> > > > > > > >>>>>>>>> the
> > > > > > > >>>>>>>>>>>> marker
> > > > > > > >>>>>>>>>>>>>>>>> format
> > > > > > > >>>>>>>>>>>>>>>>>> at
> > > > > > > >>>>>>>>>>>>>>>>>>>>> all.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> 71. Yes, I guess I was going under
> > > > > > > >>>>>> the
> > > > > > > >>>>>>>>>>>> assumption
> > > > > > > >>>>>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>> log
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> just
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> look at its last epoch and treat it
> > > > > > > >>>>>> as
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>>>> current
> > > > > > > >>>>>>>>>>>>>>>>> epoch. I
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> suppose
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> have some special logic that if the
> > > > > > > >>>>>>> last
> > > > > > > >>>>>>>>>> epoch
> > > > > > > >>>>>>>>>>>> was
> > > > > > > >>>>>>>>>>>>>>> on a
> > > > > > > >>>>>>>>>>>>>>>>>>>> marker
> > > > > > > >>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> actually
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> expect the next epoch or something
> > > > > > > >>>>>>> like
> > > > > > > >>>>>>>>>> that. We
> > > > > > > >>>>>>>>>>>>>>> just
> > > > > > > >>>>>>>>>>>>>>>>> need
> > > > > > > >>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> distinguish
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> based on whether we had a
> > > > > > > >>>>>> commit/abort
> > > > > > > >>>>>>>>>> marker.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> 72.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> if the producer epoch hasn't been
> > > > > > > >>>>>>>> bumped
> > > > > > > >>>>>>>>>> on
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
> > > > > > > >>>>>>>> message
> > > > > > > >>>>>>>>>> will
> > > > > > > >>>>>>>>>>>> fail
> > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>> sequence
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> validation
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the
> producer
> > > > > > > >>>>>>>> epoch
> > > > > > > >>>>>>>>>> has
> > > > > > > >>>>>>>>>>>> been
> > > > > > > >>>>>>>>>>>>>>>>> bumped,
> > > > > > > >>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> ignore
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck
> message
> > > > > > > >>>>>>>> could
> > > > > > > >>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>> appended
> > > > > > > >>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>> log.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> So,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
> > > > > > > >>>>>> guard?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I follow that "the
> > > > > > > >>>>>>> message
> > > > > > > >>>>>>>>> will
> > > > > > > >>>>>>>>>>>> fail
> > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>> sequence
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> validation".
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> In some of these cases, we had an
> > > > > > > >>>>>>> abort
> > > > > > > >>>>>>>>>> marker
> > > > > > > >>>>>>>>>>>> (due
> > > > > > > >>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>> an
> > > > > > > >>>>>>>>>>>>>>>>>>>> error)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> then
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the late message comes in with the
> > > > > > > >>>>>>>> correct
> > > > > > > >>>>>>>>>>>> sequence
> > > > > > > >>>>>>>>>>>>>>>>> number.
> > > > > > > >>>>>>>>>>>>>>>>>>>> This
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> is a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> case
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> covered by the KIP.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> The latter case is actually not
> > > > > > > >>>>>>>> something
> > > > > > > >>>>>>>>>> we've
> > > > > > > >>>>>>>>>>>>>>>>> considered
> > > > > > > >>>>>>>>>>>>>>>>>>>>> here. I
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> think
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> generally when we bump the epoch,
> we
> > > > > > > >>>>>>> are
> > > > > > > >>>>>>>>>>>> accepting
> > > > > > > >>>>>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> sequence
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> does
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> not need to be checked anymore. My
> > > > > > > >>>>>>>>>>>> understanding is
> > > > > > > >>>>>>>>>>>>>>>> also
> > > > > > > >>>>>>>>>>>>>>>>>>>> that we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> typically bump epoch mid
> transaction
> > > > > > > >>>>>>>>> (based
> > > > > > > >>>>>>>>>> on a
> > > > > > > >>>>>>>>>>>>>>> quick
> > > > > > > >>>>>>>>>>>>>>>>> look
> > > > > > > >>>>>>>>>>>>>>>>>>>> at
> > > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> code)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but let me know if that is the
> case.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 15, 2022 at 12:23 PM
> Jun
> > > > > > > >>>>>>> Rao
> > > > > > > >>>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the reply.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Assigning a new pid on int
> > > > > > > >>>>>>>> overflow
> > > > > > > >>>>>>>>>> seems
> > > > > > > >>>>>>>>>>>> a
> > > > > > > >>>>>>>>>>>>>>> bit
> > > > > > > >>>>>>>>>>>>>>>>>> hacky.
> > > > > > > >>>>>>>>>>>>>>>>>>>> If
> > > > > > > >>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> need a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> level id, it will be better to
> > > > > > > >>>>>> model
> > > > > > > >>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>> explicitly.
> > > > > > > >>>>>>>>>>>>>>>>>>>> Adding a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> field
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> would require a bit more work
> > > > > > > >>>>>> since
> > > > > > > >>>>>>> it
> > > > > > > >>>>>>>>>>>> requires a
> > > > > > > >>>>>>>>>>>>>>> new
> > > > > > > >>>>>>>>>>>>>>>>> txn
> > > > > > > >>>>>>>>>>>>>>>>>>>>> marker
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> format
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the log. So, we probably need to
> > > > > > > >>>>>>> guard
> > > > > > > >>>>>>>>> it
> > > > > > > >>>>>>>>>>>> with an
> > > > > > > >>>>>>>>>>>>>>> IBP
> > > > > > > >>>>>>>>>>>>>>>>> or
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> metadata
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> version
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and document the impact on
> > > > > > > >>>>>> downgrade
> > > > > > > >>>>>>>>> once
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>> new
> > > > > > > >>>>>>>>>>>>>>>>> format
> > > > > > > >>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> written
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 71. Hmm, once the marker is
> > > > > > > >>>>>> written,
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>>>> partition
> > > > > > > >>>>>>>>>>>>>>>> will
> > > > > > > >>>>>>>>>>>>>>>>>>>> expect
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> next
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> append to be on the next epoch.
> > > > > > > >>>>>> Does
> > > > > > > >>>>>>>>> that
> > > > > > > >>>>>>>>>>>> cover
> > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>> case
> > > > > > > >>>>>>>>>>>>>>>>>>>> you
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> mentioned?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 72. Also, just to be clear on the
> > > > > > > >>>>>>>>> stucked
> > > > > > > >>>>>>>>>>>> message
> > > > > > > >>>>>>>>>>>>>>>> issue
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> described
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> motivation. With EoS, we also
> > > > > > > >>>>>>> validate
> > > > > > > >>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>> sequence
> > > > > > > >>>>>>>>>>>>>>>> id
> > > > > > > >>>>>>>>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> idempotency.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> So,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> with the current logic, if the
> > > > > > > >>>>>>>> producer
> > > > > > > >>>>>>>>>> epoch
> > > > > > > >>>>>>>>>>>>>>> hasn't
> > > > > > > >>>>>>>>>>>>>>>>> been
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> bumped on
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
> > > > > > > >>>>>>>>> message
> > > > > > > >>>>>>>>>> will
> > > > > > > >>>>>>>>>>>>>>> fail
> > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>> sequence
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> validation
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the
> > > > > > > >>>>>> producer
> > > > > > > >>>>>>>>>> epoch has
> > > > > > > >>>>>>>>>>>>>>> been
> > > > > > > >>>>>>>>>>>>>>>>>>>> bumped, we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> ignore
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck
> > > > > > > >>>>>> message
> > > > > > > >>>>>>>>>> could be
> > > > > > > >>>>>>>>>>>>>>>> appended
> > > > > > > >>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> log.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> So,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
> > > > > > > >>>>>>> guard?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 10:44 AM
> > > > > > > >>>>>>>> Justine
> > > > > > > >>>>>>>>>>>> Olshan
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > > > >>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias — thanks again for
> > > > > > > >>>>>> taking
> > > > > > > >>>>>>>>> time
> > > > > > > >>>>>>>>>> to
> > > > > > > >>>>>>>>>>>> look
> > > > > > > >>>>>>>>>>>>>>> a
> > > > > > > >>>>>>>>>>>>>>>>> this.
> > > > > > > >>>>>>>>>>>>>>>>>>>> You
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> said:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My proposal was only focusing
> > > > > > > >>>>>> to
> > > > > > > >>>>>>>>> avoid
> > > > > > > >>>>>>>>>>>>>>> dangling
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
> > > > > > > >>>>>> added
> > > > > > > >>>>>>>>>> without
> > > > > > > >>>>>>>>>>>>>>>> registered
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> partition.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> --
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more details
> > > > > > > >>>>>> to
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>> KIP
> > > > > > > >>>>>>>>>>>> about
> > > > > > > >>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>>> scenario
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> better
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I understand what
> > > > > > > >>>>>> you
> > > > > > > >>>>>>>>> mean
> > > > > > > >>>>>>>>>>>> here.
> > > > > > > >>>>>>>>>>>>>>> The
> > > > > > > >>>>>>>>>>>>>>>>>>>> motivation
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> section
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> describes two scenarios about
> > > > > > > >>>>>> how
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>> record
> > > > > > > >>>>>>>>>>>>>>> can be
> > > > > > > >>>>>>>>>>>>>>>>>> added
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> without a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> registered partition:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Another way hanging
> > > > > > > >>>>>> transactions
> > > > > > > >>>>>>>> can
> > > > > > > >>>>>>>>>>>> occur is
> > > > > > > >>>>>>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>>>> a
> > > > > > > >>>>>>>>>>>>>>>>>>>>> client
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> buggy
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> may somehow try to write to a
> > > > > > > >>>>>>>>> partition
> > > > > > > >>>>>>>>>>>> before
> > > > > > > >>>>>>>>>>>>>>> it
> > > > > > > >>>>>>>>>>>>>>>>> adds
> > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> partition
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> For the first example of this
> > > > > > > >>>>>>> would
> > > > > > > >>>>>>>> it
> > > > > > > >>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>> helpful
> > > > > > > >>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>> say
> > > > > > > >>>>>>>>>>>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> comes in after the abort, but
> > > > > > > >>>>>>> before
> > > > > > > >>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>> partition
> > > > > > > >>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>> added
> > > > > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> next
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction so it becomes
> > > > > > > >>>>>>> "hanging."
> > > > > > > >>>>>>>>>>>> Perhaps the
> > > > > > > >>>>>>>>>>>>>>>> next
> > > > > > > >>>>>>>>>>>>>>>>>>>>> sentence
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> describing
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the message becoming part of the
> > > > > > > >>>>>>>> next
> > > > > > > >>>>>>>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>>> (a
> > > > > > > >>>>>>>>>>>>>>>>>>>>> different
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> case)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> was
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not properly differentiated.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun — thanks for reading the
> > > > > > > >>>>>> KIP.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. The int typing was a
> > > > > > > >>>>>> concern.
> > > > > > > >>>>>>>>>> Currently
> > > > > > > >>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>> have a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> mechanism
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> place
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> fence the final epoch when the
> > > > > > > >>>>>>> epoch
> > > > > > > >>>>>>>>> is
> > > > > > > >>>>>>>>>>>> about to
> > > > > > > >>>>>>>>>>>>>>>>>> overflow
> > > > > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> assign
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer ID with epoch 0. Of
> > > > > > > >>>>>>> course,
> > > > > > > >>>>>>>>>> this
> > > > > > > >>>>>>>>>>>> is a
> > > > > > > >>>>>>>>>>>>>>> bit
> > > > > > > >>>>>>>>>>>>>>>>>> tricky
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> when it
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> comes
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response back to the client.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Making this a long could be
> > > > > > > >>>>>>> another
> > > > > > > >>>>>>>>>> option,
> > > > > > > >>>>>>>>>>>> but
> > > > > > > >>>>>>>>>>>>>>> I
> > > > > > > >>>>>>>>>>>>>>>>>> wonder
> > > > > > > >>>>>>>>>>>>>>>>>>>> are
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> there
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> implications on changing this
> > > > > > > >>>>>>> field
> > > > > > > >>>>>>>> if
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>> epoch is
> > > > > > > >>>>>>>>>>>>>>>>>>>>> persisted
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> disk?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to check the usages.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 71.This was something Matthias
> > > > > > > >>>>>>> asked
> > > > > > > >>>>>>>>>> about
> > > > > > > >>>>>>>>>>>> as
> > > > > > > >>>>>>>>>>>>>>>> well. I
> > > > > > > >>>>>>>>>>>>>>>>>> was
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> considering a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible edge case where a
> > > > > > > >>>>>> produce
> > > > > > > >>>>>>>>>> request
> > > > > > > >>>>>>>>>>>> from
> > > > > > > >>>>>>>>>>>>>>> a
> > > > > > > >>>>>>>>>>>>>>>> new
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> somehow
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> gets sent right after the marker
> > > > > > > >>>>>>> is
> > > > > > > >>>>>>>>>>>> written, but
> > > > > > > >>>>>>>>>>>>>>>>> before
> > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> alerted of the newly bumped
> > > > > > > >>>>>> epoch.
> > > > > > > >>>>>>>> In
> > > > > > > >>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>> case, we
> > > > > > > >>>>>>>>>>>>>>>>> may
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> include
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> record
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we don't want to. I suppose
> > > > > > > >>>>>>> we
> > > > > > > >>>>>>>>>> could
> > > > > > > >>>>>>>>>>>> try
> > > > > > > >>>>>>>>>>>>>>> to do
> > > > > > > >>>>>>>>>>>>>>>>>>>>> something
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> client
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> side
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to bump the epoch after sending
> > > > > > > >>>>>> an
> > > > > > > >>>>>>>>>> endTxn as
> > > > > > > >>>>>>>>>>>>>>> well
> > > > > > > >>>>>>>>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> scenario
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> —
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> I
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wonder how it would work when
> > > > > > > >>>>>> the
> > > > > > > >>>>>>>>>> server is
> > > > > > > >>>>>>>>>>>>>>>> aborting
> > > > > > > >>>>>>>>>>>>>>>>>>>> based
> > > > > > > >>>>>>>>>>>>>>>>>>>>> on
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> error. I could also be missing
> > > > > > > >>>>>>>>>> something and
> > > > > > > >>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>> scenario
> > > > > > > >>>>>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> actually
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again to everyone reading
> > > > > > > >>>>>>> and
> > > > > > > >>>>>>>>>>>> commenting.
> > > > > > > >>>>>>>>>>>>>>>> Let
> > > > > > > >>>>>>>>>>>>>>>>> me
> > > > > > > >>>>>>>>>>>>>>>>>>>> know
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> about
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> further questions or comments.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 9:41 AM
> > > > > > > >>>>>>> Jun
> > > > > > > >>>>>>>>> Rao
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP. A couple
> > > > > > > >>>>>> of
> > > > > > > >>>>>>>>>> comments.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Currently, the producer
> > > > > > > >>>>>>> epoch
> > > > > > > >>>>>>>> is
> > > > > > > >>>>>>>>>> an
> > > > > > > >>>>>>>>>>>> int.
> > > > > > > >>>>>>>>>>>>>>> I am
> > > > > > > >>>>>>>>>>>>>>>>> not
> > > > > > > >>>>>>>>>>>>>>>>>>>> sure
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> it's
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> enough
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to accommodate all
> > > > > > > >>>>>> transactions
> > > > > > > >>>>>>> in
> > > > > > > >>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>> lifetime
> > > > > > > >>>>>>>>>>>>>>>> of
> > > > > > > >>>>>>>>>>>>>>>>> a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> producer.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Should
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> change that to a long or add a
> > > > > > > >>>>>>> new
> > > > > > > >>>>>>>>>> long
> > > > > > > >>>>>>>>>>>> field
> > > > > > > >>>>>>>>>>>>>>>> like
> > > > > > > >>>>>>>>>>>>>>>>>>>> txnId?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 71. "it will write the prepare
> > > > > > > >>>>>>>>> commit
> > > > > > > >>>>>>>>>>>> message
> > > > > > > >>>>>>>>>>>>>>>> with
> > > > > > > >>>>>>>>>>>>>>>>> a
> > > > > > > >>>>>>>>>>>>>>>>>>>>> bumped
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> send
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WriteTxnMarkerRequests with
> > > > > > > >>>>>> the
> > > > > > > >>>>>>>>> bumped
> > > > > > > >>>>>>>>>>>> epoch."
> > > > > > > >>>>>>>>>>>>>>>> Hmm,
> > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> associated
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the current txn right?
> > > > > > > >>>>>> So,
> > > > > > > >>>>>>> it
> > > > > > > >>>>>>>>>> seems
> > > > > > > >>>>>>>>>>>>>>> weird to
> > > > > > > >>>>>>>>>>>>>>>>>>>> write a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> commit
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with a bumped epoch. Should we
> > > > > > > >>>>>>>> only
> > > > > > > >>>>>>>>>> bump
> > > > > > > >>>>>>>>>>>> up
> > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>> epoch
> > > > > > > >>>>>>>>>>>>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> EndTxnResponse
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rename the field to sth like
> > > > > > > >>>>>>>>>>>>>>> nextProducerEpoch?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 12, 2022 at 8:54
> > > > > > > >>>>>> PM
> > > > > > > >>>>>>>>>> Matthias
> > > > > > > >>>>>>>>>>>> J.
> > > > > > > >>>>>>>>>>>>>>> Sax <
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the background.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30: SGTM. My proposal was
> > > > > > > >>>>>>>> only
> > > > > > > >>>>>>>>>>>> focusing
> > > > > > > >>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>> avoid
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> dangling
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
> > > > > > > >>>>>>>> added
> > > > > > > >>>>>>>>>>>> without
> > > > > > > >>>>>>>>>>>>>>>>>> registered
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> partition.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> --
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more
> > > > > > > >>>>>> details
> > > > > > > >>>>>>>> to
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>> KIP
> > > > > > > >>>>>>>>>>>>>>> about
> > > > > > > >>>>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> scenario
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40: I think you hit a fair
> > > > > > > >>>>>>> point
> > > > > > > >>>>>>>>>> about
> > > > > > > >>>>>>>>>>>> race
> > > > > > > >>>>>>>>>>>>>>>>>>>> conditions
> > > > > > > >>>>>>>>>>>>>>>>>>>>> or
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> client
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> bugs
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (incorrectly not bumping the
> > > > > > > >>>>>>>>>> epoch). The
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> complexity/confusion
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> using
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the bumped epoch I see, is
> > > > > > > >>>>>>>> mainly
> > > > > > > >>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>> internal
> > > > > > > >>>>>>>>>>>>>>>>>>>>> debugging,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> ie,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> inspecting
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log segment dumps -- it
> > > > > > > >>>>>> seems
> > > > > > > >>>>>>>>>> harder to
> > > > > > > >>>>>>>>>>>>>>> reason
> > > > > > > >>>>>>>>>>>>>>>>>> about
> > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> system
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> us
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> humans. But if we get better
> > > > > > > >>>>>>>>>>>> guarantees, it
> > > > > > > >>>>>>>>>>>>>>>> would
> > > > > > > >>>>>>>>>>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> worth to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> use
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped epoch.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60: as I mentioned already,
> > > > > > > >>>>>> I
> > > > > > > >>>>>>>>> don't
> > > > > > > >>>>>>>>>>>> know the
> > > > > > > >>>>>>>>>>>>>>>>> broker
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> internals
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> provide
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more input. So if nobody
> > > > > > > >>>>>> else
> > > > > > > >>>>>>>>> chimes
> > > > > > > >>>>>>>>>>>> in, we
> > > > > > > >>>>>>>>>>>>>>>>> should
> > > > > > > >>>>>>>>>>>>>>>>>>>> just
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> move
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> forward
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with your proposal.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 12/6/22 4:22 PM, Justine
> > > > > > > >>>>>>>> Olshan
> > > > > > > >>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi all,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> After Artem's questions
> > > > > > > >>>>>>> about
> > > > > > > >>>>>>>>>> error
> > > > > > > >>>>>>>>>>>>>>> behavior,
> > > > > > > >>>>>>>>>>>>>>>>>> I've
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> re-evaluated
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unknown producer ID
> > > > > > > >>>>>>> exception
> > > > > > > >>>>>>>>> and
> > > > > > > >>>>>>>>>> had
> > > > > > > >>>>>>>>>>>> some
> > > > > > > >>>>>>>>>>>>>>>>>>>> discussions
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> offline.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think generally it makes
> > > > > > > >>>>>>>> sense
> > > > > > > >>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>> simplify
> > > > > > > >>>>>>>>>>>>>>>>>> error
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> handling
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> cases
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this and the
> > > > > > > >>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > >>>>>>>>>> error
> > > > > > > >>>>>>>>>>>>>>> has a
> > > > > > > >>>>>>>>>>>>>>>>>> pretty
> > > > > > > >>>>>>>>>>>>>>>>>>>>> long
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complicated
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> history. Because of this,
> > > > > > > >>>>>> I
> > > > > > > >>>>>>>>>> propose
> > > > > > > >>>>>>>>>>>>>>> adding a
> > > > > > > >>>>>>>>>>>>>>>>> new
> > > > > > > >>>>>>>>>>>>>>>>>>>> error
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> code
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ABORTABLE_ERROR
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that when encountered by
> > > > > > > >>>>>> new
> > > > > > > >>>>>>>>>> clients
> > > > > > > >>>>>>>>>>>>>>> (gated
> > > > > > > >>>>>>>>>>>>>>>> by
> > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> produce
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will simply abort the
> > > > > > > >>>>>>>>> transaction.
> > > > > > > >>>>>>>>>>>> This
> > > > > > > >>>>>>>>>>>>>>>> allows
> > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> server
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> some
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> say
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in whether the client
> > > > > > > >>>>>> aborts
> > > > > > > >>>>>>>> and
> > > > > > > >>>>>>>>>> makes
> > > > > > > >>>>>>>>>>>>>>>> handling
> > > > > > > >>>>>>>>>>>>>>>>>>>> much
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> simpler.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> future, we can also use
> > > > > > > >>>>>> this
> > > > > > > >>>>>>>>>> error in
> > > > > > > >>>>>>>>>>>>>>> other
> > > > > > > >>>>>>>>>>>>>>>>>>>> situations
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> where
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> want
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> abort the transactions. We
> > > > > > > >>>>>>> can
> > > > > > > >>>>>>>>>> even
> > > > > > > >>>>>>>>>>>> use on
> > > > > > > >>>>>>>>>>>>>>>>> other
> > > > > > > >>>>>>>>>>>>>>>>>>>> apis.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've added this to the
> > > > > > > >>>>>> KIP.
> > > > > > > >>>>>>>> Let
> > > > > > > >>>>>>>>> me
> > > > > > > >>>>>>>>>>>> know if
> > > > > > > >>>>>>>>>>>>>>>>> there
> > > > > > > >>>>>>>>>>>>>>>>>>>> are
> > > > > > > >>>>>>>>>>>>>>>>>>>>> any
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> questions
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> or
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 2, 2022 at
> > > > > > > >>>>>> 10:22
> > > > > > > >>>>>>>> AM
> > > > > > > >>>>>>>>>>>> Justine
> > > > > > > >>>>>>>>>>>>>>>> Olshan
> > > > > > > >>>>>>>>>>>>>>>>> <
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> jolshan@confluent.io
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey Matthias,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30 — Maybe I also
> > > > > > > >>>>>> didn't
> > > > > > > >>>>>>>>>> express
> > > > > > > >>>>>>>>>>>>>>> myself
> > > > > > > >>>>>>>>>>>>>>>>>>>> clearly.
> > > > > > > >>>>>>>>>>>>>>>>>>>>> For
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't have a way to
> > > > > > > >>>>>>>> distinguish
> > > > > > > >>>>>>>>>>>> between a
> > > > > > > >>>>>>>>>>>>>>>>>> previous
> > > > > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> current
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction since we
> > > > > > > >>>>>> don't
> > > > > > > >>>>>>>> have
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>> epoch
> > > > > > > >>>>>>>>>>>>>>>>> bump.
> > > > > > > >>>>>>>>>>>>>>>>>>>> This
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> means
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message from the previous
> > > > > > > >>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>> may be
> > > > > > > >>>>>>>>>>>>>>>>>>>> added to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> one.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> With
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older clients — we can't
> > > > > > > >>>>>>>>>> guarantee
> > > > > > > >>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>> won't
> > > > > > > >>>>>>>>>>>>>>>>>>>> happen
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> if we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> already
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call
> > > > > > > >>>>>>> (why
> > > > > > > >>>>>>>> we
> > > > > > > >>>>>>>>>> make
> > > > > > > >>>>>>>>>>>>>>> changes
> > > > > > > >>>>>>>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> newer
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> client)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can at least gate some by
> > > > > > > >>>>>>>>>> ensuring
> > > > > > > >>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>> partition
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> has
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> been
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> added
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. The
> > > > > > > >>>>>> rationale
> > > > > > > >>>>>>>> here
> > > > > > > >>>>>>>>>> is
> > > > > > > >>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>>> there
> > > > > > > >>>>>>>>>>>>>>>>>> are
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> likely
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> LESS
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrivals
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as time goes on, so
> > > > > > > >>>>>>> hopefully
> > > > > > > >>>>>>>>>> most
> > > > > > > >>>>>>>>>>>> late
> > > > > > > >>>>>>>>>>>>>>>>> arrivals
> > > > > > > >>>>>>>>>>>>>>>>>>>> will
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> come
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> BEFORE
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call.
> > > > > > > >>>>>>>> Those
> > > > > > > >>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>> arrive
> > > > > > > >>>>>>>>>>>>>>>>>> before
> > > > > > > >>>>>>>>>>>>>>>>>>>>> will
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> properly
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> gated
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> > > > > > > >>>>>>> describeTransactions
> > > > > > > >>>>>>>>>>>> approach.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we take the approach
> > > > > > > >>>>>> you
> > > > > > > >>>>>>>>>>>> suggested,
> > > > > > > >>>>>>>>>>>>>>> ANY
> > > > > > > >>>>>>>>>>>>>>>>> late
> > > > > > > >>>>>>>>>>>>>>>>>>>>> arrival
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> from a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> previous
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction will be
> > > > > > > >>>>>> added.
> > > > > > > >>>>>>>> And
> > > > > > > >>>>>>>>> we
> > > > > > > >>>>>>>>>>>> don't
> > > > > > > >>>>>>>>>>>>>>> want
> > > > > > > >>>>>>>>>>>>>>>>>>>> that. I
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> also
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> see
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> benefit in sending
> > > > > > > >>>>>>>>>> addPartitionsToTxn
> > > > > > > >>>>>>>>>>>>>>> over
> > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> describeTxns
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> They
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> both be one extra RPC to
> > > > > > > >>>>>>> the
> > > > > > > >>>>>>>>> Txn
> > > > > > > >>>>>>>>>>>>>>>> coordinator.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To be clear — newer
> > > > > > > >>>>>> clients
> > > > > > > >>>>>>>>> will
> > > > > > > >>>>>>>>>> use
> > > > > > > >>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> instead
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTxns.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that if we
> > > > > > > >>>>>>> have
> > > > > > > >>>>>>>>>> some
> > > > > > > >>>>>>>>>>>> delay
> > > > > > > >>>>>>>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>> client
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> bump
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it could continue to send
> > > > > > > >>>>>>>> epoch
> > > > > > > >>>>>>>>>> 73
> > > > > > > >>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>> those
> > > > > > > >>>>>>>>>>>>>>>>>>>> records
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fenced.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Perhaps this is not an
> > > > > > > >>>>>>> issue
> > > > > > > >>>>>>>> if
> > > > > > > >>>>>>>>>> we
> > > > > > > >>>>>>>>>>>> don't
> > > > > > > >>>>>>>>>>>>>>>> allow
> > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>> next
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> produce
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> go
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through before the EndTxn
> > > > > > > >>>>>>>>> request
> > > > > > > >>>>>>>>>>>>>>> returns.
> > > > > > > >>>>>>>>>>>>>>>> I'm
> > > > > > > >>>>>>>>>>>>>>>>>>>> also
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> thinking
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> about
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cases of
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure. I will need to
> > > > > > > >>>>>>> think
> > > > > > > >>>>>>>>> on
> > > > > > > >>>>>>>>>>>> this a
> > > > > > > >>>>>>>>>>>>>>> bit.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I wasn't sure if it was
> > > > > > > >>>>>>> that
> > > > > > > >>>>>>>>>>>> confusing.
> > > > > > > >>>>>>>>>>>>>>> But
> > > > > > > >>>>>>>>>>>>>>>> if
> > > > > > > >>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> think it
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> is,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> investigate other ways.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure these are
> > > > > > > >>>>>> the
> > > > > > > >>>>>>>> same
> > > > > > > >>>>>>>>>>>>>>> purgatories
> > > > > > > >>>>>>>>>>>>>>>>>> since
> > > > > > > >>>>>>>>>>>>>>>>>>>> one
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> is a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> produce
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory (I was planning
> > > > > > > >>>>>>> on
> > > > > > > >>>>>>>>>> using a
> > > > > > > >>>>>>>>>>>>>>>> callback
> > > > > > > >>>>>>>>>>>>>>>>>>>> rather
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> than
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the other is simply a
> > > > > > > >>>>>>> request
> > > > > > > >>>>>>>>> to
> > > > > > > >>>>>>>>>>>> append
> > > > > > > >>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>> log.
> > > > > > > >>>>>>>>>>>>>>>>>>>>> Not
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> sure
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> structure here for
> > > > > > > >>>>>>> ordering,
> > > > > > > >>>>>>>>> but
> > > > > > > >>>>>>>>>> my
> > > > > > > >>>>>>>>>>>>>>>>>> understanding
> > > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handle the write request
> > > > > > > >>>>>>>> before
> > > > > > > >>>>>>>>>> it
> > > > > > > >>>>>>>>>>>> hears
> > > > > > > >>>>>>>>>>>>>>>> back
> > > > > > > >>>>>>>>>>>>>>>>>> from
> > > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Txn
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Coordinator.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if I
> > > > > > > >>>>>>>> misunderstood
> > > > > > > >>>>>>>>>>>> something
> > > > > > > >>>>>>>>>>>>>>> or
> > > > > > > >>>>>>>>>>>>>>>>>>>> something
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> was
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> unclear.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 1, 2022 at
> > > > > > > >>>>>>> 12:15
> > > > > > > >>>>>>>> PM
> > > > > > > >>>>>>>>>>>> Matthias
> > > > > > > >>>>>>>>>>>>>>> J.
> > > > > > > >>>>>>>>>>>>>>>>> Sax
> > > > > > > >>>>>>>>>>>>>>>>>> <
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the details
> > > > > > > >>>>>>>>> Justine!
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side change
> > > > > > > >>>>>>> for
> > > > > > > >>>>>>>> 2
> > > > > > > >>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>> removing
> > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> addPartitions
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need to
> > > > > > > >>>>>>> make
> > > > > > > >>>>>>>>>> this
> > > > > > > >>>>>>>>>>>> from
> > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>> producer
> > > > > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think I did not
> > > > > > > >>>>>> express
> > > > > > > >>>>>>>>> myself
> > > > > > > >>>>>>>>>>>>>>> clearly. I
> > > > > > > >>>>>>>>>>>>>>>>>>>>> understand
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should) change the
> > > > > > > >>>>>>> producer
> > > > > > > >>>>>>>> to
> > > > > > > >>>>>>>>>> not
> > > > > > > >>>>>>>>>>>> send
> > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> longer. But I don't
> > > > > > > >>>>>> thinks
> > > > > > > >>>>>>>>> it's
> > > > > > > >>>>>>>>>>>>>>> requirement
> > > > > > > >>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>> change
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> broker?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What I am trying to say
> > > > > > > >>>>>>> is:
> > > > > > > >>>>>>>>> as a
> > > > > > > >>>>>>>>>>>>>>> safe-guard
> > > > > > > >>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> improvement
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producers, the partition
> > > > > > > >>>>>>>>> leader
> > > > > > > >>>>>>>>>> can
> > > > > > > >>>>>>>>>>>> just
> > > > > > > >>>>>>>>>>>>>>>> send
> > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request to the
> > > > > > > >>>>>>>> TX-coordinator
> > > > > > > >>>>>>>>>> in any
> > > > > > > >>>>>>>>>>>>>>> case
> > > > > > > >>>>>>>>>>>>>>>> --
> > > > > > > >>>>>>>>>>>>>>>>> if
> > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> old
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> correctly did send the
> > > > > > > >>>>>>>>>>>> `addPartition`
> > > > > > > >>>>>>>>>>>>>>>> request
> > > > > > > >>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> already, the
> > > > > > > >>>>>>> TX-coordinator
> > > > > > > >>>>>>>>> can
> > > > > > > >>>>>>>>>> just
> > > > > > > >>>>>>>>>>>>>>>> "ignore"
> > > > > > > >>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>> as
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> idempotent.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if the old producer has
> > > > > > > >>>>>> a
> > > > > > > >>>>>>>> bug
> > > > > > > >>>>>>>>>> and
> > > > > > > >>>>>>>>>>>> did
> > > > > > > >>>>>>>>>>>>>>>> forget
> > > > > > > >>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>> sent
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `addPartition`
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request, we would now
> > > > > > > >>>>>>> ensure
> > > > > > > >>>>>>>>>> that
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>> partition
> > > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> indeed
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> added
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX and thus fix a
> > > > > > > >>>>>>> potential
> > > > > > > >>>>>>>>>>>> producer bug
> > > > > > > >>>>>>>>>>>>>>>>> (even
> > > > > > > >>>>>>>>>>>>>>>>>>>> if we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> get
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fencing via the bump
> > > > > > > >>>>>>> epoch).
> > > > > > > >>>>>>>>> --
> > > > > > > >>>>>>>>>> It
> > > > > > > >>>>>>>>>>>>>>> seems to
> > > > > > > >>>>>>>>>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>>>>> a
> > > > > > > >>>>>>>>>>>>>>>>>>>>> good
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Or
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there a reason to not do
> > > > > > > >>>>>>>> this?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is ongoing
> > > > > > > >>>>>> =
> > > > > > > >>>>>>>>>> partition
> > > > > > > >>>>>>>>>>>> was
> > > > > > > >>>>>>>>>>>>>>>> added
> > > > > > > >>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> via
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn. We
> > > > > > > >>>>>>>> check
> > > > > > > >>>>>>>>>> this
> > > > > > > >>>>>>>>>>>> with
> > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this wasn't
> > > > > > > >>>>>>>>>> sufficiently
> > > > > > > >>>>>>>>>>>>>>>>> explained
> > > > > > > >>>>>>>>>>>>>>>>>>>> here:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do what I propose
> > > > > > > >>>>>> in
> > > > > > > >>>>>>>>>> (20), we
> > > > > > > >>>>>>>>>>>>>>> don't
> > > > > > > >>>>>>>>>>>>>>>>>> really
> > > > > > > >>>>>>>>>>>>>>>>>>>>> need
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> make
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `DescribeTransaction`
> > > > > > > >>>>>>> call,
> > > > > > > >>>>>>>> as
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>> partition
> > > > > > > >>>>>>>>>>>>>>>>>>>> leader
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> adds
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for older clients and we
> > > > > > > >>>>>>> get
> > > > > > > >>>>>>>>>> this
> > > > > > > >>>>>>>>>>>> check
> > > > > > > >>>>>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>> free.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is that
> > > > > > > >>>>>> if
> > > > > > > >>>>>>>> any
> > > > > > > >>>>>>>>>>>> messages
> > > > > > > >>>>>>>>>>>>>>>>> somehow
> > > > > > > >>>>>>>>>>>>>>>>>>>> come
> > > > > > > >>>>>>>>>>>>>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> before
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the producer,
> > > > > > > >>>>>>> they
> > > > > > > >>>>>>>>>> will be
> > > > > > > >>>>>>>>>>>>>>>> fenced.
> > > > > > > >>>>>>>>>>>>>>>>>>>>> However,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it can be
> > > > > > > >>>>>>>>>> discussed
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree that we should
> > > > > > > >>>>>>> have
> > > > > > > >>>>>>>>>> epoch
> > > > > > > >>>>>>>>>>>>>>> fencing.
> > > > > > > >>>>>>>>>>>>>>>> My
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> question is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> different:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Assume we are at epoch
> > > > > > > >>>>>> 73,
> > > > > > > >>>>>>>> and
> > > > > > > >>>>>>>>>> we
> > > > > > > >>>>>>>>>>>> have
> > > > > > > >>>>>>>>>>>>>>> an
> > > > > > > >>>>>>>>>>>>>>>>>> ongoing
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed. It seems
> > > > > > > >>>>>>> natural
> > > > > > > >>>>>>>> to
> > > > > > > >>>>>>>>>>>> write the
> > > > > > > >>>>>>>>>>>>>>>>>> "prepare
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> commit"
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> marker
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
> > > > > > > >>>>>>> both
> > > > > > > >>>>>>>>> with
> > > > > > > >>>>>>>>>>>> epoch
> > > > > > > >>>>>>>>>>>>>>> 73,
> > > > > > > >>>>>>>>>>>>>>>>> too,
> > > > > > > >>>>>>>>>>>>>>>>>>>> as
> > > > > > > >>>>>>>>>>>>>>>>>>>>> it
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> belongs
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current transaction. Of
> > > > > > > >>>>>>>>> course,
> > > > > > > >>>>>>>>>> we
> > > > > > > >>>>>>>>>>>> now
> > > > > > > >>>>>>>>>>>>>>> also
> > > > > > > >>>>>>>>>>>>>>>>>> bump
> > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> expect
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the next requests to
> > > > > > > >>>>>> have
> > > > > > > >>>>>>>>> epoch
> > > > > > > >>>>>>>>>> 74,
> > > > > > > >>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>> would
> > > > > > > >>>>>>>>>>>>>>>>>>>> reject
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> an
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch 73, as the
> > > > > > > >>>>>>>> corresponding
> > > > > > > >>>>>>>>>> TX
> > > > > > > >>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>> epoch
> > > > > > > >>>>>>>>>>>>>>>>> 73
> > > > > > > >>>>>>>>>>>>>>>>>>>> was
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> already
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems you propose to
> > > > > > > >>>>>>>> write
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>> "prepare
> > > > > > > >>>>>>>>>>>>>>>>>> commit
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> marker"
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
> > > > > > > >>>>>>> with
> > > > > > > >>>>>>>>>> epoch 74
> > > > > > > >>>>>>>>>>>>>>>> though,
> > > > > > > >>>>>>>>>>>>>>>>>> what
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> work,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems confusing. Is
> > > > > > > >>>>>> there
> > > > > > > >>>>>>> a
> > > > > > > >>>>>>>>>> reason
> > > > > > > >>>>>>>>>>>> why
> > > > > > > >>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>> would
> > > > > > > >>>>>>>>>>>>>>>>>>>> use
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> bumped
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 74
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of the current
> > > > > > > >>>>>>> epoch
> > > > > > > >>>>>>>>> 73?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are checking if
> > > > > > > >>>>>>> the
> > > > > > > >>>>>>>>>>>>>>> transaction is
> > > > > > > >>>>>>>>>>>>>>>>>>>> ongoing,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the leader
> > > > > > > >>>>>>>>> partition
> > > > > > > >>>>>>>>>> to
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for this
> > > > > > > >>>>>>>>>> message to
> > > > > > > >>>>>>>>>>>> come
> > > > > > > >>>>>>>>>>>>>>>>> back,
> > > > > > > >>>>>>>>>>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> theory
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
> > > > > > > >>>>>> that
> > > > > > > >>>>>>>>> would
> > > > > > > >>>>>>>>>>>> make the
> > > > > > > >>>>>>>>>>>>>>>>>> original
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> result
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out of
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why we
> > > > > > > >>>>>> can
> > > > > > > >>>>>>>>> check
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>> leader
> > > > > > > >>>>>>>>>>>>>>>>>> state
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> before
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks. Got it.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, is this really
> > > > > > > >>>>>> an
> > > > > > > >>>>>>>>>> issue?
> > > > > > > >>>>>>>>>>>> We put
> > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>> produce
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory, so how could
> > > > > > > >>>>>> we
> > > > > > > >>>>>>>>>> process
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> first?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Don't we need to put the
> > > > > > > >>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
> > > > > > > >>>>>>>>>>>>>>>>>>>>> into
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> too,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for this case, and
> > > > > > > >>>>>> process
> > > > > > > >>>>>>>>> both
> > > > > > > >>>>>>>>>>>> request
> > > > > > > >>>>>>>>>>>>>>>>>> in-order?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> (Again,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> my
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> knowledge is limited and
> > > > > > > >>>>>>>> maybe
> > > > > > > >>>>>>>>>> we
> > > > > > > >>>>>>>>>>>> don't
> > > > > > > >>>>>>>>>>>>>>>>>> maintain
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> order
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case, what seems to be
> > > > > > > >>>>>> an
> > > > > > > >>>>>>>>> issue
> > > > > > > >>>>>>>>>>>> IMHO,
> > > > > > > >>>>>>>>>>>>>>> and I
> > > > > > > >>>>>>>>>>>>>>>>> am
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> wondering
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> changing
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request handling to
> > > > > > > >>>>>>> preserve
> > > > > > > >>>>>>>>>> order
> > > > > > > >>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>> case
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> might be
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> cleaner
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution?)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/30/22 3:28 PM,
> > > > > > > >>>>>> Artem
> > > > > > > >>>>>>>>>> Livshits
> > > > > > > >>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think the interesting
> > > > > > > >>>>>>>> part
> > > > > > > >>>>>>>>> is
> > > > > > > >>>>>>>>>>>> not in
> > > > > > > >>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>> logic
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> (because
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> tries
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> figure out when
> > > > > > > >>>>>>>>>>>> UNKNOWN_PRODUCER_ID is
> > > > > > > >>>>>>>>>>>>>>>>>> retriable
> > > > > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> it's
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retryable,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's definitely not
> > > > > > > >>>>>>> fatal),
> > > > > > > >>>>>>>>> but
> > > > > > > >>>>>>>>>>>> what
> > > > > > > >>>>>>>>>>>>>>>> happens
> > > > > > > >>>>>>>>>>>>>>>>>>>> when
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> logic
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'true' and falls
> > > > > > > >>>>>> through.
> > > > > > > >>>>>>>> In
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>> old
> > > > > > > >>>>>>>>>>>>>>>>> clients
> > > > > > > >>>>>>>>>>>>>>>>>> it
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> seems
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> keep the behavior in
> > > > > > > >>>>>> the
> > > > > > > >>>>>>>> new
> > > > > > > >>>>>>>>>>>> clients,
> > > > > > > >>>>>>>>>>>>>>> I'd
> > > > > > > >>>>>>>>>>>>>>>>>>>> expect it
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> well.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 29, 2022 at
> > > > > > > >>>>>>>> 11:57
> > > > > > > >>>>>>>>>> AM
> > > > > > > >>>>>>>>>>>>>>> Justine
> > > > > > > >>>>>>>>>>>>>>>>>> Olshan
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>> <jolshan@confluent.io.invalid
> > > > > > > >>>>>>>>>>
> > > > > > > >>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Artem and Jeff,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
> > > > > > > >>>>>> look
> > > > > > > >>>>>>>> and
> > > > > > > >>>>>>>>>>>> sorry for
> > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>> slow
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> response.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You both mentioned the
> > > > > > > >>>>>>>>> change
> > > > > > > >>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>> handle
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear — this error
> > > > > > > >>>>>> code
> > > > > > > >>>>>>>> will
> > > > > > > >>>>>>>>>> only
> > > > > > > >>>>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>> sent
> > > > > > > >>>>>>>>>>>>>>>>>> again
> > > > > > > >>>>>>>>>>>>>>>>>>>>> when
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> client's
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version is high enough
> > > > > > > >>>>>>> to
> > > > > > > >>>>>>>>>> ensure
> > > > > > > >>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>> handle
> > > > > > > >>>>>>>>>>>>>>>>> it
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> correctly.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The current (Java)
> > > > > > > >>>>>>> client
> > > > > > > >>>>>>>>>> handles
> > > > > > > >>>>>>>>>>>>>>> this by
> > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> following
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> (somewhat
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> long)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code snippet:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // An
> > > > > > > >>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > >>>>>>>>>> means
> > > > > > > >>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>> have
> > > > > > > >>>>>>>>>>>>>>>>>>>> lost
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker. Depending on
> > > > > > > >>>>>> the
> > > > > > > >>>>>>>> log
> > > > > > > >>>>>>>>>> start
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // offset, we may want
> > > > > > > >>>>>>> to
> > > > > > > >>>>>>>>>> retry
> > > > > > > >>>>>>>>>>>>>>> these, as
> > > > > > > >>>>>>>>>>>>>>>>>>>>> described
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> each
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> case
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below. If
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> none of those apply,
> > > > > > > >>>>>>> then
> > > > > > > >>>>>>>>> for
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // idempotent
> > > > > > > >>>>>> producer,
> > > > > > > >>>>>>> we
> > > > > > > >>>>>>>>>> will
> > > > > > > >>>>>>>>>>>>>>> locally
> > > > > > > >>>>>>>>>>>>>>>>> bump
> > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> reset
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence numbers of
> > > > > > > >>>>>>>>> in-flight
> > > > > > > >>>>>>>>>>>> batches
> > > > > > > >>>>>>>>>>>>>>>> from
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // sequence 0, then
> > > > > > > >>>>>>> retry
> > > > > > > >>>>>>>>> the
> > > > > > > >>>>>>>>>>>> failed
> > > > > > > >>>>>>>>>>>>>>>> batch,
> > > > > > > >>>>>>>>>>>>>>>>>>>> which
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> should
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> now
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> succeed.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the transactional
> > > > > > > >>>>>>>> producer,
> > > > > > > >>>>>>>>>> allow
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // batch to fail. When
> > > > > > > >>>>>>>>>> processing
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>> failed
> > > > > > > >>>>>>>>>>>>>>>>>>>>> batch,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transition
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an abortable error and
> > > > > > > >>>>>>>> set a
> > > > > > > >>>>>>>>>> flag
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // indicating that we
> > > > > > > >>>>>>> need
> > > > > > > >>>>>>>>> to
> > > > > > > >>>>>>>>>>>> bump the
> > > > > > > >>>>>>>>>>>>>>>>> epoch
> > > > > > > >>>>>>>>>>>>>>>>>>>> (if
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> supported
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> by
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker).
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if (error ==
> > > > > > > >>>>>>>>>>>>>>>> Errors.*UNKNOWN_PRODUCER_ID*)
> > > > > > > >>>>>>>>>>>>>>>>> {
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > > > > > > >>>>>>>>>> (response.logStartOffset
> > > > > > > >>>>>>>>>>>> ==
> > > > > > > >>>>>>>>>>>>>>> -1)
> > > > > > > >>>>>>>>>>>>>>>> {
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // We don't
> > > > > > > >>>>>>> know
> > > > > > > >>>>>>>>>> the log
> > > > > > > >>>>>>>>>>>>>>> start
> > > > > > > >>>>>>>>>>>>>>>>>> offset
> > > > > > > >>>>>>>>>>>>>>>>>>>>> with
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> response.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the request
> > > > > > > >>>>>>>> until
> > > > > > > >>>>>>>>>> we
> > > > > > > >>>>>>>>>>>> get
> > > > > > > >>>>>>>>>>>>>>> it.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // The
> > > > > > > >>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > >>>>>>>>>>>>>>>> error
> > > > > > > >>>>>>>>>>>>>>>>>> code
> > > > > > > >>>>>>>>>>>>>>>>>>>>> was
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> added
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> along
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProduceResponse which
> > > > > > > >>>>>>>>>> includes the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              //
> > > > > > > >>>>>>>> logStartOffset.
> > > > > > > >>>>>>>>>> So
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>> '-1'
> > > > > > > >>>>>>>>>>>>>>>>>>>> sentinel
> > > > > > > >>>>>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> backward
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatibility.
> > > > > > > >>>>>> Instead,
> > > > > > > >>>>>>> it
> > > > > > > >>>>>>>>> is
> > > > > > > >>>>>>>>>>>> possible
> > > > > > > >>>>>>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // a broker
> > > > > > > >>>>>> to
> > > > > > > >>>>>>>> not
> > > > > > > >>>>>>>>>> know
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> logStartOffset at
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> when
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returning
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response because
> > > > > > > >>>>>> the
> > > > > > > >>>>>>>>>> partition
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // may have
> > > > > > > >>>>>>>> moved
> > > > > > > >>>>>>>>>> away
> > > > > > > >>>>>>>>>>>> from
> > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>> broker
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> from
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error was
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initially raised to
> > > > > > > >>>>>> the
> > > > > > > >>>>>>>> time
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // response
> > > > > > > >>>>>>> was
> > > > > > > >>>>>>>>>> being
> > > > > > > >>>>>>>>>>>>>>>>> constructed.
> > > > > > > >>>>>>>>>>>>>>>>>> In
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> these
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> cases,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retry the request: we
> > > > > > > >>>>>>> are
> > > > > > > >>>>>>>>>>>> guaranteed
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // to
> > > > > > > >>>>>>> eventually
> > > > > > > >>>>>>>>>> get a
> > > > > > > >>>>>>>>>>>>>>>>>> logStartOffset
> > > > > > > >>>>>>>>>>>>>>>>>>>>> once
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> things
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> settle
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> down.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > > > > > > >>>>>>>>>>>>>>> (batch.sequenceHasBeenReset()) {
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // When the
> > > > > > > >>>>>>>> first
> > > > > > > >>>>>>>>>>>> inflight
> > > > > > > >>>>>>>>>>>>>>>> batch
> > > > > > > >>>>>>>>>>>>>>>>>>>> fails
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> due to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> truncation
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> then the sequences of
> > > > > > > >>>>>>> all
> > > > > > > >>>>>>>>> the
> > > > > > > >>>>>>>>>>>> other
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // in flight
> > > > > > > >>>>>>>>> batches
> > > > > > > >>>>>>>>>>>> would
> > > > > > > >>>>>>>>>>>>>>> have
> > > > > > > >>>>>>>>>>>>>>>>>> been
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> restarted
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> from
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> beginning.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, when those
> > > > > > > >>>>>>>>> responses
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // come back
> > > > > > > >>>>>>>> from
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>> broker,
> > > > > > > >>>>>>>>>>>>>>>>> they
> > > > > > > >>>>>>>>>>>>>>>>>>>> would
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> also
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> come
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > >>>>>>> error.
> > > > > > > >>>>>>>>> In
> > > > > > > >>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>> case,
> > > > > > > >>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>> should
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // reset the
> > > > > > > >>>>>>>>>> sequence
> > > > > > > >>>>>>>>>>>>>>> numbers
> > > > > > > >>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> beginning.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          } else if
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>> (lastAckedOffset(batch.topicPartition).orElse(
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> response.logStartOffset) {
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // The head
> > > > > > > >>>>>> of
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>> log
> > > > > > > >>>>>>>>>>>> has
> > > > > > > >>>>>>>>>>>>>>> been
> > > > > > > >>>>>>>>>>>>>>>>>>>> removed,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> probably
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> due
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retention time
> > > > > > > >>>>>> elapsing.
> > > > > > > >>>>>>>> In
> > > > > > > >>>>>>>>>> this
> > > > > > > >>>>>>>>>>>> case,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // we expect
> > > > > > > >>>>>>> to
> > > > > > > >>>>>>>>>> lose the
> > > > > > > >>>>>>>>>>>>>>>> producer
> > > > > > > >>>>>>>>>>>>>>>>>>>> state.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> For
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer, reset the
> > > > > > > >>>>>>>>> sequences
> > > > > > > >>>>>>>>>> of
> > > > > > > >>>>>>>>>>>> all
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // inflight
> > > > > > > >>>>>>>>> batches
> > > > > > > >>>>>>>>>> to
> > > > > > > >>>>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>> from
> > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> beginning
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> retry
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the transaction
> > > > > > > >>>>>>> does
> > > > > > > >>>>>>>>> not
> > > > > > > >>>>>>>>>>>> need to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // be
> > > > > > > >>>>>> aborted.
> > > > > > > >>>>>>>> For
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>> idempotent
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> producer,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> bump
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> avoid
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reusing (sequence,
> > > > > > > >>>>>>> epoch)
> > > > > > > >>>>>>>>>> pairs
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              if
> > > > > > > >>>>>>>>>> (isTransactional()) {
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>
> > > > > > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>> this.producerIdAndEpoch);
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              } else {
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>
> > requestEpochBumpForPartition(batch.topicPartition);
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              }
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > > > > > > >>>>>>>>> (!isTransactional())
> > > > > > > >>>>>>>>>> {
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // For the
> > > > > > > >>>>>>>>>> idempotent
> > > > > > > >>>>>>>>>>>>>>> producer,
> > > > > > > >>>>>>>>>>>>>>>>>>>> always
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> retry
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors. If the batch
> > > > > > > >>>>>> has
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>>>> current
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // producer
> > > > > > > >>>>>> ID
> > > > > > > >>>>>>>> and
> > > > > > > >>>>>>>>>>>> epoch,
> > > > > > > >>>>>>>>>>>>>>>>> request a
> > > > > > > >>>>>>>>>>>>>>>>>>>> bump
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Otherwise
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the
> > > > > > > >>>>>> produce.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>
> requestEpochBumpForPartition(batch.topicPartition);
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I was considering
> > > > > > > >>>>>>> keeping
> > > > > > > >>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>> behavior —
> > > > > > > >>>>>>>>>>>>>>>>> but
> > > > > > > >>>>>>>>>>>>>>>>>>>> am
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> open
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simplifying
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We are leaving changes
> > > > > > > >>>>>>> to
> > > > > > > >>>>>>>>>> older
> > > > > > > >>>>>>>>>>>>>>> clients
> > > > > > > >>>>>>>>>>>>>>>> off
> > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> table
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> here
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> since
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> caused
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many issues for
> > > > > > > >>>>>> clients
> > > > > > > >>>>>>> in
> > > > > > > >>>>>>>>> the
> > > > > > > >>>>>>>>>>>> past.
> > > > > > > >>>>>>>>>>>>>>>>>> Previously
> > > > > > > >>>>>>>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> was
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we didn't have the
> > > > > > > >>>>>>>>> mechanisms
> > > > > > > >>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>>> place to
> > > > > > > >>>>>>>>>>>>>>>>>>>> detect
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> when
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> was
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> legitimate
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case vs some bug or
> > > > > > > >>>>>> gap
> > > > > > > >>>>>>> in
> > > > > > > >>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>> protocol.
> > > > > > > >>>>>>>>>>>>>>>>>>>> Ensuring
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> each
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> its
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> own epoch should close
> > > > > > > >>>>>>>> this
> > > > > > > >>>>>>>>>> gap.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> And to address Jeff's
> > > > > > > >>>>>>>> second
> > > > > > > >>>>>>>>>>>> point:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *does the typical
> > > > > > > >>>>>>> produce
> > > > > > > >>>>>>>>>> request
> > > > > > > >>>>>>>>>>>> path
> > > > > > > >>>>>>>>>>>>>>>>> append
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> records
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> local
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> log
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along*
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *with the
> > > > > > > >>>>>>>>>> currentTxnFirstOffset
> > > > > > > >>>>>>>>>>>>>>>>> information?
> > > > > > > >>>>>>>>>>>>>>>>>> I
> > > > > > > >>>>>>>>>>>>>>>>>>>>> would
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> like
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand*
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *when the field is
> > > > > > > >>>>>>> written
> > > > > > > >>>>>>>>> to
> > > > > > > >>>>>>>>>>>> disk.*
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, the first produce
> > > > > > > >>>>>>>>> request
> > > > > > > >>>>>>>>>>>>>>> populates
> > > > > > > >>>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>>> field
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> writes
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> offset
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as part of the record
> > > > > > > >>>>>>>> batch
> > > > > > > >>>>>>>>>> and
> > > > > > > >>>>>>>>>>>> also
> > > > > > > >>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>> producer
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> state
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> snapshot.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we reload the records
> > > > > > > >>>>>> on
> > > > > > > >>>>>>>>>> restart
> > > > > > > >>>>>>>>>>>>>>> and/or
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> reassignment,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> repopulate
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> field with the
> > > > > > > >>>>>> snapshot
> > > > > > > >>>>>>>> from
> > > > > > > >>>>>>>>>> disk
> > > > > > > >>>>>>>>>>>>>>> along
> > > > > > > >>>>>>>>>>>>>>>>> with
> > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> rest
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if there
> > > > > > > >>>>>> are
> > > > > > > >>>>>>>>>> further
> > > > > > > >>>>>>>>>>>>>>> comments
> > > > > > > >>>>>>>>>>>>>>>>>>>> and/or
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> questions.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > > > > > > >>>>>> at
> > > > > > > >>>>>>>> 9:00
> > > > > > > >>>>>>>>>> PM
> > > > > > > >>>>>>>>>>>> Jeff
> > > > > > > >>>>>>>>>>>>>>> Kim
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>> <jeff.kim@confluent.io.invalid
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP! I
> > > > > > > >>>>>>>> have
> > > > > > > >>>>>>>>>> two
> > > > > > > >>>>>>>>>>>>>>>> questions:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1) For new clients,
> > > > > > > >>>>>> we
> > > > > > > >>>>>>>> can
> > > > > > > >>>>>>>>>> once
> > > > > > > >>>>>>>>>>>> again
> > > > > > > >>>>>>>>>>>>>>>>> return
> > > > > > > >>>>>>>>>>>>>>>>>>>> an
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> error
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for sequences
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that are non-zero
> > > > > > > >>>>>> when
> > > > > > > >>>>>>>>> there
> > > > > > > >>>>>>>>>> is
> > > > > > > >>>>>>>>>>>> no
> > > > > > > >>>>>>>>>>>>>>>>> producer
> > > > > > > >>>>>>>>>>>>>>>>>>>> state
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> present
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> on
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This will indicate we
> > > > > > > >>>>>>>>> missed
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>> 0
> > > > > > > >>>>>>>>>>>>>>>>> sequence
> > > > > > > >>>>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> yet
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> want
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the log.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would like to
> > > > > > > >>>>>>>> understand
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>> current
> > > > > > > >>>>>>>>>>>>>>>>>>>> behavior
> > > > > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> handle
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and if there are any
> > > > > > > >>>>>>>>> changes
> > > > > > > >>>>>>>>>> we
> > > > > > > >>>>>>>>>>>> are
> > > > > > > >>>>>>>>>>>>>>>>> making.
> > > > > > > >>>>>>>>>>>>>>>>>>>> Maybe
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> I'm
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> missing
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but we would want to
> > > > > > > >>>>>>>>> identify
> > > > > > > >>>>>>>>>>>>>>> whether we
> > > > > > > >>>>>>>>>>>>>>>>>>>> missed
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the 0
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> sequence
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients, no?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2) Upon returning
> > > > > > > >>>>>> from
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> coordinator, we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> set
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as ongoing on the
> > > > > > > >>>>>>> leader
> > > > > > > >>>>>>>> by
> > > > > > > >>>>>>>>>>>>>>> populating
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> currentTxnFirstOffset
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through the typical
> > > > > > > >>>>>>>> produce
> > > > > > > >>>>>>>>>>>> request
> > > > > > > >>>>>>>>>>>>>>>>>> handling.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does the typical
> > > > > > > >>>>>>> produce
> > > > > > > >>>>>>>>>> request
> > > > > > > >>>>>>>>>>>> path
> > > > > > > >>>>>>>>>>>>>>>>> append
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> records
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> local
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> log
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> > > > > > > >>>>>>>>>> currentTxnFirstOffset
> > > > > > > >>>>>>>>>>>>>>>>> information?
> > > > > > > >>>>>>>>>>>>>>>>>> I
> > > > > > > >>>>>>>>>>>>>>>>>>>>> would
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> like
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when the field is
> > > > > > > >>>>>>> written
> > > > > > > >>>>>>>>> to
> > > > > > > >>>>>>>>>>>> disk.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeff
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > > > > > > >>>>>> at
> > > > > > > >>>>>>>>> 4:44
> > > > > > > >>>>>>>>>> PM
> > > > > > > >>>>>>>>>>>> Artem
> > > > > > > >>>>>>>>>>>>>>>>>> Livshits
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
> > > > > > > >>>>>>> alivshits@confluent.io
> > > > > > > >>>>>>>>>> .invalid>
> > > > > > > >>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for the
> > > > > > > >>>>>> KIP.
> > > > > > > >>>>>>>> I
> > > > > > > >>>>>>>>>> have
> > > > > > > >>>>>>>>>>>> one
> > > > > > > >>>>>>>>>>>>>>>>>> question.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 5) For new clients,
> > > > > > > >>>>>> we
> > > > > > > >>>>>>>> can
> > > > > > > >>>>>>>>>> once
> > > > > > > >>>>>>>>>>>>>>> again
> > > > > > > >>>>>>>>>>>>>>>>>> return
> > > > > > > >>>>>>>>>>>>>>>>>>>> an
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> error
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe we had
> > > > > > > >>>>>>>> problems
> > > > > > > >>>>>>>>>> in the
> > > > > > > >>>>>>>>>>>>>>> past
> > > > > > > >>>>>>>>>>>>>>>>> with
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> returning
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because it was
> > > > > > > >>>>>>>> considered
> > > > > > > >>>>>>>>>> fatal
> > > > > > > >>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>> required
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> client
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> restart.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> It
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good to spell out
> > > > > > > >>>>>> the
> > > > > > > >>>>>>>> new
> > > > > > > >>>>>>>>>> client
> > > > > > > >>>>>>>>>>>>>>>> behavior
> > > > > > > >>>>>>>>>>>>>>>>>>>> when
> > > > > > > >>>>>>>>>>>>>>>>>>>>> it
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> receives
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > > > > > > >>>>>>> at
> > > > > > > >>>>>>>>>> 10:00 AM
> > > > > > > >>>>>>>>>>>>>>>> Justine
> > > > > > > >>>>>>>>>>>>>>>>>>>> Olshan
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > > > >>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
> > > > > > > >>>>>>>> look
> > > > > > > >>>>>>>>>>>> Matthias.
> > > > > > > >>>>>>>>>>>>>>>> I've
> > > > > > > >>>>>>>>>>>>>>>>>>>> tried
> > > > > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> answer
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> your
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> questions
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 10)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Right — so the
> > > > > > > >>>>>>> hanging
> > > > > > > >>>>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>>> only
> > > > > > > >>>>>>>>>>>>>>>>>>>> occurs
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> when
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> come in and the
> > > > > > > >>>>>>>> partition
> > > > > > > >>>>>>>>>> is
> > > > > > > >>>>>>>>>>>> never
> > > > > > > >>>>>>>>>>>>>>>> added
> > > > > > > >>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>> a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> again.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never add the
> > > > > > > >>>>>>> partition
> > > > > > > >>>>>>>>> to
> > > > > > > >>>>>>>>>> a
> > > > > > > >>>>>>>>>>>>>>>>> transaction,
> > > > > > > >>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>> will
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> never
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> marker
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never advance the
> > > > > > > >>>>>>> LSO.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do end up
> > > > > > > >>>>>>> adding
> > > > > > > >>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>> partition
> > > > > > > >>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (I
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suppose
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happen before or
> > > > > > > >>>>>>> after
> > > > > > > >>>>>>>>> the
> > > > > > > >>>>>>>>>> late
> > > > > > > >>>>>>>>>>>>>>>> message
> > > > > > > >>>>>>>>>>>>>>>>>>>> comes
> > > > > > > >>>>>>>>>>>>>>>>>>>>> in)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> then
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> include
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late message in the
> > > > > > > >>>>>>>> next
> > > > > > > >>>>>>>>>>>>>>> (incorrect)
> > > > > > > >>>>>>>>>>>>>>>>>>>>> transaction.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> So perhaps it is
> > > > > > > >>>>>>>> clearer
> > > > > > > >>>>>>>>> to
> > > > > > > >>>>>>>>>>>> make
> > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>> distinction
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> between
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> eventually get
> > > > > > > >>>>>> added
> > > > > > > >>>>>>> to
> > > > > > > >>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>>>>> (but
> > > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> wrong
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> one)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> or
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that never get
> > > > > > > >>>>>> added
> > > > > > > >>>>>>>> and
> > > > > > > >>>>>>>>>> become
> > > > > > > >>>>>>>>>>>>>>>> hanging.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side
> > > > > > > >>>>>>> change
> > > > > > > >>>>>>>>> for
> > > > > > > >>>>>>>>>> 2 is
> > > > > > > >>>>>>>>>>>>>>>> removing
> > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> addPartitions
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need
> > > > > > > >>>>>>> to
> > > > > > > >>>>>>>>> make
> > > > > > > >>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>> from
> > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In my opinion, the
> > > > > > > >>>>>>>> issue
> > > > > > > >>>>>>>>>> with
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> call
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is that we don't
> > > > > > > >>>>>> have
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>> epoch
> > > > > > > >>>>>>>>>>>>>>> bump,
> > > > > > > >>>>>>>>>>>>>>>> so
> > > > > > > >>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>> don't
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> know
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> belongs to the
> > > > > > > >>>>>>> previous
> > > > > > > >>>>>>>>>>>>>>> transaction or
> > > > > > > >>>>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>>> one.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> We
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition has been
> > > > > > > >>>>>>>> added
> > > > > > > >>>>>>>>> to
> > > > > > > >>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>> transaction.
> > > > > > > >>>>>>>>>>>>>>>>>>>>> Of
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> course,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> means
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't completely
> > > > > > > >>>>>>> cover
> > > > > > > >>>>>>>>> the
> > > > > > > >>>>>>>>>> case
> > > > > > > >>>>>>>>>>>>>>> where
> > > > > > > >>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>> have a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> really
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have added the
> > > > > > > >>>>>>>> partition
> > > > > > > >>>>>>>>> to
> > > > > > > >>>>>>>>>>>> the new
> > > > > > > >>>>>>>>>>>>>>>>>>>>> transaction,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> that's
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something we will
> > > > > > > >>>>>>> need
> > > > > > > >>>>>>>>> the
> > > > > > > >>>>>>>>>> new
> > > > > > > >>>>>>>>>>>>>>> clients
> > > > > > > >>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>> cover.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is
> > > > > > > >>>>>>> ongoing
> > > > > > > >>>>>>>> =
> > > > > > > >>>>>>>>>>>> partition
> > > > > > > >>>>>>>>>>>>>>> was
> > > > > > > >>>>>>>>>>>>>>>>>>>> added to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> via
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn.
> > > > > > > >>>>>>> We
> > > > > > > >>>>>>>>>> check
> > > > > > > >>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>> with
> > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this
> > > > > > > >>>>>>> wasn't
> > > > > > > >>>>>>>>>>>> sufficiently
> > > > > > > >>>>>>>>>>>>>>>>>>>> explained
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> here:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>
> > > > > > > >>>>>>>>>
> > > > > > > >>>>>>>>
> > > > > > > >>>>>>>
> > > > > > > >>>>>>
> > > > > > > >>>>
> > > > > > > >>
> > > > > >
> > > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is
> > > > > > > >>>>>> that
> > > > > > > >>>>>>>> if
> > > > > > > >>>>>>>>>> any
> > > > > > > >>>>>>>>>>>>>>> messages
> > > > > > > >>>>>>>>>>>>>>>>>>>> somehow
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> come
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> before
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the
> > > > > > > >>>>>>> producer,
> > > > > > > >>>>>>>>> they
> > > > > > > >>>>>>>>>>>> will be
> > > > > > > >>>>>>>>>>>>>>>>>> fenced.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> However,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it
> > > > > > > >>>>>> can
> > > > > > > >>>>>>> be
> > > > > > > >>>>>>>>>>>> discussed
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 50)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It should be
> > > > > > > >>>>>>>> synchronous
> > > > > > > >>>>>>>>>>>> because
> > > > > > > >>>>>>>>>>>>>>> if we
> > > > > > > >>>>>>>>>>>>>>>>>> have
> > > > > > > >>>>>>>>>>>>>>>>>>>> an
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> event
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (ie,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> an
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes us to need
> > > > > > > >>>>>> to
> > > > > > > >>>>>>>>> abort
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>> transaction,
> > > > > > > >>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> need
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions to send
> > > > > > > >>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>> markers
> > > > > > > >>>>>>>>>>>>>>>>> to.
> > > > > > > >>>>>>>>>>>>>>>>>>>> We
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> know
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we added them to
> > > > > > > >>>>>> the
> > > > > > > >>>>>>>>>>>> coordinator
> > > > > > > >>>>>>>>>>>>>>> via
> > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Previously we have
> > > > > > > >>>>>>> had
> > > > > > > >>>>>>>>>>>> asynchronous
> > > > > > > >>>>>>>>>>>>>>>>> calls
> > > > > > > >>>>>>>>>>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> past
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (ie,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit markers when
> > > > > > > >>>>>>> the
> > > > > > > >>>>>>>>>>>>>>> transaction is
> > > > > > > >>>>>>>>>>>>>>>>>>>>> completed)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> often
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes confusion as
> > > > > > > >>>>>>> we
> > > > > > > >>>>>>>>>> need to
> > > > > > > >>>>>>>>>>>> wait
> > > > > > > >>>>>>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>> some
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> operations
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complete.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing commit
> > > > > > > >>>>>>> markers
> > > > > > > >>>>>>>>>> case,
> > > > > > > >>>>>>>>>>>>>>> clients
> > > > > > > >>>>>>>>>>>>>>>>> often
> > > > > > > >>>>>>>>>>>>>>>>>>>> see
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>> CONCURRENT_TRANSACTIONs
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error messages and
> > > > > > > >>>>>>> that
> > > > > > > >>>>>>>>>> can be
> > > > > > > >>>>>>>>>>>>>>>>> confusing.
> > > > > > > >>>>>>>>>>>>>>>>>>>> For
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> reason,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> may
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simpler to just
> > > > > > > >>>>>> have
> > > > > > > >>>>>>>>>>>> synchronous
> > > > > > > >>>>>>>>>>>>>>>> calls —
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> especially
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> block
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some operation's
> > > > > > > >>>>>>>>> completion
> > > > > > > >>>>>>>>>>>> anyway
> > > > > > > >>>>>>>>>>>>>>>>> before
> > > > > > > >>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>> can
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> start
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> next
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. And
> > > > > > > >>>>>>> yes, I
> > > > > > > >>>>>>>>>> meant
> > > > > > > >>>>>>>>>>>>>>>>>> coordinator. I
> > > > > > > >>>>>>>>>>>>>>>>>>>>> will
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> fix
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> that.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are
> > > > > > > >>>>>> checking
> > > > > > > >>>>>>> if
> > > > > > > >>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> ongoing,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the
> > > > > > > >>>>>> leader
> > > > > > > >>>>>>>>>> partition
> > > > > > > >>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for
> > > > > > > >>>>>>> this
> > > > > > > >>>>>>>>>>>> message to
> > > > > > > >>>>>>>>>>>>>>>> come
> > > > > > > >>>>>>>>>>>>>>>>>>>> back,
> > > > > > > >>>>>>>>>>>>>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> theory
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
> > > > > > > >>>>>>>> that
> > > > > > > >>>>>>>>>> would
> > > > > > > >>>>>>>>>>>> make
> > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>> original
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> result
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why
> > > > > > > >>>>>> we
> > > > > > > >>>>>>>> can
> > > > > > > >>>>>>>>>> check
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>> leader
> > > > > > > >>>>>>>>>>>>>>>>>>>> state
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> before
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm happy to update
> > > > > > > >>>>>>> the
> > > > > > > >>>>>>>>>> KIP if
> > > > > > > >>>>>>>>>>>>>>> some of
> > > > > > > >>>>>>>>>>>>>>>>>> these
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> things
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> were
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Nov 21,
> > > > > > > >>>>>> 2022
> > > > > > > >>>>>>> at
> > > > > > > >>>>>>>>>> 7:11 PM
> > > > > > > >>>>>>>>>>>>>>>> Matthias
> > > > > > > >>>>>>>>>>>>>>>>>> J.
> > > > > > > >>>>>>>>>>>>>>>>>>>>> Sax <
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the
> > > > > > > >>>>>> KIP.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Couple of
> > > > > > > >>>>>>>> clarification
> > > > > > > >>>>>>>>>>>> questions
> > > > > > > >>>>>>>>>>>>>>> (I
> > > > > > > >>>>>>>>>>>>>>>> am
> > > > > > > >>>>>>>>>>>>>>>>>>>> not a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> broker
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> expert
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> do
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some question are
> > > > > > > >>>>>>>>> obvious
> > > > > > > >>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>> others,
> > > > > > > >>>>>>>>>>>>>>>>> but
> > > > > > > >>>>>>>>>>>>>>>>>>>> not
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> me
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> my
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lack
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker knowledge).
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (10)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What happens if
> > > > > > > >>>>>> the
> > > > > > > >>>>>>>>>> message
> > > > > > > >>>>>>>>>>>> come
> > > > > > > >>>>>>>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>>>>>> before
> > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> next
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request? It seems
> > > > > > > >>>>>>> the
> > > > > > > >>>>>>>>>> broker
> > > > > > > >>>>>>>>>>>>>>> hosting
> > > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>> data
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anything about it
> > > > > > > >>>>>>> and
> > > > > > > >>>>>>>>>> append
> > > > > > > >>>>>>>>>>>> it to
> > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> partition,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> too?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> What
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> difference between
> > > > > > > >>>>>>>> both
> > > > > > > >>>>>>>>>> cases?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, it seems a
> > > > > > > >>>>>> TX
> > > > > > > >>>>>>>>> would
> > > > > > > >>>>>>>>>> only
> > > > > > > >>>>>>>>>>>>>>> hang,
> > > > > > > >>>>>>>>>>>>>>>> if
> > > > > > > >>>>>>>>>>>>>>>>>>>> there
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> is no
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> following
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> either committer
> > > > > > > >>>>>> or
> > > > > > > >>>>>>>>>> aborted?
> > > > > > > >>>>>>>>>>>> Thus,
> > > > > > > >>>>>>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>> case
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> above,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> TX
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> might
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually not hang
> > > > > > > >>>>>>> (of
> > > > > > > >>>>>>>>>> course,
> > > > > > > >>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>> might
> > > > > > > >>>>>>>>>>>>>>>>>> get
> > > > > > > >>>>>>>>>>>>>>>>>>>> an
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> EOS
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> violation
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX was aborted and
> > > > > > > >>>>>>> the
> > > > > > > >>>>>>>>>> second
> > > > > > > >>>>>>>>>>>>>>>>> committed,
> > > > > > > >>>>>>>>>>>>>>>>>> or
> > > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> other
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> way
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around).
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (20)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Of course, 1 and
> > > > > > > >>>>>> 2
> > > > > > > >>>>>>>>>> require
> > > > > > > >>>>>>>>>>>>>>>> client-side
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> changes, so
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> those approaches
> > > > > > > >>>>>>> won’t
> > > > > > > >>>>>>>>>> apply.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For (1) I
> > > > > > > >>>>>> understand
> > > > > > > >>>>>>>>> why a
> > > > > > > >>>>>>>>>>>> client
> > > > > > > >>>>>>>>>>>>>>>>> change
> > > > > > > >>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> necessary,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we need a client
> > > > > > > >>>>>>>> change
> > > > > > > >>>>>>>>>> for
> > > > > > > >>>>>>>>>>>> (2).
> > > > > > > >>>>>>>>>>>>>>> Can
> > > > > > > >>>>>>>>>>>>>>>>> you
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> --
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Later
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> explain
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that we should
> > > > > > > >>>>>> send
> > > > > > > >>>>>>> a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> DescribeTransactionRequest,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> but I
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> am
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Can't we not just
> > > > > > > >>>>>> do
> > > > > > > >>>>>>>> an
> > > > > > > >>>>>>>>>>>> implicit
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> AddPartiitonToTx,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> too?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> If
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> old
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer correctly
> > > > > > > >>>>>>>>>> registered
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>> partition
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> already,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can just ignore it
> > > > > > > >>>>>>> as
> > > > > > > >>>>>>>>>> it's an
> > > > > > > >>>>>>>>>>>>>>>>> idempotent
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> operation?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (30)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To cover older
> > > > > > > >>>>>>>> clients,
> > > > > > > >>>>>>>>>> we
> > > > > > > >>>>>>>>>>>> will
> > > > > > > >>>>>>>>>>>>>>>>> ensure a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ongoing
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> before
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we write to a
> > > > > > > >>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Not sure what you
> > > > > > > >>>>>>> mean
> > > > > > > >>>>>>>>> by
> > > > > > > >>>>>>>>>>>> this?
> > > > > > > >>>>>>>>>>>>>>> Can
> > > > > > > >>>>>>>>>>>>>>>> you
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (40)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [the
> > > > > > > >>>>>>> TX-coordinator]
> > > > > > > >>>>>>>>> will
> > > > > > > >>>>>>>>>>>> write
> > > > > > > >>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>> prepare
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> commit
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch and send
> > > > > > > >>>>>>>>>>>>>>> WriteTxnMarkerRequests
> > > > > > > >>>>>>>>>>>>>>>>>> with
> > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> bumped
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why do we use the
> > > > > > > >>>>>>>> bumped
> > > > > > > >>>>>>>>>>>> epoch for
> > > > > > > >>>>>>>>>>>>>>>>> both?
> > > > > > > >>>>>>>>>>>>>>>>>> It
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> seems
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> more
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> intuitive
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the current epoch,
> > > > > > > >>>>>>> and
> > > > > > > >>>>>>>>>> only
> > > > > > > >>>>>>>>>>>> return
> > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>> bumped
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (50) "Implicit
> > > > > > > >>>>>>>>>>>>>>>>> AddPartitionToTransaction"
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why does the
> > > > > > > >>>>>>>> implicitly
> > > > > > > >>>>>>>>>> sent
> > > > > > > >>>>>>>>>>>>>>> request
> > > > > > > >>>>>>>>>>>>>>>>> need
> > > > > > > >>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> synchronous?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also says
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in case we need
> > > > > > > >>>>>> to
> > > > > > > >>>>>>>>> abort
> > > > > > > >>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>> need to
> > > > > > > >>>>>>>>>>>>>>>>>> know
> > > > > > > >>>>>>>>>>>>>>>>>>>>> which
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What do you mean
> > > > > > > >>>>>> by
> > > > > > > >>>>>>>>> this?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we don’t want to
> > > > > > > >>>>>>>> write
> > > > > > > >>>>>>>>>> to it
> > > > > > > >>>>>>>>>>>>>>> before
> > > > > > > >>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>> store
> > > > > > > >>>>>>>>>>>>>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> manager
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Do you mean
> > > > > > > >>>>>>>>> TX-coordinator
> > > > > > > >>>>>>>>>>>>>>> instead of
> > > > > > > >>>>>>>>>>>>>>>>>>>>> "manager"?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (60)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For older clients
> > > > > > > >>>>>>> and
> > > > > > > >>>>>>>>>> ensuring
> > > > > > > >>>>>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>> TX
> > > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> ongoing,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> describe a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> race condition. I
> > > > > > > >>>>>> am
> > > > > > > >>>>>>>> not
> > > > > > > >>>>>>>>>> sure
> > > > > > > >>>>>>>>>>>> if I
> > > > > > > >>>>>>>>>>>>>>>> can
> > > > > > > >>>>>>>>>>>>>>>>>>>> follow
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> here.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> Can
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/18/22 1:21
> > > > > > > >>>>>> PM,
> > > > > > > >>>>>>>>>> Justine
> > > > > > > >>>>>>>>>>>>>>> Olshan
> > > > > > > >>>>>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey all!
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to
> > > > > > > >>>>>> start a
> > > > > > > >>>>>>>>>>>> discussion
> > > > > > > >>>>>>>>>>>>>>> on my
> > > > > > > >>>>>>>>>>>>>>>>>>>> proposal
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> add
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> some
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checks on
> > > > > > > >>>>>>>> transactions
> > > > > > > >>>>>>>>> to
> > > > > > > >>>>>>>>>>>> avoid
> > > > > > > >>>>>>>>>>>>>>>>> hanging
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> transactions.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> been
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an issue for some
> > > > > > > >>>>>>>> time,
> > > > > > > >>>>>>>>>> so I
> > > > > > > >>>>>>>>>>>>>>> really
> > > > > > > >>>>>>>>>>>>>>>>> hope
> > > > > > > >>>>>>>>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> KIP
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> helpful
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users of EOS.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The KIP includes
> > > > > > > >>>>>>>>> changes
> > > > > > > >>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>> will
> > > > > > > >>>>>>>>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> compatible
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> old
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> changes to
> > > > > > > >>>>>> improve
> > > > > > > >>>>>>>>>>>> performance
> > > > > > > >>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>> correctness
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> on
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Please take a
> > > > > > > >>>>>> look
> > > > > > > >>>>>>>> and
> > > > > > > >>>>>>>>>> leave
> > > > > > > >>>>>>>>>>>> any
> > > > > > > >>>>>>>>>>>>>>>>>> comments
> > > > > > > >>>>>>>>>>>>>>>>>>>> you
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> may
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> have!
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>
> > > > > > > >>>>>>>>>
> > > > > > > >>>>>>>>
> > > > > > > >>>>>>>
> > > > > > > >>>>>>
> > > > > > > >>>>
> > > > > > > >>
> > > > > >
> > > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> JIRA:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-14402
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>
> > > > > > > >>>>>>>>>
> > > > > > > >>>>>>>>
> > > > > > > >>>>>>>
> > > > > > > >>>>>>
> > > > > > > >>>>>
> > > > > > > >>>>
> > > > > > > >>>
> > > > > > > >>
> > > > > > > >
> > > > > >
> > > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Dániel Urbán <ur...@gmail.com>.
Hi,

Sorry for being super-late to this discussion, but a question came to my
mind related to the bump-on-txn-end part of the KIP.
I see that the END_TXN_RESPONSE will be updated to also include the new
epoch to be used by the producer - but what about epoch exhaustion? What
will happen when the last ongoing transaction uses the max available epoch,
and we try to end it? The producer would need a new producer ID after the
epoch bump. Because of this, I think the response should also contain the
producer-id-to-use.

Thanks,
Daniel

Artem Livshits <al...@confluent.io.invalid> ezt írta (időpont: 2023.
jan. 28., Szo, 0:17):

> >  I'd like to check that, under this KIP's circumstances, if we could ever
> have "partial committed" txns
>
> The goal of this KIP is to harden transaction implementation to never have
> partially committed txns.
>
> -Artem
>
> On Fri, Jan 27, 2023 at 10:04 AM Guozhang Wang <guozhang.wang.us@gmail.com
> >
> wrote:
>
> > Thanks folks, I agree with all of your points.
> >
> > I think in the past we did have some cases where abortable txns are
> > handled as fatal ones which we are fixing, but maybe there are some
> > vice versa as well. I'd like to check that, under this KIP's
> > circumstances, if we could ever have "partial committed" txns where
> > some of the data are considered as committed while some others are
> > aborted later. And if there were, could we recover from it. If the
> > answer is "yes and yes", then I think we can bite the complexity
> > bullet inside the client and leave users in comfort and peace :)
> >
> > Guozhang
> >
> > On Fri, Jan 27, 2023 at 9:42 AM Artem Livshits
> > <al...@confluent.io.invalid> wrote:
> > >
> > > I agree.  I also think we should go a little further and be more
> > > prescriptive with our recommendations for error handling (i.e. not just
> > > think if there is a chance, but propose design patterns for typical
> > > cases).  For example, if a transaction is aborted, the application must
> > > either keep enough state to be able to redo the operation from the
> > > beginning, or reset all state and fully recover from Kafka (i.e.
> > re-consume
> > > data from the last committed offsets).
> > >
> > > For failed commits, we need a way for the application to either learn
> the
> > > result (if commit went through, then continue, otherwise redo
> operation)
> > or
> > > just reset all state and fully recover from Kafka (i.e. re-consume data
> > > from the last committed offsets).
> > >
> > > -Artem
> > >
> > > On Fri, Jan 27, 2023 at 9:31 AM Justine Olshan
> > <jo...@confluent.io.invalid>
> > > wrote:
> > >
> > > > Thanks for clarifying Guozhang,
> > > >
> > > > I like the framing:
> > > > * “if we did abort the txn and continue, could the app ever has a
> > chance to
> > > > recover and not have any side effects violating EOS guarantees”.*
> > > >
> > > > With this principle in mind, if we could convince ourselves that we
> > won't
> > > > have side effects, then we could move forward with a non-fatal
> error. I
> > > > appreciate laying out the cases we've discussed. I think I also want
> to
> > > > take a look in very close detail and convince myself of the right
> > option.
> > > > I'll get back to this thread with the final conclusion as well as
> > update
> > > > the KIP.
> > > >
> > > > Thanks for also bringing up KIP-691. I should take a look there as
> > well. :)
> > > >
> > > > Justine
> > > >
> > > > On Thu, Jan 26, 2023 at 5:28 AM Guozhang Wang <
> > guozhang.wang.us@gmail.com>
> > > > wrote:
> > > >
> > > > > Hello Justine,
> > > > >
> > > > > Regarding which errors should be fatal v.s. abortable, I think the
> > > > > principle would be “if we did abort the txn and continue, could the
> > > > > app ever has a chance to recover and not have any side effects
> > > > > violating EOS guarantees”.Of course we can discuss whether that’s a
> > > > > good principle  e.g. one can argue that maybe it’s okay to violate
> > EOS
> > > > > some times rather than killing the whole app, whereas my rationale
> is
> > > > > that since we want to make EOS the default config, it’s crucial
> that
> > > > > we try to guarantee it as much as possible and whenever we detected
> > if
> > > > > there’s a chance that it maybe broken, we should let users know.
> And
> > > > > if we just log an error in app and try to abort and continue, it’s
> > > > > very likely that users would not be notified and probably only
> > realize
> > > > > that after a while, which may cause an even harder scenario to
> > > > > debug.If people agree on that principle, we can go back and check
> > this
> > > > > KIP’s scenario: there are several scenarios causing a partition
> > leader
> > > > > detects an out of date epoch from a produce requests:
> > > > >
> > > > > * The old produce requests arrive late after a network partition,
> and
> > > > > the old producer is already gone.
> > > > > * The old produce requests arrive late after a network partition,
> but
> > > > > the old producer is still around.
> > > > > * A buggy producer client that did not follow the protocol.
> > > > >
> > > > > For the third case, we should try to let user know asap, and as
> clear
> > > > > as possible, and hence it’s best to just stop the client app; for
> the
> > > > > first case, it does not matter since the producer is already gone;
> > > > > only for the second, probably the least likely case, we need to
> > > > > ponder, and there my rationale again is that at that time, we may
> > have
> > > > > already violated the EOS guarantees since there are some partial
> txn
> > > > > records that should be rejected while the txn itself has been
> > > > > committed. In this case I think it’s better to let clients/users
> know
> > > > > as soon as possible as well.
> > > > >
> > > > > Regarding the past scenarios where a fatal error killing the whole
> > > > > apps, I believe part of that reason is that we were doing an
> > > > > sub-optimal job on clients side handling various error cases and
> > > > > that’s what KIP-691 is trying to resolve, and hence personally I
> > would
> > > > > suggest we do not weight in too much on that if we can trust that
> > > > > KIP-691 will be successfully eliminate those
> not-necessary-hard-fail
> > > > > scenarios.
> > > > >
> > > > >
> > > > > Guozhang
> > > > >
> > > > > On Wed, Jan 25, 2023 at 5:51 PM Matthias J. Sax <mj...@apache.org>
> > > > wrote:
> > > > > >
> > > > > > So the timestamp would be set when the write happens and thus no
> > > > > > out-of-order data (base in time) can be introduced with
> > "append_time"
> > > > > > config even if a request sits in purgatory first while we check
> > the TX
> > > > > > status.
> > > > > >
> > > > > > That does make sense. Thanks for confirming, that there is no
> > > > > > out-of-order issue for this case.
> > > > > >
> > > > > >
> > > > > > -Matthias
> > > > > >
> > > > > > On 1/25/23 5:04 PM, Justine Olshan wrote:
> > > > > > > Hey Matthias,
> > > > > > > Let me put it this way, if a producer is checking if a
> > transaction is
> > > > > > > ongoing, then no writes to the partition from the producer will
> > go
> > > > > through
> > > > > > > until the transaction is confirmed ongoing.
> > > > > > >  From then, I think I can apply the writes in the order they
> > came in.
> > > > > Does
> > > > > > > that make sense?
> > > > > > >
> > > > > > > Let me know if I'm missing something.
> > > > > > > Justine
> > > > > > >
> > > > > > > On Wed, Jan 25, 2023 at 4:57 PM Matthias J. Sax <
> > mjsax@apache.org>
> > > > > wrote:
> > > > > > >
> > > > > > >>> would it build an offset map with just the latest timestamp
> > for a
> > > > > key?
> > > > > > >>
> > > > > > >> Cannot remember the details without reading the KIP, but yes,
> > > > > something
> > > > > > >> like this (I believe it actually needs to track both, offset
> and
> > > > > > >> timestamp per key).
> > > > > > >>
> > > > > > >>> I wonder if ordering assumptions are baked in there, why not
> > use
> > > > > > >> offset-based compaction.
> > > > > > >>
> > > > > > >> The use case is a compacted topic that does contain
> out-of-order
> > > > data.
> > > > > > >> If you set key k1=v1 @ 5 offset 100 and later key1 = v0 @ 3 at
> > > > offset
> > > > > > >> 200 we want to cleanup v0 with higher offset because it's
> > > > out-of-order
> > > > > > >> based on time, but keep v1 what is the actual latest version
> of
> > k1.
> > > > > > >>
> > > > > > >>
> > > > > > >>> I was also not aware of this "guarantee" with regards to
> broker
> > > > side
> > > > > > >> time.
> > > > > > >>
> > > > > > >> As already said: I am not sure if it's a public contract, but
> > based
> > > > on
> > > > > > >> my experience, people might reply on it as "implicit
> contract".
> > --
> > > > > Maybe
> > > > > > >> somebody else knows if it's public or not, and if it would be
> > ok to
> > > > > > >> "break" it.
> > > > > > >>
> > > > > > >>> Let me know if you have any concerns here.
> > > > > > >>
> > > > > > >> My understanding is: While we cannot make an offset-order
> > guarantee
> > > > > for
> > > > > > >> interleaved writes of different producer, if the topic is
> > configures
> > > > > > >> with "append_time", we "guarantee" (cf. my comment above")
> > timestamp
> > > > > > >> order... If that's the case, it would be an issue if we break
> > this
> > > > > > >> "guarantee".
> > > > > > >>
> > > > > > >> I am not sure when the broker sets the timestamp for
> > "append_time"
> > > > > > >> config? If we do it before putting the request into purgatory,
> > we
> > > > > have a
> > > > > > >> problem. However, if we set the timestamp when we actually
> > process
> > > > the
> > > > > > >> request and do the actual append, it seems there is no issue,
> > as the
> > > > > > >> request that was waiting in purgatory get the "newest"
> > timestamp and
> > > > > > >> thus cannot introduce out-of-order data.
> > > > > > >>
> > > > > > >>
> > > > > > >> -Matthias
> > > > > > >>
> > > > > > >>
> > > > > > >> On 1/24/23 10:44 AM, Justine Olshan wrote:
> > > > > > >>> Hey Matthias,
> > > > > > >>>
> > > > > > >>> I have actually never heard of KIP-280 so thanks for bringing
> > it
> > > > up.
> > > > > That
> > > > > > >>> seems interesting. I wonder how it would work though -- would
> > it
> > > > > build an
> > > > > > >>> offset map with just the latest timestamp for a key? I wonder
> > if
> > > > > ordering
> > > > > > >>> assumptions are baked in there, why not use offset-based
> > > > compaction.
> > > > > > >>>
> > > > > > >>> I was also not aware of this "guarantee" with regards to
> broker
> > > > side
> > > > > > >> time.
> > > > > > >>> I think that we can do in order handling for a given
> producer,
> > but
> > > > > not
> > > > > > >>> across all producers. However, we can't guarantee that
> anyway.
> > > > > > >>>
> > > > > > >>> Let me know if you have any concerns here.
> > > > > > >>>
> > > > > > >>> Thanks,
> > > > > > >>> Justine
> > > > > > >>>
> > > > > > >>> On Mon, Jan 23, 2023 at 6:33 PM Matthias J. Sax <
> > mjsax@apache.org>
> > > > > > >> wrote:
> > > > > > >>>
> > > > > > >>>> Just a side note about Guozhang comments about timestamps.
> > > > > > >>>>
> > > > > > >>>> If the producer sets the timestamp, putting the record into
> > > > > purgatory
> > > > > > >>>> seems not to be an issue (as already said: for this case we
> > don't
> > > > > > >>>> guarantee timestamp order between writes of different
> > producers
> > > > > anyway).
> > > > > > >>>> However, if the broker sets the timestamp, the expectation
> is
> > that
> > > > > there
> > > > > > >>>> is no out-of-order data in the partition ever; if we would
> > > > introduce
> > > > > > >>>> out-of-order data for this case (for interleaved writes of
> > > > different
> > > > > > >>>> producers), it seems we would violate the current contract?
> > (To be
> > > > > fair:
> > > > > > >>>> I don't know if that's an official contract, but I assume
> > people
> > > > > rely on
> > > > > > >>>> this behavior -- and it "advertised" in many public
> talks...)
> > > > > > >>>>
> > > > > > >>>> About compaction: there is actually KIP-280 that adds
> > timestamp
> > > > > based
> > > > > > >>>> compaction what is a very useful feature for Kafka Streams
> > with
> > > > > regard
> > > > > > >>>> to out-of-order data handling. So the impact if we introduce
> > > > > > >>>> out-of-order data could be larger scoped.
> > > > > > >>>>
> > > > > > >>>>
> > > > > > >>>> -Matthias
> > > > > > >>>>
> > > > > > >>>>
> > > > > > >>>> On 1/20/23 4:48 PM, Justine Olshan wrote:
> > > > > > >>>>> Hey Artem,
> > > > > > >>>>>
> > > > > > >>>>> I see there is a check for transactional producers. I'm
> > wondering
> > > > > if we
> > > > > > >>>>> don't handle the epoch overflow case. I'm also not sure it
> > will
> > > > be
> > > > > a
> > > > > > >> huge
> > > > > > >>>>> issue to extend to transactional producers, but maybe I'm
> > missing
> > > > > > >>>> something.
> > > > > > >>>>>
> > > > > > >>>>> As for the recovery path -- I think Guozhang's point was if
> > we
> > > > > have a
> > > > > > >> bad
> > > > > > >>>>> client that repeatedly tries to produce without adding to
> the
> > > > > > >> transaction
> > > > > > >>>>> we would do the following:
> > > > > > >>>>> a) if not fatal, we just fail the produce request over and
> > over
> > > > > > >>>>> b) if fatal, we fence the producer
> > > > > > >>>>>
> > > > > > >>>>> Here with B, the issue with the client would be made clear
> > more
> > > > > > >> quickly.
> > > > > > >>>> I
> > > > > > >>>>> suppose there are some intermediate cases where the issue
> > only
> > > > > occurs
> > > > > > >>>>> sometimes, but I wonder if we should consider how to
> recover
> > with
> > > > > > >> clients
> > > > > > >>>>> who don't behave as expected anyway.
> > > > > > >>>>>
> > > > > > >>>>> I think there is a place for the abortable error that we
> are
> > > > > adding --
> > > > > > >>>> just
> > > > > > >>>>> abort and try again. But I think there are also some cases
> > where
> > > > > trying
> > > > > > >>>> to
> > > > > > >>>>> recover overcomplicates some logic. Especially if we are
> > > > > considering
> > > > > > >>>> older
> > > > > > >>>>> clients -- there I'm not sure if there's a ton we can do
> > besides
> > > > > fail
> > > > > > >> the
> > > > > > >>>>> batch or fence the producer. With newer clients, we can
> > consider
> > > > > more
> > > > > > >>>>> options for what can just be recovered after aborting. But
> > epochs
> > > > > might
> > > > > > >>>> be
> > > > > > >>>>> a hard one unless we also want to reset producer ID.
> > > > > > >>>>>
> > > > > > >>>>> Thanks,
> > > > > > >>>>> Justine
> > > > > > >>>>>
> > > > > > >>>>>
> > > > > > >>>>>
> > > > > > >>>>> On Fri, Jan 20, 2023 at 3:59 PM Artem Livshits
> > > > > > >>>>> <al...@confluent.io.invalid> wrote:
> > > > > > >>>>>
> > > > > > >>>>>>>     besides the poorly written client case
> > > > > > >>>>>>
> > > > > > >>>>>> A poorly written client could create a lot of grief to
> > people
> > > > who
> > > > > run
> > > > > > >>>> Kafka
> > > > > > >>>>>> brokers :-), so when deciding to make an error fatal I
> > would see
> > > > > if
> > > > > > >>>> there
> > > > > > >>>>>> is a reasonable recovery path rather than how often it
> could
> > > > > happen.
> > > > > > >>>> If we
> > > > > > >>>>>> have solid implementation of transactions (which I hope
> > we'll do
> > > > > as a
> > > > > > >>>>>> result of this KIP), it would help to recover from a large
> > class
> > > > > of
> > > > > > >>>> errors
> > > > > > >>>>>> by just aborting a transaction, even if the cause of error
> > is a
> > > > > race
> > > > > > >>>>>> condition or etc.
> > > > > > >>>>>>
> > > > > > >>>>>> -Artem
> > > > > > >>>>>>
> > > > > > >>>>>> On Fri, Jan 20, 2023 at 3:26 PM Justine Olshan
> > > > > > >>>>>> <jo...@confluent.io.invalid>
> > > > > > >>>>>> wrote:
> > > > > > >>>>>>
> > > > > > >>>>>>> Artem --
> > > > > > >>>>>>> I guess the discussion path we were going down is when we
> > > > expect
> > > > > to
> > > > > > >> see
> > > > > > >>>>>>> this error. I mentioned that it was hard to come up with
> > cases
> > > > > for
> > > > > > >> when
> > > > > > >>>>>> the
> > > > > > >>>>>>> producer would still be around to receive the error
> > besides the
> > > > > > >> poorly
> > > > > > >>>>>>> written client case.
> > > > > > >>>>>>> If we don't expect to have a producer to receive the
> > response,
> > > > it
> > > > > > >> sort
> > > > > > >>>> of
> > > > > > >>>>>>> makes sense for it to be fatal.
> > > > > > >>>>>>>
> > > > > > >>>>>>> I had some discussion with Jason offline about the epoch
> > being
> > > > > off
> > > > > > >>>> cases
> > > > > > >>>>>>> and I'm not sure we could find a ton (outside of produce
> > > > > requests)
> > > > > > >>>> where
> > > > > > >>>>>> we
> > > > > > >>>>>>> could/should recover. I'd be happy to hear some examples
> > > > though,
> > > > > > >> maybe
> > > > > > >>>>>> I'm
> > > > > > >>>>>>> missing something.
> > > > > > >>>>>>>
> > > > > > >>>>>>> Thanks,
> > > > > > >>>>>>> Justine
> > > > > > >>>>>>>
> > > > > > >>>>>>> On Fri, Jan 20, 2023 at 3:19 PM Artem Livshits
> > > > > > >>>>>>> <al...@confluent.io.invalid> wrote:
> > > > > > >>>>>>>
> > > > > > >>>>>>>> In general, I'd like to avoid fatal errors as much as
> > > > possible,
> > > > > in
> > > > > > >>>> some
> > > > > > >>>>>>>> sense fatal errors just push out recovery logic to the
> > > > > application
> > > > > > >>>>>> which
> > > > > > >>>>>>>> either complicates the application or leads to
> disruption
> > > > (we've
> > > > > > >> seen
> > > > > > >>>>>>> cases
> > > > > > >>>>>>>> when a transient broker error could lead to work
> stoppage
> > when
> > > > > > >>>>>>> applications
> > > > > > >>>>>>>> need to be manually restarted).  I think we should
> strive
> > to
> > > > > define
> > > > > > >>>>>>>> recovery logic for most errors (and/or encapsulate it in
> > the
> > > > > Kafka
> > > > > > >>>>>> client
> > > > > > >>>>>>>> as much as possible).
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> One benefit of transactions is that they simplify
> recovery
> > > > from
> > > > > > >>>> errors,
> > > > > > >>>>>>>> pretty much any error (that's not handled transparently
> by
> > > > > retries
> > > > > > >> in
> > > > > > >>>>>>> Kafka
> > > > > > >>>>>>>> client) can be handled by the application via aborting
> the
> > > > > > >> transaction
> > > > > > >>>>>>> and
> > > > > > >>>>>>>> repeating the transactional logic again.  One tricky
> > error is
> > > > an
> > > > > > >> error
> > > > > > >>>>>>>> during commit, because we don't know the outcome.  For
> > commit
> > > > > > >> errors,
> > > > > > >>>>>> the
> > > > > > >>>>>>>> recommendation should be to retry the commit until it
> > returns
> > > > > the
> > > > > > >>>>>>> specific
> > > > > > >>>>>>>> result (committed or aborted).
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> -Artem
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> On Fri, Jan 20, 2023 at 2:52 PM Justine Olshan
> > > > > > >>>>>>>> <jo...@confluent.io.invalid>
> > > > > > >>>>>>>> wrote:
> > > > > > >>>>>>>>
> > > > > > >>>>>>>>> That's a fair point about other clients.
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> I think the abortable error case is interesting because
> > I'm
> > > > > curious
> > > > > > >>>>>> how
> > > > > > >>>>>>>>> other clients would handle this. I assume they would
> > need to
> > > > > > >>>>>> implement
> > > > > > >>>>>>>>> handling for the error code unless they did something
> > like
> > > > "any
> > > > > > >>>>>> unknown
> > > > > > >>>>>>>>> error codes/any codes that aren't x,y,z are
> retriable." I
> > > > would
> > > > > > >> hope
> > > > > > >>>>>>> that
> > > > > > >>>>>>>>> unknown error codes were fatal, and if the code was
> > > > > implemented it
> > > > > > >>>>>>> would
> > > > > > >>>>>>>>> abort the transaction. But I will think on this too.
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> As for InvalidRecord -- you mentioned it was not fatal,
> > but
> > > > I'm
> > > > > > >>>>>> taking
> > > > > > >>>>>>> a
> > > > > > >>>>>>>>> look through the code. We would see this on handling
> the
> > > > > produce
> > > > > > >>>>>>>> response.
> > > > > > >>>>>>>>> If I recall correctly, we check if errors are
> retriable.
> > I
> > > > > think
> > > > > > >> this
> > > > > > >>>>>>>> error
> > > > > > >>>>>>>>> would not be retriable. But I guess the concern here is
> > that
> > > > > it is
> > > > > > >>>>>> not
> > > > > > >>>>>>>>> enough for just that batch to fail. I guess I hadn't
> > > > considered
> > > > > > >> fully
> > > > > > >>>>>>>>> fencing the old producer but there are valid arguments
> > here
> > > > > why we
> > > > > > >>>>>>> would
> > > > > > >>>>>>>>> want to.
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> Thanks,
> > > > > > >>>>>>>>> Justine
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> On Fri, Jan 20, 2023 at 2:35 PM Guozhang Wang <
> > > > > > >>>>>>>> guozhang.wang.us@gmail.com>
> > > > > > >>>>>>>>> wrote:
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>> Thanks Justine for the replies! I agree with most of
> > your
> > > > > > >> thoughts.
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>> Just for 3/7), though I agree for our own AK producer,
> > since
> > > > > we do
> > > > > > >>>>>>>>>> "nextRequest(boolean hasIncompleteBatches)", we
> > guarantee
> > > > the
> > > > > > >>>>>> end-txn
> > > > > > >>>>>>>>>> would not be sent until we've effectively flushed, but
> > I was
> > > > > > >>>>>>> referring
> > > > > > >>>>>>>>>> to any future bugs or other buggy clients that the
> same
> > > > > client may
> > > > > > >>>>>>> get
> > > > > > >>>>>>>>>> into this situation, in which case we should give the
> > > > client a
> > > > > > >>>>>> clear
> > > > > > >>>>>>>>>> msg that "you did something wrong, and hence now you
> > should
> > > > > > >> fatally
> > > > > > >>>>>>>>>> close yourself". What I'm concerned about is that, by
> > seeing
> > > > > an
> > > > > > >>>>>>>>>> "abortable error" or in some rare cases an "invalid
> > record",
> > > > > the
> > > > > > >>>>>>>>>> client could not realize "something that's really bad
> > > > > happened".
> > > > > > >> So
> > > > > > >>>>>>>>>> it's not about adding a new error, it's mainly about
> > those
> > > > > real
> > > > > > >>>>>> buggy
> > > > > > >>>>>>>>>> situations causing such "should never happen" cases,
> the
> > > > > errors
> > > > > > >>>>>>> return
> > > > > > >>>>>>>>>> would not be informative enough.
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>> Thinking in other ways, if we believe that for most
> > cases
> > > > such
> > > > > > >>>>>> error
> > > > > > >>>>>>>>>> codes would not reach the original clients since they
> > would
> > > > be
> > > > > > >>>>>>>>>> disconnected or even gone by that time, and only in
> some
> > > > rare
> > > > > > >> cases
> > > > > > >>>>>>>>>> they would still be seen by the sending clients, then
> > why
> > > > not
> > > > > make
> > > > > > >>>>>>>>>> them more fatal and more specific than generic.
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>> Guozhang
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>> On Fri, Jan 20, 2023 at 1:59 PM Justine Olshan
> > > > > > >>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>> Hey Guozhang. Thanks for taking a look and for the
> > detailed
> > > > > > >>>>>>> comments!
> > > > > > >>>>>>>>>> I'll
> > > > > > >>>>>>>>>>> do my best to address below.
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>> 1. I see what you are saying here, but I think I need
> > to
> > > > look
> > > > > > >>>>>>> through
> > > > > > >>>>>>>>> the
> > > > > > >>>>>>>>>>> sequence of events you mention. Typically we've seen
> > this
> > > > > issue
> > > > > > >>>>>> in
> > > > > > >>>>>>> a
> > > > > > >>>>>>>>> few
> > > > > > >>>>>>>>>>> cases.
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>>     One is when we have a producer disconnect when
> > trying
> > > > to
> > > > > > >>>>>> produce.
> > > > > > >>>>>>>>>>> Typically in these cases, we abort the transaction.
> > We've
> > > > > seen
> > > > > > >>>>>> that
> > > > > > >>>>>>>>> after
> > > > > > >>>>>>>>>>> the markers are written, the disconnection can
> > sometimes
> > > > > cause
> > > > > > >>>>>> the
> > > > > > >>>>>>>>>> request
> > > > > > >>>>>>>>>>> to get flushed to the broker. In this case, we don't
> > need
> > > > > client
> > > > > > >>>>>>>>> handling
> > > > > > >>>>>>>>>>> because the producer we are responding to is gone. We
> > just
> > > > > needed
> > > > > > >>>>>>> to
> > > > > > >>>>>>>>> make
> > > > > > >>>>>>>>>>> sure we didn't write to the log on the broker side.
> I'm
> > > > > trying to
> > > > > > >>>>>>>> think
> > > > > > >>>>>>>>>> of
> > > > > > >>>>>>>>>>> a case where we do have the client to return to. I'd
> > think
> > > > > the
> > > > > > >>>>>> same
> > > > > > >>>>>>>>>> client
> > > > > > >>>>>>>>>>> couldn't progress to committing the transaction
> unless
> > the
> > > > > > >>>>>> produce
> > > > > > >>>>>>>>>> request
> > > > > > >>>>>>>>>>> returned right? Of course, there is the incorrectly
> > written
> > > > > > >>>>>> clients
> > > > > > >>>>>>>>> case.
> > > > > > >>>>>>>>>>> I'll think on this a bit more and let you know if I
> > come up
> > > > > with
> > > > > > >>>>>>>>> another
> > > > > > >>>>>>>>>>> scenario when we would return to an active client
> when
> > the
> > > > > > >>>>>>>> transaction
> > > > > > >>>>>>>>> is
> > > > > > >>>>>>>>>>> no longer ongoing.
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>> I was not aware that we checked the result of a send
> > after
> > > > we
> > > > > > >>>>>>> commit
> > > > > > >>>>>>>>>>> though. I'll need to look into that a bit more.
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>> 2. There were some questions about this in the
> > discussion.
> > > > > The
> > > > > > >>>>>> plan
> > > > > > >>>>>>>> is
> > > > > > >>>>>>>>> to
> > > > > > >>>>>>>>>>> handle overflow with the mechanism we currently have
> > in the
> > > > > > >>>>>>> producer.
> > > > > > >>>>>>>>> If
> > > > > > >>>>>>>>>> we
> > > > > > >>>>>>>>>>> try to bump and the epoch will overflow, we actually
> > > > > allocate a
> > > > > > >>>>>> new
> > > > > > >>>>>>>>>>> producer ID. I need to confirm the fencing logic on
> the
> > > > last
> > > > > > >>>>>> epoch
> > > > > > >>>>>>>> (ie,
> > > > > > >>>>>>>>>> we
> > > > > > >>>>>>>>>>> probably shouldn't allow any records to be produced
> > with
> > > > the
> > > > > > >>>>>> final
> > > > > > >>>>>>>>> epoch
> > > > > > >>>>>>>>>>> since we can never properly fence that one).
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>> 3. I can agree with you that the current error
> > handling is
> > > > > > >>>>>> messy. I
> > > > > > >>>>>>>>>> recall
> > > > > > >>>>>>>>>>> taking a look at your KIP a while back, but I think I
> > > > mostly
> > > > > saw
> > > > > > >>>>>>> the
> > > > > > >>>>>>>>>>> section about how the errors were wrapped. Maybe I
> > need to
> > > > > take
> > > > > > >>>>>>>> another
> > > > > > >>>>>>>>>>> look. As for abortable error, the idea was that the
> > > > handling
> > > > > > >>>>>> would
> > > > > > >>>>>>> be
> > > > > > >>>>>>>>>>> simple -- if this error is seen, the transaction
> > should be
> > > > > > >>>>>> aborted
> > > > > > >>>>>>> --
> > > > > > >>>>>>>>> no
> > > > > > >>>>>>>>>>> other logic about previous state or requests
> > necessary. Is
> > > > > your
> > > > > > >>>>>>>> concern
> > > > > > >>>>>>>>>>> simply about adding new errors? We were hoping to
> have
> > an
> > > > > error
> > > > > > >>>>>>> that
> > > > > > >>>>>>>>>> would
> > > > > > >>>>>>>>>>> have one meaning and many of the current errors have
> a
> > > > > history of
> > > > > > >>>>>>>>> meaning
> > > > > > >>>>>>>>>>> different things on different client versions. That
> > was the
> > > > > main
> > > > > > >>>>>>>>>> motivation
> > > > > > >>>>>>>>>>> for adding a new error.
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>> 4. This is a good point about record timestamp
> > reordering.
> > > > > > >>>>>>> Timestamps
> > > > > > >>>>>>>>>> don't
> > > > > > >>>>>>>>>>> affect compaction, but they do affect retention
> > deletion.
> > > > For
> > > > > > >>>>>> that,
> > > > > > >>>>>>>>> kafka
> > > > > > >>>>>>>>>>> considers the largest timestamp in the segment, so I
> > think
> > > > a
> > > > > > >>>>>> small
> > > > > > >>>>>>>>> amount
> > > > > > >>>>>>>>>>> of reordering (hopefully on the order of milliseconds
> > or
> > > > even
> > > > > > >>>>>>>> seconds)
> > > > > > >>>>>>>>>> will
> > > > > > >>>>>>>>>>> be ok. We take timestamps from clients so there is
> > already
> > > > a
> > > > > > >>>>>>>>> possibility
> > > > > > >>>>>>>>>>> for some drift and non-monotonically increasing
> > timestamps.
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>> 5. Thanks for catching. The error is there, but it's
> > > > actually
> > > > > > >>>>>> that
> > > > > > >>>>>>>>> those
> > > > > > >>>>>>>>>>> fields should be 4+! Due to how the message generator
> > > > works,
> > > > > I
> > > > > > >>>>>>>> actually
> > > > > > >>>>>>>>>>> have to redefine those fields inside the
> > > > > > >>>>>>>>> `"AddPartitionsToTxnTransaction`
> > > > > > >>>>>>>>>>> block for it to build correctly. I'll fix it to be
> > correct.
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>> 6. Correct -- we will only add the request to
> > purgatory if
> > > > > the
> > > > > > >>>>>>> cache
> > > > > > >>>>>>>>> has
> > > > > > >>>>>>>>>> no
> > > > > > >>>>>>>>>>> ongoing transaction. I can change the wording to make
> > that
> > > > > > >>>>>> clearer
> > > > > > >>>>>>>> that
> > > > > > >>>>>>>>>> we
> > > > > > >>>>>>>>>>> only place the request in purgatory if we need to
> > contact
> > > > the
> > > > > > >>>>>>>>> transaction
> > > > > > >>>>>>>>>>> coordinator.
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>> 7. We did take a look at some of the errors and it
> was
> > hard
> > > > > to
> > > > > > >>>>>> come
> > > > > > >>>>>>>> up
> > > > > > >>>>>>>>>> with
> > > > > > >>>>>>>>>>> a good one. I agree that InvalidTxnStateException is
> > ideal
> > > > > except
> > > > > > >>>>>>> for
> > > > > > >>>>>>>>> the
> > > > > > >>>>>>>>>>> fact that it hasn't been returned on Produce requests
> > > > > before. The
> > > > > > >>>>>>>> error
> > > > > > >>>>>>>>>>> handling for clients is a bit vague (which is why I
> > opened
> > > > > > >>>>>>>> KAFKA-14439
> > > > > > >>>>>>>>>>> <https://issues.apache.org/jira/browse/KAFKA-14439
> >),
> > but
> > > > > the
> > > > > > >>>>>>>> decision
> > > > > > >>>>>>>>>> we
> > > > > > >>>>>>>>>>> made here was to only return errors that have been
> > > > previously
> > > > > > >>>>>>>> returned
> > > > > > >>>>>>>>> to
> > > > > > >>>>>>>>>>> producers. As for not being fatal, I think part of
> the
> > > > > theory was
> > > > > > >>>>>>>> that
> > > > > > >>>>>>>>> in
> > > > > > >>>>>>>>>>> many cases, the producer would be disconnected. (See
> > point
> > > > > 1) and
> > > > > > >>>>>>>> this
> > > > > > >>>>>>>>>>> would just be an error to return from the server. I
> did
> > > > plan
> > > > > to
> > > > > > >>>>>>> think
> > > > > > >>>>>>>>>> about
> > > > > > >>>>>>>>>>> other cases, so let me know if you think of any as
> > well!
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>> Lots to say! Let me know if you have further
> thoughts!
> > > > > > >>>>>>>>>>> Justine
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>> On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang <
> > > > > > >>>>>>>>>> guozhang.wang.us@gmail.com>
> > > > > > >>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>>> Hello Justine,
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> Thanks for the great write-up! I made a quick pass
> > through
> > > > > it
> > > > > > >>>>>> and
> > > > > > >>>>>>>>> here
> > > > > > >>>>>>>>>>>> are some thoughts (I have not been able to read
> > through
> > > > this
> > > > > > >>>>>>> thread
> > > > > > >>>>>>>>> so
> > > > > > >>>>>>>>>>>> pardon me if they have overlapped or subsumed by
> > previous
> > > > > > >>>>>>>> comments):
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> First are some meta ones:
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> 1. I think we need to also improve the client's
> > experience
> > > > > once
> > > > > > >>>>>>> we
> > > > > > >>>>>>>>>>>> have this defence in place. More concretely, say a
> > user's
> > > > > > >>>>>>> producer
> > > > > > >>>>>>>>>>>> code is like following:
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> future = producer.send();
> > > > > > >>>>>>>>>>>> // producer.flush();
> > > > > > >>>>>>>>>>>> producer.commitTransaction();
> > > > > > >>>>>>>>>>>> future.get();
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> Which resulted in the order of a) produce-request
> > sent by
> > > > > > >>>>>>> producer,
> > > > > > >>>>>>>>> b)
> > > > > > >>>>>>>>>>>> end-txn-request sent by producer, c)
> end-txn-response
> > sent
> > > > > > >>>>>> back,
> > > > > > >>>>>>> d)
> > > > > > >>>>>>>>>>>> txn-marker-request sent from coordinator to
> partition
> > > > > leader,
> > > > > > >>>>>> e)
> > > > > > >>>>>>>>>>>> produce-request finally received by the partition
> > leader,
> > > > > > >>>>>> before
> > > > > > >>>>>>>> this
> > > > > > >>>>>>>>>>>> KIP e) step would be accepted causing a dangling
> txn;
> > now
> > > > it
> > > > > > >>>>>>> would
> > > > > > >>>>>>>> be
> > > > > > >>>>>>>>>>>> rejected in step e) which is good. But from the
> > client's
> > > > > point
> > > > > > >>>>>> of
> > > > > > >>>>>>>>> view
> > > > > > >>>>>>>>>>>> now it becomes confusing since the
> > `commitTransaction()`
> > > > > > >>>>>> returns
> > > > > > >>>>>>>>>>>> successfully, but the "future" throws an
> invalid-epoch
> > > > > error,
> > > > > > >>>>>> and
> > > > > > >>>>>>>>> they
> > > > > > >>>>>>>>>>>> are not sure if the transaction did succeed or not.
> In
> > > > > fact, it
> > > > > > >>>>>>>>>>>> "partially succeeded" with some msgs being rejected
> > but
> > > > > others
> > > > > > >>>>>>>>>>>> committed successfully.
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> Of course the easy way to avoid this is, always call
> > > > > > >>>>>>>>>>>> "producer.flush()" before commitTxn and that's what
> > we do
> > > > > > >>>>>>>> ourselves,
> > > > > > >>>>>>>>>>>> and what we recommend users do. But I suspect not
> > everyone
> > > > > does
> > > > > > >>>>>>> it.
> > > > > > >>>>>>>>> In
> > > > > > >>>>>>>>>>>> fact I just checked the javadoc in KafkaProducer and
> > our
> > > > > code
> > > > > > >>>>>>>> snippet
> > > > > > >>>>>>>>>>>> does not include a `flush()` call. So I'm thinking
> > maybe
> > > > we
> > > > > can
> > > > > > >>>>>>> in
> > > > > > >>>>>>>>>>>> side the `commitTxn` code to enforce flushing before
> > > > sending
> > > > > > >>>>>> the
> > > > > > >>>>>>>>>>>> end-txn request.
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> 2. I'd like to clarify a bit details on "just add
> > > > > partitions to
> > > > > > >>>>>>> the
> > > > > > >>>>>>>>>>>> transaction on the first produce request during a
> > > > > transaction".
> > > > > > >>>>>>> My
> > > > > > >>>>>>>>>>>> understanding is that the partition leader's cache
> > has the
> > > > > > >>>>>>> producer
> > > > > > >>>>>>>>> id
> > > > > > >>>>>>>>>>>> / sequence / epoch for the latest txn, either
> > on-going or
> > > > is
> > > > > > >>>>>>>>> completed
> > > > > > >>>>>>>>>>>> (upon receiving the marker request from
> coordinator).
> > > > When a
> > > > > > >>>>>>>> produce
> > > > > > >>>>>>>>>>>> request is received, if
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> * producer's epoch < cached epoch, or producer's
> > epoch ==
> > > > > > >>>>>> cached
> > > > > > >>>>>>>>> epoch
> > > > > > >>>>>>>>>>>> but the latest txn is completed, leader directly
> > reject
> > > > with
> > > > > > >>>>>>>>>>>> invalid-epoch.
> > > > > > >>>>>>>>>>>> * producer's epoch > cached epoch, park the the
> > request
> > > > and
> > > > > > >>>>>> send
> > > > > > >>>>>>>>>>>> add-partitions request to coordinator.
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> In order to do it, does the coordinator need to bump
> > the
> > > > > > >>>>>> sequence
> > > > > > >>>>>>>> and
> > > > > > >>>>>>>>>>>> reset epoch to 0 when the next epoch is going to
> > overflow?
> > > > > If
> > > > > > >>>>>> no
> > > > > > >>>>>>>> need
> > > > > > >>>>>>>>>>>> to do so, then how we handle the (admittedly rare,
> but
> > > > still
> > > > > > >>>>>> may
> > > > > > >>>>>>>>>>>> happen) epoch overflow situation?
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> 3. I'm a bit concerned about adding a generic
> > > > > "ABORTABLE_ERROR"
> > > > > > >>>>>>>> given
> > > > > > >>>>>>>>>>>> we already have a pretty messy error classification
> > and
> > > > > error
> > > > > > >>>>>>>>> handling
> > > > > > >>>>>>>>>>>> on the producer clients side --- I have a summary
> > about
> > > > the
> > > > > > >>>>>>> issues
> > > > > > >>>>>>>>> and
> > > > > > >>>>>>>>>>>> a proposal to address this in
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>
> > > > > > >>>>
> > > > > > >>
> > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
> > > > > > >>>>>>>>>>>> -- I understand we do not want to use
> > > > "UNKNOWN_PRODUCER_ID"
> > > > > > >>>>>>> anymore
> > > > > > >>>>>>>>>>>> and in fact we intend to deprecate it in KIP-360 and
> > > > > eventually
> > > > > > >>>>>>>>> remove
> > > > > > >>>>>>>>>>>> it; but I'm wondering can we still use specific
> error
> > > > codes.
> > > > > > >>>>>> E.g.
> > > > > > >>>>>>>>> what
> > > > > > >>>>>>>>>>>> about "InvalidProducerEpochException" since for new
> > > > clients,
> > > > > > >>>>>> the
> > > > > > >>>>>>>>>>>> actual reason this would actually be rejected is
> > indeed
> > > > > because
> > > > > > >>>>>>> the
> > > > > > >>>>>>>>>>>> epoch on the coordinator caused the
> > add-partitions-request
> > > > > from
> > > > > > >>>>>>> the
> > > > > > >>>>>>>>>>>> brokers to be rejected anyways?
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> 4. It seems we put the producer request into
> purgatory
> > > > > before
> > > > > > >>>>>> we
> > > > > > >>>>>>>> ever
> > > > > > >>>>>>>>>>>> append the records, while other producer's records
> may
> > > > > still be
> > > > > > >>>>>>>>>>>> appended during the time; and that potentially may
> > result
> > > > in
> > > > > > >>>>>> some
> > > > > > >>>>>>>>>>>> re-ordering compared with reception order. I'm not
> > super
> > > > > > >>>>>>> concerned
> > > > > > >>>>>>>>>>>> about it since Kafka does not guarantee reception
> > ordering
> > > > > > >>>>>> across
> > > > > > >>>>>>>>>>>> producers anyways, but it may make the timestamps of
> > > > records
> > > > > > >>>>>>>> inside a
> > > > > > >>>>>>>>>>>> partition to be more out-of-ordered. Are we aware of
> > any
> > > > > > >>>>>>> scenarios
> > > > > > >>>>>>>>>>>> such as future enhancements on log compactions that
> > may be
> > > > > > >>>>>>> affected
> > > > > > >>>>>>>>> by
> > > > > > >>>>>>>>>>>> this effect?
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> Below are just minor comments:
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> 5. In "AddPartitionsToTxnTransaction" field of
> > > > > > >>>>>>>>>>>> "AddPartitionsToTxnRequest" RPC, the versions of
> those
> > > > inner
> > > > > > >>>>>>> fields
> > > > > > >>>>>>>>>>>> are "0-3" while I thought they should be "0+" still?
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> 6. Regarding "we can place the request in a
> purgatory
> > of
> > > > > sorts
> > > > > > >>>>>>> and
> > > > > > >>>>>>>>>>>> check if there is any state for the transaction on
> the
> > > > > > >>>>>> broker": i
> > > > > > >>>>>>>>>>>> think at this time when we just do the checks
> against
> > the
> > > > > > >>>>>> cached
> > > > > > >>>>>>>>>>>> state, we do not need to put the request to
> purgatory
> > yet?
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> 7. This is related to 3) above. I feel using
> > > > > > >>>>>>>> "InvalidRecordException"
> > > > > > >>>>>>>>>>>> for older clients may also be a bit confusing, and
> > also it
> > > > > is
> > > > > > >>>>>> not
> > > > > > >>>>>>>>>>>> fatal -- for old clients, it better to be fatal
> since
> > this
> > > > > > >>>>>>>> indicates
> > > > > > >>>>>>>>>>>> the clients is doing something wrong and hence it
> > should
> > > > be
> > > > > > >>>>>>> closed.
> > > > > > >>>>>>>>>>>> And in general I'd prefer to use slightly more
> > specific
> > > > > meaning
> > > > > > >>>>>>>> error
> > > > > > >>>>>>>>>>>> codes for clients. That being said, I also feel
> > > > > > >>>>>>>>>>>> "InvalidProducerEpochException" is not suitable for
> > old
> > > > > > >>>>>> versioned
> > > > > > >>>>>>>>>>>> clients, and we'd have to pick one that old clients
> > > > > recognize.
> > > > > > >>>>>>> I'd
> > > > > > >>>>>>>>>>>> prefer "InvalidTxnStateException" but that one is
> > supposed
> > > > > to
> > > > > > >>>>>> be
> > > > > > >>>>>>>>>>>> returned from txn coordinators only today. I'd
> > suggest we
> > > > > do a
> > > > > > >>>>>>>> quick
> > > > > > >>>>>>>>>>>> check in the current client's code path and see if
> > that
> > > > one
> > > > > > >>>>>> would
> > > > > > >>>>>>>> be
> > > > > > >>>>>>>>>>>> handled if it's from a produce-response, and if yes,
> > use
> > > > > this
> > > > > > >>>>>>> one;
> > > > > > >>>>>>>>>>>> otherwise, use "ProducerFencedException" which is
> much
> > > > less
> > > > > > >>>>>>>>> meaningful
> > > > > > >>>>>>>>>>>> but it's still a fatal error.
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> Thanks,
> > > > > > >>>>>>>>>>>> Guozhang
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
> > > > > > >>>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>> Yeah -- looks like we already have code to handle
> > bumping
> > > > > the
> > > > > > >>>>>>>> epoch
> > > > > > >>>>>>>>>> and
> > > > > > >>>>>>>>>>>>> when the epoch is Short.MAX_VALUE, we get a new
> > producer
> > > > > ID.
> > > > > > >>>>>>>> Since
> > > > > > >>>>>>>>>> this
> > > > > > >>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>> already the behavior, do we want to change it
> > further?
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>> Justine
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>> On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <
> > > > > > >>>>>>>>> jolshan@confluent.io
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>> Hey all, just wanted to quickly update and say
> I've
> > > > > > >>>>>> modified
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>> KIP to
> > > > > > >>>>>>>>>>>>>> explicitly mention that
> AddOffsetCommitsToTxnRequest
> > > > will
> > > > > > >>>>>> be
> > > > > > >>>>>>>>>> replaced
> > > > > > >>>>>>>>>>>> by
> > > > > > >>>>>>>>>>>>>> a coordinator-side (inter-broker)
> AddPartitionsToTxn
> > > > > > >>>>>> implicit
> > > > > > >>>>>>>>>> request.
> > > > > > >>>>>>>>>>>> This
> > > > > > >>>>>>>>>>>>>> mirrors the user partitions and will implicitly
> add
> > > > offset
> > > > > > >>>>>>>>>> partitions
> > > > > > >>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>> transactions when we commit offsets on them. We
> will
> > > > > > >>>>>>> deprecate
> > > > > > >>>>>>>>>>>> AddOffsetCommitsToTxnRequest
> > > > > > >>>>>>>>>>>>>> for new clients.
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>> Also to address Artem's comments --
> > > > > > >>>>>>>>>>>>>> I'm a bit unsure if the changes here will change
> the
> > > > > > >>>>>> previous
> > > > > > >>>>>>>>>> behavior
> > > > > > >>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>> fencing producers. In the case you mention in the
> > first
> > > > > > >>>>>>>>> paragraph,
> > > > > > >>>>>>>>>> are
> > > > > > >>>>>>>>>>>> you
> > > > > > >>>>>>>>>>>>>> saying we bump the epoch before we try to abort
> the
> > > > > > >>>>>>>> transaction?
> > > > > > >>>>>>>>> I
> > > > > > >>>>>>>>>>>> think I
> > > > > > >>>>>>>>>>>>>> need to understand the scenarios you mention a bit
> > > > better.
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>> As for the second part -- I think it makes sense
> to
> > have
> > > > > > >>>>>> some
> > > > > > >>>>>>>>> sort
> > > > > > >>>>>>>>>> of
> > > > > > >>>>>>>>>>>>>> "sentinel" epoch to signal epoch is about to
> > overflow (I
> > > > > > >>>>>>> think
> > > > > > >>>>>>>> we
> > > > > > >>>>>>>>>> sort
> > > > > > >>>>>>>>>>>> of
> > > > > > >>>>>>>>>>>>>> have this value in place in some ways) so we can
> > codify
> > > > it
> > > > > > >>>>>> in
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>> KIP.
> > > > > > >>>>>>>>>>>> I'll
> > > > > > >>>>>>>>>>>>>> look into that and try to update soon.
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>> Thanks,
> > > > > > >>>>>>>>>>>>>> Justine.
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>> On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
> > > > > > >>>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> It's good to know that KIP-588 addressed some of
> > the
> > > > > > >>>>>> issues.
> > > > > > >>>>>>>>>> Looking
> > > > > > >>>>>>>>>>>> at
> > > > > > >>>>>>>>>>>>>>> the code, it still looks like there are some
> cases
> > that
> > > > > > >>>>>>> would
> > > > > > >>>>>>>>>> result
> > > > > > >>>>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>>> fatal error, e.g. PRODUCER_FENCED is issued by
> the
> > > > > > >>>>>>> transaction
> > > > > > >>>>>>>>>>>> coordinator
> > > > > > >>>>>>>>>>>>>>> if epoch doesn't match, and the client treats it
> > as a
> > > > > > >>>>>> fatal
> > > > > > >>>>>>>>> error
> > > > > > >>>>>>>>>>>> (code in
> > > > > > >>>>>>>>>>>>>>> TransactionManager request handling).  If we
> > consider,
> > > > > for
> > > > > > >>>>>>>>>> example,
> > > > > > >>>>>>>>>>>>>>> committing a transaction that returns a timeout,
> > but
> > > > > > >>>>>>> actually
> > > > > > >>>>>>>>>>>> succeeds,
> > > > > > >>>>>>>>>>>>>>> trying to abort it or re-commit may result in
> > > > > > >>>>>>> PRODUCER_FENCED
> > > > > > >>>>>>>>>> error
> > > > > > >>>>>>>>>>>>>>> (because of epoch bump).
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> For failed commits, specifically, we need to know
> > the
> > > > > > >>>>>> actual
> > > > > > >>>>>>>>>> outcome,
> > > > > > >>>>>>>>>>>>>>> because if we return an error the application may
> > think
> > > > > > >>>>>> that
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>>>>>>> transaction is aborted and redo the work, leading
> > to
> > > > > > >>>>>>>> duplicates.
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> Re: overflowing epoch.  We could either do it on
> > the TC
> > > > > > >>>>>> and
> > > > > > >>>>>>>>> return
> > > > > > >>>>>>>>>>>> both
> > > > > > >>>>>>>>>>>>>>> producer id and epoch (e.g. change the protocol),
> > or
> > > > > > >>>>>> signal
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>> client
> > > > > > >>>>>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>> it needs to get a new producer id.  Checking for
> > max
> > > > > epoch
> > > > > > >>>>>>>> could
> > > > > > >>>>>>>>>> be a
> > > > > > >>>>>>>>>>>>>>> reasonable signal, the value to check should
> > probably
> > > > be
> > > > > > >>>>>>>> present
> > > > > > >>>>>>>>>> in
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>> KIP
> > > > > > >>>>>>>>>>>>>>> as this is effectively a part of the contract.
> > Also,
> > > > the
> > > > > > >>>>>> TC
> > > > > > >>>>>>>>>> should
> > > > > > >>>>>>>>>>>>>>> probably return an error if the client didn't
> > change
> > > > > > >>>>>>> producer
> > > > > > >>>>>>>> id
> > > > > > >>>>>>>>>> after
> > > > > > >>>>>>>>>>>>>>> hitting max epoch.
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> -Artem
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
> > > > > > >>>>>>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> Thanks for the discussion Artem.
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> With respect to the handling of fenced
> producers,
> > we
> > > > > > >>>>>> have
> > > > > > >>>>>>>> some
> > > > > > >>>>>>>>>>>> behavior
> > > > > > >>>>>>>>>>>>>>>> already in place. As of KIP-588:
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>
> > > > > > >>>>
> > > > > > >>
> > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> > > > > > >>>>>>>>>>>>>>>> ,
> > > > > > >>>>>>>>>>>>>>>> we handle timeouts more gracefully. The producer
> > can
> > > > > > >>>>>>>> recover.
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> Produce requests can also recover from epoch
> > fencing
> > > > by
> > > > > > >>>>>>>>>> aborting the
> > > > > > >>>>>>>>>>>>>>>> transaction and starting over.
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> What other cases were you considering that would
> > cause
> > > > > > >>>>>> us
> > > > > > >>>>>>> to
> > > > > > >>>>>>>>>> have a
> > > > > > >>>>>>>>>>>>>>> fenced
> > > > > > >>>>>>>>>>>>>>>> epoch but we'd want to recover?
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> The first point about handling epoch overflows
> is
> > > > fair.
> > > > > > >>>>>> I
> > > > > > >>>>>>>>> think
> > > > > > >>>>>>>>>>>> there is
> > > > > > >>>>>>>>>>>>>>>> some logic we'd need to consider. (ie, if we are
> > one
> > > > > > >>>>>> away
> > > > > > >>>>>>>> from
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>> max
> > > > > > >>>>>>>>>>>>>>>> epoch, we need to reset the producer ID.) I'm
> > still
> > > > > > >>>>>>>> wondering
> > > > > > >>>>>>>>> if
> > > > > > >>>>>>>>>>>> there
> > > > > > >>>>>>>>>>>>>>> is a
> > > > > > >>>>>>>>>>>>>>>> way to direct this from the response, or if
> > everything
> > > > > > >>>>>>>> should
> > > > > > >>>>>>>>> be
> > > > > > >>>>>>>>>>>> done on
> > > > > > >>>>>>>>>>>>>>>> the client side. Let me know if you have any
> > thoughts
> > > > > > >>>>>>> here.
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> Thanks,
> > > > > > >>>>>>>>>>>>>>>> Justine
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
> > > > > > >>>>>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> There are some workflows in the client that are
> > > > > > >>>>>> implied
> > > > > > >>>>>>> by
> > > > > > >>>>>>>>>>>> protocol
> > > > > > >>>>>>>>>>>>>>>>> changes, e.g.:
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> - for new clients, epoch changes with every
> > > > > > >>>>>> transaction
> > > > > > >>>>>>>> and
> > > > > > >>>>>>>>>> can
> > > > > > >>>>>>>>>>>>>>> overflow,
> > > > > > >>>>>>>>>>>>>>>>> in old clients this condition was handled
> > > > > > >>>>>> transparently,
> > > > > > >>>>>>>>>> because
> > > > > > >>>>>>>>>>>> epoch
> > > > > > >>>>>>>>>>>>>>>> was
> > > > > > >>>>>>>>>>>>>>>>> bumped in InitProducerId and it would return a
> > new
> > > > > > >>>>>>>> producer
> > > > > > >>>>>>>>>> id if
> > > > > > >>>>>>>>>>>>>>> epoch
> > > > > > >>>>>>>>>>>>>>>>> overflows, the new clients would need to
> > implement
> > > > > > >>>>>> some
> > > > > > >>>>>>>>>> workflow
> > > > > > >>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>> refresh
> > > > > > >>>>>>>>>>>>>>>>> producer id
> > > > > > >>>>>>>>>>>>>>>>> - how to handle fenced producers, for new
> clients
> > > > > > >>>>>> epoch
> > > > > > >>>>>>>>>> changes
> > > > > > >>>>>>>>>>>> with
> > > > > > >>>>>>>>>>>>>>>> every
> > > > > > >>>>>>>>>>>>>>>>> transaction, so in presence of failures during
> > > > > > >>>>>> commits /
> > > > > > >>>>>>>>>> aborts,
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>> producer could get easily fenced, old clients
> > would
> > > > > > >>>>>>> pretty
> > > > > > >>>>>>>>>> much
> > > > > > >>>>>>>>>>>> would
> > > > > > >>>>>>>>>>>>>>> get
> > > > > > >>>>>>>>>>>>>>>>> fenced when a new incarnation of the producer
> was
> > > > > > >>>>>>>>> initialized
> > > > > > >>>>>>>>>> with
> > > > > > >>>>>>>>>>>>>>>>> InitProducerId so it's ok to treat as a fatal
> > error,
> > > > > > >>>>>> the
> > > > > > >>>>>>>> new
> > > > > > >>>>>>>>>>>> clients
> > > > > > >>>>>>>>>>>>>>>> would
> > > > > > >>>>>>>>>>>>>>>>> need to implement some workflow to handle that
> > error,
> > > > > > >>>>>>>>>> otherwise
> > > > > > >>>>>>>>>>>> they
> > > > > > >>>>>>>>>>>>>>>> could
> > > > > > >>>>>>>>>>>>>>>>> get fenced by themselves
> > > > > > >>>>>>>>>>>>>>>>> - in particular (as a subset of the previous
> > issue),
> > > > > > >>>>>>> what
> > > > > > >>>>>>>>>> would
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>> client
> > > > > > >>>>>>>>>>>>>>>>> do if it got a timeout during commit?  commit
> > > > could've
> > > > > > >>>>>>>>>> succeeded
> > > > > > >>>>>>>>>>>> or
> > > > > > >>>>>>>>>>>>>>>> failed
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> Not sure if this has to be defined in the KIP
> as
> > > > > > >>>>>>>>> implementing
> > > > > > >>>>>>>>>>>> those
> > > > > > >>>>>>>>>>>>>>>>> probably wouldn't require protocol changes, but
> > we
> > > > > > >>>>>> have
> > > > > > >>>>>>>>>> multiple
> > > > > > >>>>>>>>>>>>>>>>> implementations of Kafka clients, so probably
> > would
> > > > be
> > > > > > >>>>>>>> good
> > > > > > >>>>>>>>> to
> > > > > > >>>>>>>>>>>> have
> > > > > > >>>>>>>>>>>>>>> some
> > > > > > >>>>>>>>>>>>>>>>> client implementation guidance.  Could also be
> > done
> > > > > > >>>>>> as a
> > > > > > >>>>>>>>>> separate
> > > > > > >>>>>>>>>>>> doc.
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> -Artem
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
> > > > > > >>>>>>>>>>>>>>>> <jolshan@confluent.io.invalid
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> Hey all, I've updated the KIP to incorporate
> > Jason's
> > > > > > >>>>>>>>>>>> suggestions.
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>
> > > > > > >>>>
> > > > > > >>
> > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> 1. Use AddPartitionsToTxn + verify flag to
> > check on
> > > > > > >>>>>>> old
> > > > > > >>>>>>>>>> clients
> > > > > > >>>>>>>>>>>>>>>>>> 2. Updated AddPartitionsToTxn API to support
> > > > > > >>>>>>> transaction
> > > > > > >>>>>>>>>>>> batching
> > > > > > >>>>>>>>>>>>>>>>>> 3. Mention IBP bump
> > > > > > >>>>>>>>>>>>>>>>>> 4. Mention auth change on new
> AddPartitionsToTxn
> > > > > > >>>>>>>> version.
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> I'm planning on opening a vote soon.
> > > > > > >>>>>>>>>>>>>>>>>> Thanks,
> > > > > > >>>>>>>>>>>>>>>>>> Justine
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan
> <
> > > > > > >>>>>>>>>>>> jolshan@confluent.io
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>> Thanks Jason. Those changes make sense to
> me. I
> > > > > > >>>>>> will
> > > > > > >>>>>>>>>> update
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>> KIP.
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:31 PM Jason
> Gustafson
> > > > > > >>>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
> > > > > > >>>>>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>> Hey Justine,
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>> I was wondering about compatibility here.
> > When
> > > > > > >>>>>> we
> > > > > > >>>>>>>>> send
> > > > > > >>>>>>>>>>>> requests
> > > > > > >>>>>>>>>>>>>>>>>>>> between brokers, we want to ensure that the
> > > > > > >>>>>>> receiving
> > > > > > >>>>>>>>>> broker
> > > > > > >>>>>>>>>>>>>>>>> understands
> > > > > > >>>>>>>>>>>>>>>>>>>> the request (specifically the new fields).
> > > > > > >>>>>>> Typically
> > > > > > >>>>>>>>>> this is
> > > > > > >>>>>>>>>>>> done
> > > > > > >>>>>>>>>>>>>>>> via
> > > > > > >>>>>>>>>>>>>>>>>>>> IBP/metadata version.
> > > > > > >>>>>>>>>>>>>>>>>>>> I'm trying to think if there is a way around
> > it
> > > > > > >>>>>> but
> > > > > > >>>>>>>> I'm
> > > > > > >>>>>>>>>> not
> > > > > > >>>>>>>>>>>> sure
> > > > > > >>>>>>>>>>>>>>>> there
> > > > > > >>>>>>>>>>>>>>>>>> is.
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>> Yes. I think we would gate usage of this
> > behind
> > > > > > >>>>>> an
> > > > > > >>>>>>>> IBP
> > > > > > >>>>>>>>>> bump.
> > > > > > >>>>>>>>>>>> Does
> > > > > > >>>>>>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>>>>>>> seem
> > > > > > >>>>>>>>>>>>>>>>>>>> reasonable?
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>> As for the improvements -- can you clarify
> > how
> > > > > > >>>>>>> the
> > > > > > >>>>>>>>>> multiple
> > > > > > >>>>>>>>>>>>>>>>>>>> transactional
> > > > > > >>>>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking of a
> > case
> > > > > > >>>>>>>> where
> > > > > > >>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>> wait/batch
> > > > > > >>>>>>>>>>>>>>>>>>>> multiple produce requests together? My
> > > > > > >>>>>>> understanding
> > > > > > >>>>>>>>> for
> > > > > > >>>>>>>>>> now
> > > > > > >>>>>>>>>>>> was
> > > > > > >>>>>>>>>>>>>>> 1
> > > > > > >>>>>>>>>>>>>>>>>>>> transactional ID and one validation per 1
> > produce
> > > > > > >>>>>>>>>> request.
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>> Each call to `AddPartitionsToTxn` is
> > essentially
> > > > > > >>>>>> a
> > > > > > >>>>>>>>> write
> > > > > > >>>>>>>>>> to
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>>>>>>> log and must block on replication. The more
> we
> > > > > > >>>>>> can
> > > > > > >>>>>>>> fit
> > > > > > >>>>>>>>>> into a
> > > > > > >>>>>>>>>>>>>>> single
> > > > > > >>>>>>>>>>>>>>>>>>>> request, the more writes we can do in
> > parallel.
> > > > > > >>>>>> The
> > > > > > >>>>>>>>>>>> alternative
> > > > > > >>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>> make
> > > > > > >>>>>>>>>>>>>>>>>>>> use of more connections, but usually we
> prefer
> > > > > > >>>>>>>> batching
> > > > > > >>>>>>>>>>>> since the
> > > > > > >>>>>>>>>>>>>>>>>> network
> > > > > > >>>>>>>>>>>>>>>>>>>> stack is not really optimized for high
> > > > > > >>>>>>>>> connection/request
> > > > > > >>>>>>>>>>>> loads.
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>> Finally with respect to the
> authorizations, I
> > > > > > >>>>>>> think
> > > > > > >>>>>>>>> it
> > > > > > >>>>>>>>>>>> makes
> > > > > > >>>>>>>>>>>>>>> sense
> > > > > > >>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>> skip
> > > > > > >>>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit confused
> > by
> > > > > > >>>>>> the
> > > > > > >>>>>>>>>> "leader
> > > > > > >>>>>>>>>>>> ID"
> > > > > > >>>>>>>>>>>>>>>> field.
> > > > > > >>>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the request as
> > > > > > >>>>>> from a
> > > > > > >>>>>>>>>> broker
> > > > > > >>>>>>>>>>>> (does
> > > > > > >>>>>>>>>>>>>>> it
> > > > > > >>>>>>>>>>>>>>>>>> matter
> > > > > > >>>>>>>>>>>>>>>>>>>> which one?).
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>> We could also make it version-based. For the
> > next
> > > > > > >>>>>>>>>> version, we
> > > > > > >>>>>>>>>>>>>>> could
> > > > > > >>>>>>>>>>>>>>>>>>>> require
> > > > > > >>>>>>>>>>>>>>>>>>>> CLUSTER auth. So clients would not be able
> to
> > use
> > > > > > >>>>>>> the
> > > > > > >>>>>>>>> API
> > > > > > >>>>>>>>>>>>>>> anymore,
> > > > > > >>>>>>>>>>>>>>>>> which
> > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>> probably what we want.
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>> -Jason
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:43 AM Justine
> Olshan
> > > > > > >>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>> As a follow up, I was just thinking about
> the
> > > > > > >>>>>>>>> batching
> > > > > > >>>>>>>>>> a
> > > > > > >>>>>>>>>>>> bit
> > > > > > >>>>>>>>>>>>>>> more.
> > > > > > >>>>>>>>>>>>>>>>>>>>> I suppose if we have one request in flight
> > and
> > > > > > >>>>>> we
> > > > > > >>>>>>>>>> queue up
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>> other
> > > > > > >>>>>>>>>>>>>>>>>>>>> produce requests in some sort of purgatory,
> > we
> > > > > > >>>>>>>> could
> > > > > > >>>>>>>>>> send
> > > > > > >>>>>>>>>>>>>>>>> information
> > > > > > >>>>>>>>>>>>>>>>>>>> out
> > > > > > >>>>>>>>>>>>>>>>>>>>> for all of them rather than one by one. So
> > that
> > > > > > >>>>>>>> would
> > > > > > >>>>>>>>>> be a
> > > > > > >>>>>>>>>>>>>>> benefit
> > > > > > >>>>>>>>>>>>>>>>> of
> > > > > > >>>>>>>>>>>>>>>>>>>>> batching partitions to add per transaction.
> > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>> I'll need to think a bit more on the design
> > of
> > > > > > >>>>>>> this
> > > > > > >>>>>>>>>> part
> > > > > > >>>>>>>>>>>> of the
> > > > > > >>>>>>>>>>>>>>>> KIP,
> > > > > > >>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>> will update the KIP in the next few days.
> > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > >>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:22 AM Justine
> > Olshan
> > > > > > >>>>>> <
> > > > > > >>>>>>>>>>>>>>>>> jolshan@confluent.io>
> > > > > > >>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>> Hey Jason -- thanks for the input -- I was
> > > > > > >>>>>> just
> > > > > > >>>>>>>>>> digging
> > > > > > >>>>>>>>>>>> a bit
> > > > > > >>>>>>>>>>>>>>>>> deeper
> > > > > > >>>>>>>>>>>>>>>>>>>> into
> > > > > > >>>>>>>>>>>>>>>>>>>>>> the design + implementation of the
> > validation
> > > > > > >>>>>>>> calls
> > > > > > >>>>>>>>>> here
> > > > > > >>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>> what
> > > > > > >>>>>>>>>>>>>>>>>> you
> > > > > > >>>>>>>>>>>>>>>>>>>> say
> > > > > > >>>>>>>>>>>>>>>>>>>>>> makes sense.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>> I was wondering about compatibility here.
> > > > > > >>>>>> When
> > > > > > >>>>>>> we
> > > > > > >>>>>>>>>> send
> > > > > > >>>>>>>>>>>>>>> requests
> > > > > > >>>>>>>>>>>>>>>>>>>>>> between brokers, we want to ensure that
> the
> > > > > > >>>>>>>>> receiving
> > > > > > >>>>>>>>>>>> broker
> > > > > > >>>>>>>>>>>>>>>>>>>> understands
> > > > > > >>>>>>>>>>>>>>>>>>>>>> the request (specifically the new fields).
> > > > > > >>>>>>>>> Typically
> > > > > > >>>>>>>>>>>> this is
> > > > > > >>>>>>>>>>>>>>>> done
> > > > > > >>>>>>>>>>>>>>>>>> via
> > > > > > >>>>>>>>>>>>>>>>>>>>>> IBP/metadata version.
> > > > > > >>>>>>>>>>>>>>>>>>>>>> I'm trying to think if there is a way
> around
> > > > > > >>>>>> it
> > > > > > >>>>>>>> but
> > > > > > >>>>>>>>>> I'm
> > > > > > >>>>>>>>>>>> not
> > > > > > >>>>>>>>>>>>>>> sure
> > > > > > >>>>>>>>>>>>>>>>>> there
> > > > > > >>>>>>>>>>>>>>>>>>>>> is.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>> As for the improvements -- can you clarify
> > > > > > >>>>>> how
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>>>> multiple
> > > > > > >>>>>>>>>>>>>>>>>>>> transactional
> > > > > > >>>>>>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking of
> a
> > > > > > >>>>>>> case
> > > > > > >>>>>>>>>> where we
> > > > > > >>>>>>>>>>>>>>>>> wait/batch
> > > > > > >>>>>>>>>>>>>>>>>>>>>> multiple produce requests together? My
> > > > > > >>>>>>>>> understanding
> > > > > > >>>>>>>>>> for
> > > > > > >>>>>>>>>>>> now
> > > > > > >>>>>>>>>>>>>>>> was 1
> > > > > > >>>>>>>>>>>>>>>>>>>>>> transactional ID and one validation per 1
> > > > > > >>>>>>> produce
> > > > > > >>>>>>>>>>>> request.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>> Finally with respect to the
> authorizations,
> > I
> > > > > > >>>>>>>> think
> > > > > > >>>>>>>>>> it
> > > > > > >>>>>>>>>>>> makes
> > > > > > >>>>>>>>>>>>>>>> sense
> > > > > > >>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>> skip
> > > > > > >>>>>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit
> confused
> > > > > > >>>>>> by
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>>>> "leader
> > > > > > >>>>>>>>>>>>>>> ID"
> > > > > > >>>>>>>>>>>>>>>>>> field.
> > > > > > >>>>>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the request
> as
> > > > > > >>>>>>>> from a
> > > > > > >>>>>>>>>>>> broker
> > > > > > >>>>>>>>>>>>>>> (does
> > > > > > >>>>>>>>>>>>>>>>> it
> > > > > > >>>>>>>>>>>>>>>>>>>>> matter
> > > > > > >>>>>>>>>>>>>>>>>>>>>> which one?).
> > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>> I think I want to adopt these suggestions,
> > > > > > >>>>>> just
> > > > > > >>>>>>>> had
> > > > > > >>>>>>>>>> a few
> > > > > > >>>>>>>>>>>>>>>>> questions
> > > > > > >>>>>>>>>>>>>>>>>> on
> > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>> details.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > >>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>> On Thu, Jan 5, 2023 at 5:05 PM Jason
> > > > > > >>>>>> Gustafson
> > > > > > >>>>>>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
> > > > > > >>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> Thanks for the proposal.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> I was thinking about the implementation a
> > > > > > >>>>>>> little
> > > > > > >>>>>>>>>> bit.
> > > > > > >>>>>>>>>>>> In the
> > > > > > >>>>>>>>>>>>>>>>>> current
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> proposal, the behavior depends on whether
> > we
> > > > > > >>>>>>>> have
> > > > > > >>>>>>>>> an
> > > > > > >>>>>>>>>>>> old or
> > > > > > >>>>>>>>>>>>>>> new
> > > > > > >>>>>>>>>>>>>>>>>>>> client.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> For
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> old clients, we send
> `DescribeTransactions`
> > > > > > >>>>>>> and
> > > > > > >>>>>>>>>> verify
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>> result
> > > > > > >>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> new clients, we send
> `AddPartitionsToTxn`.
> > > > > > >>>>>> We
> > > > > > >>>>>>>>> might
> > > > > > >>>>>>>>>> be
> > > > > > >>>>>>>>>>>> able
> > > > > > >>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>> simplify
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> implementation if we can use the same
> > > > > > >>>>>> request
> > > > > > >>>>>>>>> type.
> > > > > > >>>>>>>>>> For
> > > > > > >>>>>>>>>>>>>>>> example,
> > > > > > >>>>>>>>>>>>>>>>>>>> what if
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> bump the protocol version for
> > > > > > >>>>>>>> `AddPartitionsToTxn`
> > > > > > >>>>>>>>>> and
> > > > > > >>>>>>>>>>>> add a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> `validateOnly`
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> flag? For older versions, we can set
> > > > > > >>>>>>>>>>>> `validateOnly=true` so
> > > > > > >>>>>>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> request only returns successfully if the
> > > > > > >>>>>>>> partition
> > > > > > >>>>>>>>>> had
> > > > > > >>>>>>>>>>>>>>> already
> > > > > > >>>>>>>>>>>>>>>>> been
> > > > > > >>>>>>>>>>>>>>>>>>>>> added.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> For new versions, we can set
> > > > > > >>>>>>>> `validateOnly=false`
> > > > > > >>>>>>>>>> and
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>> partition
> > > > > > >>>>>>>>>>>>>>>>>>>> will
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> added to the transaction. The other
> > slightly
> > > > > > >>>>>>>>>> annoying
> > > > > > >>>>>>>>>>>> thing
> > > > > > >>>>>>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> get around is the need to collect the
> > > > > > >>>>>>>> transaction
> > > > > > >>>>>>>>>> state
> > > > > > >>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>> all
> > > > > > >>>>>>>>>>>>>>>>>>>>> partitions
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> even when we only care about a subset.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> Some additional improvements to consider:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> - We can give `AddPartitionsToTxn` better
> > > > > > >>>>>>> batch
> > > > > > >>>>>>>>>> support
> > > > > > >>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>>>> inter-broker
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> usage. Currently we only allow one
> > > > > > >>>>>>>>>> `TransactionalId` to
> > > > > > >>>>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>>>>> specified,
> > > > > > >>>>>>>>>>>>>>>>>>>>> but
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> the broker may get some benefit being
> able
> > > > > > >>>>>> to
> > > > > > >>>>>>>>> batch
> > > > > > >>>>>>>>>>>> across
> > > > > > >>>>>>>>>>>>>>>>> multiple
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> transactions.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> - Another small improvement is skipping
> > > > > > >>>>>> topic
> > > > > > >>>>>>>>>>>> authorization
> > > > > > >>>>>>>>>>>>>>>>> checks
> > > > > > >>>>>>>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> `AddPartitionsToTxn` when the request is
> > > > > > >>>>>> from
> > > > > > >>>>>>> a
> > > > > > >>>>>>>>>> broker.
> > > > > > >>>>>>>>>>>>>>> Perhaps
> > > > > > >>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>> can
> > > > > > >>>>>>>>>>>>>>>>>>>>> add
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> a field for the `LeaderId` or something
> > like
> > > > > > >>>>>>>> that
> > > > > > >>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>> require
> > > > > > >>>>>>>>>>>>>>>>>> CLUSTER
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> permission when set.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> Best,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> Jason
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
> > > > > > >>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation. It makes
> sense
> > > > > > >>>>>>> to
> > > > > > >>>>>>>> me
> > > > > > >>>>>>>>>> now.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 1:42 PM Justine
> > > > > > >>>>>>> Olshan
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> My understanding of the mechanism is
> > > > > > >>>>>> that
> > > > > > >>>>>>>> when
> > > > > > >>>>>>>>>> we
> > > > > > >>>>>>>>>>>> get to
> > > > > > >>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>> last
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> epoch,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> increment to the fencing/last epoch and
> > > > > > >>>>>> if
> > > > > > >>>>>>>> any
> > > > > > >>>>>>>>>>>> further
> > > > > > >>>>>>>>>>>>>>>>> requests
> > > > > > >>>>>>>>>>>>>>>>>>>> come
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> this producer ID they are fenced. Then
> > > > > > >>>>>> the
> > > > > > >>>>>>>>>> producer
> > > > > > >>>>>>>>>>>>>>> gets a
> > > > > > >>>>>>>>>>>>>>>>> new
> > > > > > >>>>>>>>>>>>>>>>>> ID
> > > > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> restarts with epoch/sequence 0. The
> > > > > > >>>>>> fenced
> > > > > > >>>>>>>>> epoch
> > > > > > >>>>>>>>>>>> sticks
> > > > > > >>>>>>>>>>>>>>>>> around
> > > > > > >>>>>>>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> duration of producer.id.expiration.ms
> > > > > > >>>>>> and
> > > > > > >>>>>>>>>> blocks
> > > > > > >>>>>>>>>>>> any
> > > > > > >>>>>>>>>>>>>>> late
> > > > > > >>>>>>>>>>>>>>>>>>>> messages
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> there.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> The new ID will get to take advantage
> of
> > > > > > >>>>>>> the
> > > > > > >>>>>>>>>>>> improved
> > > > > > >>>>>>>>>>>>>>>>> semantics
> > > > > > >>>>>>>>>>>>>>>>>>>>> around
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> non-zero start sequences. So I think we
> > > > > > >>>>>>> are
> > > > > > >>>>>>>>>> covered.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> The only potential issue is overloading
> > > > > > >>>>>>> the
> > > > > > >>>>>>>>>> cache,
> > > > > > >>>>>>>>>>>> but
> > > > > > >>>>>>>>>>>>>>>>>> hopefully
> > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> improvements (lowered
> > > > > > >>>>>>>>> producer.id.expiration.ms
> > > > > > >>>>>>>>>> )
> > > > > > >>>>>>>>>>>> will
> > > > > > >>>>>>>>>>>>>>> help
> > > > > > >>>>>>>>>>>>>>>>>> with
> > > > > > >>>>>>>>>>>>>>>>>>>>> that.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Let
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> me know if you still have concerns.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 10:24 AM Jun
> Rao
> > > > > > >>>>>>>>>>>>>>>>>>>> <ju...@confluent.io.invalid>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> 70. The proposed fencing logic doesn't
> > > > > > >>>>>>>> apply
> > > > > > >>>>>>>>>> when
> > > > > > >>>>>>>>>>>> pid
> > > > > > >>>>>>>>>>>>>>>>>> changes,
> > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> right? If so, I am not sure how
> > > > > > >>>>>> complete
> > > > > > >>>>>>>> we
> > > > > > >>>>>>>>>> are
> > > > > > >>>>>>>>>>>>>>>> addressing
> > > > > > >>>>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> issue
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the pid changes more frequently.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 16, 2022 at 9:16 AM
> > > > > > >>>>>> Justine
> > > > > > >>>>>>>>> Olshan
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for replying!
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> 70.We already do the overflow
> > > > > > >>>>>>> mechanism,
> > > > > > >>>>>>>>> so
> > > > > > >>>>>>>>>> my
> > > > > > >>>>>>>>>>>>>>> change
> > > > > > >>>>>>>>>>>>>>>>> would
> > > > > > >>>>>>>>>>>>>>>>>>>> just
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> make
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> happen more often.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I was also not suggesting a new
> > > > > > >>>>>> field
> > > > > > >>>>>>> in
> > > > > > >>>>>>>>> the
> > > > > > >>>>>>>>>>>> log,
> > > > > > >>>>>>>>>>>>>>> but
> > > > > > >>>>>>>>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> response,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> which would be gated by the client
> > > > > > >>>>>>>>> version.
> > > > > > >>>>>>>>>>>> Sorry if
> > > > > > >>>>>>>>>>>>>>>>>>>> something
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> there
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> unclear. I think we are starting to
> > > > > > >>>>>>>>> diverge.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> The goal of this KIP is to not
> > > > > > >>>>>> change
> > > > > > >>>>>>> to
> > > > > > >>>>>>>>> the
> > > > > > >>>>>>>>>>>> marker
> > > > > > >>>>>>>>>>>>>>>>> format
> > > > > > >>>>>>>>>>>>>>>>>> at
> > > > > > >>>>>>>>>>>>>>>>>>>>> all.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> 71. Yes, I guess I was going under
> > > > > > >>>>>> the
> > > > > > >>>>>>>>>>>> assumption
> > > > > > >>>>>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>> log
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> just
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> look at its last epoch and treat it
> > > > > > >>>>>> as
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>>>> current
> > > > > > >>>>>>>>>>>>>>>>> epoch. I
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> suppose
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> have some special logic that if the
> > > > > > >>>>>>> last
> > > > > > >>>>>>>>>> epoch
> > > > > > >>>>>>>>>>>> was
> > > > > > >>>>>>>>>>>>>>> on a
> > > > > > >>>>>>>>>>>>>>>>>>>> marker
> > > > > > >>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> actually
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> expect the next epoch or something
> > > > > > >>>>>>> like
> > > > > > >>>>>>>>>> that. We
> > > > > > >>>>>>>>>>>>>>> just
> > > > > > >>>>>>>>>>>>>>>>> need
> > > > > > >>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> distinguish
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> based on whether we had a
> > > > > > >>>>>> commit/abort
> > > > > > >>>>>>>>>> marker.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> 72.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> if the producer epoch hasn't been
> > > > > > >>>>>>>> bumped
> > > > > > >>>>>>>>>> on
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
> > > > > > >>>>>>>> message
> > > > > > >>>>>>>>>> will
> > > > > > >>>>>>>>>>>> fail
> > > > > > >>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>> sequence
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> validation
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the producer
> > > > > > >>>>>>>> epoch
> > > > > > >>>>>>>>>> has
> > > > > > >>>>>>>>>>>> been
> > > > > > >>>>>>>>>>>>>>>>> bumped,
> > > > > > >>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> ignore
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck message
> > > > > > >>>>>>>> could
> > > > > > >>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>> appended
> > > > > > >>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>> log.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> So,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
> > > > > > >>>>>> guard?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I follow that "the
> > > > > > >>>>>>> message
> > > > > > >>>>>>>>> will
> > > > > > >>>>>>>>>>>> fail
> > > > > > >>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>> sequence
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> validation".
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> In some of these cases, we had an
> > > > > > >>>>>>> abort
> > > > > > >>>>>>>>>> marker
> > > > > > >>>>>>>>>>>> (due
> > > > > > >>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>> an
> > > > > > >>>>>>>>>>>>>>>>>>>> error)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> then
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the late message comes in with the
> > > > > > >>>>>>>> correct
> > > > > > >>>>>>>>>>>> sequence
> > > > > > >>>>>>>>>>>>>>>>> number.
> > > > > > >>>>>>>>>>>>>>>>>>>> This
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> is a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> case
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> covered by the KIP.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> The latter case is actually not
> > > > > > >>>>>>>> something
> > > > > > >>>>>>>>>> we've
> > > > > > >>>>>>>>>>>>>>>>> considered
> > > > > > >>>>>>>>>>>>>>>>>>>>> here. I
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> think
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> generally when we bump the epoch, we
> > > > > > >>>>>>> are
> > > > > > >>>>>>>>>>>> accepting
> > > > > > >>>>>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> sequence
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> does
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> not need to be checked anymore. My
> > > > > > >>>>>>>>>>>> understanding is
> > > > > > >>>>>>>>>>>>>>>> also
> > > > > > >>>>>>>>>>>>>>>>>>>> that we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> typically bump epoch mid transaction
> > > > > > >>>>>>>>> (based
> > > > > > >>>>>>>>>> on a
> > > > > > >>>>>>>>>>>>>>> quick
> > > > > > >>>>>>>>>>>>>>>>> look
> > > > > > >>>>>>>>>>>>>>>>>>>> at
> > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> code)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but let me know if that is the case.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 15, 2022 at 12:23 PM Jun
> > > > > > >>>>>>> Rao
> > > > > > >>>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the reply.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Assigning a new pid on int
> > > > > > >>>>>>>> overflow
> > > > > > >>>>>>>>>> seems
> > > > > > >>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>> bit
> > > > > > >>>>>>>>>>>>>>>>>> hacky.
> > > > > > >>>>>>>>>>>>>>>>>>>> If
> > > > > > >>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> need a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> level id, it will be better to
> > > > > > >>>>>> model
> > > > > > >>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>> explicitly.
> > > > > > >>>>>>>>>>>>>>>>>>>> Adding a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> field
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> would require a bit more work
> > > > > > >>>>>> since
> > > > > > >>>>>>> it
> > > > > > >>>>>>>>>>>> requires a
> > > > > > >>>>>>>>>>>>>>> new
> > > > > > >>>>>>>>>>>>>>>>> txn
> > > > > > >>>>>>>>>>>>>>>>>>>>> marker
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> format
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the log. So, we probably need to
> > > > > > >>>>>>> guard
> > > > > > >>>>>>>>> it
> > > > > > >>>>>>>>>>>> with an
> > > > > > >>>>>>>>>>>>>>> IBP
> > > > > > >>>>>>>>>>>>>>>>> or
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> metadata
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> version
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and document the impact on
> > > > > > >>>>>> downgrade
> > > > > > >>>>>>>>> once
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>> new
> > > > > > >>>>>>>>>>>>>>>>> format
> > > > > > >>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> written
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 71. Hmm, once the marker is
> > > > > > >>>>>> written,
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>>>> partition
> > > > > > >>>>>>>>>>>>>>>> will
> > > > > > >>>>>>>>>>>>>>>>>>>> expect
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> next
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> append to be on the next epoch.
> > > > > > >>>>>> Does
> > > > > > >>>>>>>>> that
> > > > > > >>>>>>>>>>>> cover
> > > > > > >>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>> case
> > > > > > >>>>>>>>>>>>>>>>>>>> you
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> mentioned?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 72. Also, just to be clear on the
> > > > > > >>>>>>>>> stucked
> > > > > > >>>>>>>>>>>> message
> > > > > > >>>>>>>>>>>>>>>> issue
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> described
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> motivation. With EoS, we also
> > > > > > >>>>>>> validate
> > > > > > >>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>> sequence
> > > > > > >>>>>>>>>>>>>>>> id
> > > > > > >>>>>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> idempotency.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> So,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> with the current logic, if the
> > > > > > >>>>>>>> producer
> > > > > > >>>>>>>>>> epoch
> > > > > > >>>>>>>>>>>>>>> hasn't
> > > > > > >>>>>>>>>>>>>>>>> been
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> bumped on
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
> > > > > > >>>>>>>>> message
> > > > > > >>>>>>>>>> will
> > > > > > >>>>>>>>>>>>>>> fail
> > > > > > >>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>> sequence
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> validation
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the
> > > > > > >>>>>> producer
> > > > > > >>>>>>>>>> epoch has
> > > > > > >>>>>>>>>>>>>>> been
> > > > > > >>>>>>>>>>>>>>>>>>>> bumped, we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> ignore
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck
> > > > > > >>>>>> message
> > > > > > >>>>>>>>>> could be
> > > > > > >>>>>>>>>>>>>>>> appended
> > > > > > >>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> log.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> So,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
> > > > > > >>>>>>> guard?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 10:44 AM
> > > > > > >>>>>>>> Justine
> > > > > > >>>>>>>>>>>> Olshan
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > > >>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias — thanks again for
> > > > > > >>>>>> taking
> > > > > > >>>>>>>>> time
> > > > > > >>>>>>>>>> to
> > > > > > >>>>>>>>>>>> look
> > > > > > >>>>>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>>>> this.
> > > > > > >>>>>>>>>>>>>>>>>>>> You
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> said:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My proposal was only focusing
> > > > > > >>>>>> to
> > > > > > >>>>>>>>> avoid
> > > > > > >>>>>>>>>>>>>>> dangling
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
> > > > > > >>>>>> added
> > > > > > >>>>>>>>>> without
> > > > > > >>>>>>>>>>>>>>>> registered
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> partition.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> --
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more details
> > > > > > >>>>>> to
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>> KIP
> > > > > > >>>>>>>>>>>> about
> > > > > > >>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>>> scenario
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> better
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I understand what
> > > > > > >>>>>> you
> > > > > > >>>>>>>>> mean
> > > > > > >>>>>>>>>>>> here.
> > > > > > >>>>>>>>>>>>>>> The
> > > > > > >>>>>>>>>>>>>>>>>>>> motivation
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> section
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> describes two scenarios about
> > > > > > >>>>>> how
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>> record
> > > > > > >>>>>>>>>>>>>>> can be
> > > > > > >>>>>>>>>>>>>>>>>> added
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> without a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> registered partition:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Another way hanging
> > > > > > >>>>>> transactions
> > > > > > >>>>>>>> can
> > > > > > >>>>>>>>>>>> occur is
> > > > > > >>>>>>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>>>>>>>> client
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> buggy
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> may somehow try to write to a
> > > > > > >>>>>>>>> partition
> > > > > > >>>>>>>>>>>> before
> > > > > > >>>>>>>>>>>>>>> it
> > > > > > >>>>>>>>>>>>>>>>> adds
> > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> partition
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> For the first example of this
> > > > > > >>>>>>> would
> > > > > > >>>>>>>> it
> > > > > > >>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>> helpful
> > > > > > >>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>> say
> > > > > > >>>>>>>>>>>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> comes in after the abort, but
> > > > > > >>>>>>> before
> > > > > > >>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>> partition
> > > > > > >>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>> added
> > > > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> next
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction so it becomes
> > > > > > >>>>>>> "hanging."
> > > > > > >>>>>>>>>>>> Perhaps the
> > > > > > >>>>>>>>>>>>>>>> next
> > > > > > >>>>>>>>>>>>>>>>>>>>> sentence
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> describing
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the message becoming part of the
> > > > > > >>>>>>>> next
> > > > > > >>>>>>>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>>> (a
> > > > > > >>>>>>>>>>>>>>>>>>>>> different
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> case)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> was
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not properly differentiated.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun — thanks for reading the
> > > > > > >>>>>> KIP.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. The int typing was a
> > > > > > >>>>>> concern.
> > > > > > >>>>>>>>>> Currently
> > > > > > >>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>> have a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> mechanism
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> place
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> fence the final epoch when the
> > > > > > >>>>>>> epoch
> > > > > > >>>>>>>>> is
> > > > > > >>>>>>>>>>>> about to
> > > > > > >>>>>>>>>>>>>>>>>> overflow
> > > > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> assign
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer ID with epoch 0. Of
> > > > > > >>>>>>> course,
> > > > > > >>>>>>>>>> this
> > > > > > >>>>>>>>>>>> is a
> > > > > > >>>>>>>>>>>>>>> bit
> > > > > > >>>>>>>>>>>>>>>>>> tricky
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> when it
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> comes
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response back to the client.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Making this a long could be
> > > > > > >>>>>>> another
> > > > > > >>>>>>>>>> option,
> > > > > > >>>>>>>>>>>> but
> > > > > > >>>>>>>>>>>>>>> I
> > > > > > >>>>>>>>>>>>>>>>>> wonder
> > > > > > >>>>>>>>>>>>>>>>>>>> are
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> there
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> implications on changing this
> > > > > > >>>>>>> field
> > > > > > >>>>>>>> if
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>> epoch is
> > > > > > >>>>>>>>>>>>>>>>>>>>> persisted
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> disk?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to check the usages.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 71.This was something Matthias
> > > > > > >>>>>>> asked
> > > > > > >>>>>>>>>> about
> > > > > > >>>>>>>>>>>> as
> > > > > > >>>>>>>>>>>>>>>> well. I
> > > > > > >>>>>>>>>>>>>>>>>> was
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> considering a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible edge case where a
> > > > > > >>>>>> produce
> > > > > > >>>>>>>>>> request
> > > > > > >>>>>>>>>>>> from
> > > > > > >>>>>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>>> new
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> somehow
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> gets sent right after the marker
> > > > > > >>>>>>> is
> > > > > > >>>>>>>>>>>> written, but
> > > > > > >>>>>>>>>>>>>>>>> before
> > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> alerted of the newly bumped
> > > > > > >>>>>> epoch.
> > > > > > >>>>>>>> In
> > > > > > >>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>> case, we
> > > > > > >>>>>>>>>>>>>>>>> may
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> include
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> record
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we don't want to. I suppose
> > > > > > >>>>>>> we
> > > > > > >>>>>>>>>> could
> > > > > > >>>>>>>>>>>> try
> > > > > > >>>>>>>>>>>>>>> to do
> > > > > > >>>>>>>>>>>>>>>>>>>>> something
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> client
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> side
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to bump the epoch after sending
> > > > > > >>>>>> an
> > > > > > >>>>>>>>>> endTxn as
> > > > > > >>>>>>>>>>>>>>> well
> > > > > > >>>>>>>>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> scenario
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> —
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> I
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wonder how it would work when
> > > > > > >>>>>> the
> > > > > > >>>>>>>>>> server is
> > > > > > >>>>>>>>>>>>>>>> aborting
> > > > > > >>>>>>>>>>>>>>>>>>>> based
> > > > > > >>>>>>>>>>>>>>>>>>>>> on
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> error. I could also be missing
> > > > > > >>>>>>>>>> something and
> > > > > > >>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>> scenario
> > > > > > >>>>>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> actually
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again to everyone reading
> > > > > > >>>>>>> and
> > > > > > >>>>>>>>>>>> commenting.
> > > > > > >>>>>>>>>>>>>>>> Let
> > > > > > >>>>>>>>>>>>>>>>> me
> > > > > > >>>>>>>>>>>>>>>>>>>> know
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> about
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> further questions or comments.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 9:41 AM
> > > > > > >>>>>>> Jun
> > > > > > >>>>>>>>> Rao
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP. A couple
> > > > > > >>>>>> of
> > > > > > >>>>>>>>>> comments.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Currently, the producer
> > > > > > >>>>>>> epoch
> > > > > > >>>>>>>> is
> > > > > > >>>>>>>>>> an
> > > > > > >>>>>>>>>>>> int.
> > > > > > >>>>>>>>>>>>>>> I am
> > > > > > >>>>>>>>>>>>>>>>> not
> > > > > > >>>>>>>>>>>>>>>>>>>> sure
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> it's
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> enough
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to accommodate all
> > > > > > >>>>>> transactions
> > > > > > >>>>>>> in
> > > > > > >>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>> lifetime
> > > > > > >>>>>>>>>>>>>>>> of
> > > > > > >>>>>>>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> producer.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Should
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> change that to a long or add a
> > > > > > >>>>>>> new
> > > > > > >>>>>>>>>> long
> > > > > > >>>>>>>>>>>> field
> > > > > > >>>>>>>>>>>>>>>> like
> > > > > > >>>>>>>>>>>>>>>>>>>> txnId?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 71. "it will write the prepare
> > > > > > >>>>>>>>> commit
> > > > > > >>>>>>>>>>>> message
> > > > > > >>>>>>>>>>>>>>>> with
> > > > > > >>>>>>>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>>>>>>>> bumped
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> send
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WriteTxnMarkerRequests with
> > > > > > >>>>>> the
> > > > > > >>>>>>>>> bumped
> > > > > > >>>>>>>>>>>> epoch."
> > > > > > >>>>>>>>>>>>>>>> Hmm,
> > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> associated
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the current txn right?
> > > > > > >>>>>> So,
> > > > > > >>>>>>> it
> > > > > > >>>>>>>>>> seems
> > > > > > >>>>>>>>>>>>>>> weird to
> > > > > > >>>>>>>>>>>>>>>>>>>> write a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> commit
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with a bumped epoch. Should we
> > > > > > >>>>>>>> only
> > > > > > >>>>>>>>>> bump
> > > > > > >>>>>>>>>>>> up
> > > > > > >>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>> epoch
> > > > > > >>>>>>>>>>>>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> EndTxnResponse
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rename the field to sth like
> > > > > > >>>>>>>>>>>>>>> nextProducerEpoch?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 12, 2022 at 8:54
> > > > > > >>>>>> PM
> > > > > > >>>>>>>>>> Matthias
> > > > > > >>>>>>>>>>>> J.
> > > > > > >>>>>>>>>>>>>>> Sax <
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the background.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30: SGTM. My proposal was
> > > > > > >>>>>>>> only
> > > > > > >>>>>>>>>>>> focusing
> > > > > > >>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>> avoid
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> dangling
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
> > > > > > >>>>>>>> added
> > > > > > >>>>>>>>>>>> without
> > > > > > >>>>>>>>>>>>>>>>>> registered
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> partition.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> --
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more
> > > > > > >>>>>> details
> > > > > > >>>>>>>> to
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>> KIP
> > > > > > >>>>>>>>>>>>>>> about
> > > > > > >>>>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> scenario
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40: I think you hit a fair
> > > > > > >>>>>>> point
> > > > > > >>>>>>>>>> about
> > > > > > >>>>>>>>>>>> race
> > > > > > >>>>>>>>>>>>>>>>>>>> conditions
> > > > > > >>>>>>>>>>>>>>>>>>>>> or
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> client
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> bugs
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (incorrectly not bumping the
> > > > > > >>>>>>>>>> epoch). The
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> complexity/confusion
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> using
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the bumped epoch I see, is
> > > > > > >>>>>>>> mainly
> > > > > > >>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>> internal
> > > > > > >>>>>>>>>>>>>>>>>>>>> debugging,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> ie,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> inspecting
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log segment dumps -- it
> > > > > > >>>>>> seems
> > > > > > >>>>>>>>>> harder to
> > > > > > >>>>>>>>>>>>>>> reason
> > > > > > >>>>>>>>>>>>>>>>>> about
> > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> system
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> us
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> humans. But if we get better
> > > > > > >>>>>>>>>>>> guarantees, it
> > > > > > >>>>>>>>>>>>>>>> would
> > > > > > >>>>>>>>>>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> worth to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> use
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped epoch.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60: as I mentioned already,
> > > > > > >>>>>> I
> > > > > > >>>>>>>>> don't
> > > > > > >>>>>>>>>>>> know the
> > > > > > >>>>>>>>>>>>>>>>> broker
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> internals
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> provide
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more input. So if nobody
> > > > > > >>>>>> else
> > > > > > >>>>>>>>> chimes
> > > > > > >>>>>>>>>>>> in, we
> > > > > > >>>>>>>>>>>>>>>>> should
> > > > > > >>>>>>>>>>>>>>>>>>>> just
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> move
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> forward
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with your proposal.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 12/6/22 4:22 PM, Justine
> > > > > > >>>>>>>> Olshan
> > > > > > >>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi all,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> After Artem's questions
> > > > > > >>>>>>> about
> > > > > > >>>>>>>>>> error
> > > > > > >>>>>>>>>>>>>>> behavior,
> > > > > > >>>>>>>>>>>>>>>>>> I've
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> re-evaluated
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unknown producer ID
> > > > > > >>>>>>> exception
> > > > > > >>>>>>>>> and
> > > > > > >>>>>>>>>> had
> > > > > > >>>>>>>>>>>> some
> > > > > > >>>>>>>>>>>>>>>>>>>> discussions
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> offline.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think generally it makes
> > > > > > >>>>>>>> sense
> > > > > > >>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>> simplify
> > > > > > >>>>>>>>>>>>>>>>>> error
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> handling
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> cases
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this and the
> > > > > > >>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > >>>>>>>>>> error
> > > > > > >>>>>>>>>>>>>>> has a
> > > > > > >>>>>>>>>>>>>>>>>> pretty
> > > > > > >>>>>>>>>>>>>>>>>>>>> long
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complicated
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> history. Because of this,
> > > > > > >>>>>> I
> > > > > > >>>>>>>>>> propose
> > > > > > >>>>>>>>>>>>>>> adding a
> > > > > > >>>>>>>>>>>>>>>>> new
> > > > > > >>>>>>>>>>>>>>>>>>>> error
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> code
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ABORTABLE_ERROR
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that when encountered by
> > > > > > >>>>>> new
> > > > > > >>>>>>>>>> clients
> > > > > > >>>>>>>>>>>>>>> (gated
> > > > > > >>>>>>>>>>>>>>>> by
> > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> produce
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will simply abort the
> > > > > > >>>>>>>>> transaction.
> > > > > > >>>>>>>>>>>> This
> > > > > > >>>>>>>>>>>>>>>> allows
> > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> server
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> some
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> say
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in whether the client
> > > > > > >>>>>> aborts
> > > > > > >>>>>>>> and
> > > > > > >>>>>>>>>> makes
> > > > > > >>>>>>>>>>>>>>>> handling
> > > > > > >>>>>>>>>>>>>>>>>>>> much
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> simpler.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> future, we can also use
> > > > > > >>>>>> this
> > > > > > >>>>>>>>>> error in
> > > > > > >>>>>>>>>>>>>>> other
> > > > > > >>>>>>>>>>>>>>>>>>>> situations
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> where
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> want
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> abort the transactions. We
> > > > > > >>>>>>> can
> > > > > > >>>>>>>>>> even
> > > > > > >>>>>>>>>>>> use on
> > > > > > >>>>>>>>>>>>>>>>> other
> > > > > > >>>>>>>>>>>>>>>>>>>> apis.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've added this to the
> > > > > > >>>>>> KIP.
> > > > > > >>>>>>>> Let
> > > > > > >>>>>>>>> me
> > > > > > >>>>>>>>>>>> know if
> > > > > > >>>>>>>>>>>>>>>>> there
> > > > > > >>>>>>>>>>>>>>>>>>>> are
> > > > > > >>>>>>>>>>>>>>>>>>>>> any
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> questions
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> or
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 2, 2022 at
> > > > > > >>>>>> 10:22
> > > > > > >>>>>>>> AM
> > > > > > >>>>>>>>>>>> Justine
> > > > > > >>>>>>>>>>>>>>>> Olshan
> > > > > > >>>>>>>>>>>>>>>>> <
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> jolshan@confluent.io
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey Matthias,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30 — Maybe I also
> > > > > > >>>>>> didn't
> > > > > > >>>>>>>>>> express
> > > > > > >>>>>>>>>>>>>>> myself
> > > > > > >>>>>>>>>>>>>>>>>>>> clearly.
> > > > > > >>>>>>>>>>>>>>>>>>>>> For
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't have a way to
> > > > > > >>>>>>>> distinguish
> > > > > > >>>>>>>>>>>> between a
> > > > > > >>>>>>>>>>>>>>>>>> previous
> > > > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> current
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction since we
> > > > > > >>>>>> don't
> > > > > > >>>>>>>> have
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>> epoch
> > > > > > >>>>>>>>>>>>>>>>> bump.
> > > > > > >>>>>>>>>>>>>>>>>>>> This
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> means
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message from the previous
> > > > > > >>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>> may be
> > > > > > >>>>>>>>>>>>>>>>>>>> added to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> one.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> With
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older clients — we can't
> > > > > > >>>>>>>>>> guarantee
> > > > > > >>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>> won't
> > > > > > >>>>>>>>>>>>>>>>>>>> happen
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> if we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> already
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call
> > > > > > >>>>>>> (why
> > > > > > >>>>>>>> we
> > > > > > >>>>>>>>>> make
> > > > > > >>>>>>>>>>>>>>> changes
> > > > > > >>>>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> newer
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> client)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can at least gate some by
> > > > > > >>>>>>>>>> ensuring
> > > > > > >>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>> partition
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> has
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> been
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> added
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. The
> > > > > > >>>>>> rationale
> > > > > > >>>>>>>> here
> > > > > > >>>>>>>>>> is
> > > > > > >>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>>> there
> > > > > > >>>>>>>>>>>>>>>>>> are
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> likely
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> LESS
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrivals
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as time goes on, so
> > > > > > >>>>>>> hopefully
> > > > > > >>>>>>>>>> most
> > > > > > >>>>>>>>>>>> late
> > > > > > >>>>>>>>>>>>>>>>> arrivals
> > > > > > >>>>>>>>>>>>>>>>>>>> will
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> come
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> BEFORE
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call.
> > > > > > >>>>>>>> Those
> > > > > > >>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>> arrive
> > > > > > >>>>>>>>>>>>>>>>>> before
> > > > > > >>>>>>>>>>>>>>>>>>>>> will
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> properly
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> gated
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> > > > > > >>>>>>> describeTransactions
> > > > > > >>>>>>>>>>>> approach.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we take the approach
> > > > > > >>>>>> you
> > > > > > >>>>>>>>>>>> suggested,
> > > > > > >>>>>>>>>>>>>>> ANY
> > > > > > >>>>>>>>>>>>>>>>> late
> > > > > > >>>>>>>>>>>>>>>>>>>>> arrival
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> from a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> previous
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction will be
> > > > > > >>>>>> added.
> > > > > > >>>>>>>> And
> > > > > > >>>>>>>>> we
> > > > > > >>>>>>>>>>>> don't
> > > > > > >>>>>>>>>>>>>>> want
> > > > > > >>>>>>>>>>>>>>>>>>>> that. I
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> also
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> see
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> benefit in sending
> > > > > > >>>>>>>>>> addPartitionsToTxn
> > > > > > >>>>>>>>>>>>>>> over
> > > > > > >>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> describeTxns
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> They
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> both be one extra RPC to
> > > > > > >>>>>>> the
> > > > > > >>>>>>>>> Txn
> > > > > > >>>>>>>>>>>>>>>> coordinator.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To be clear — newer
> > > > > > >>>>>> clients
> > > > > > >>>>>>>>> will
> > > > > > >>>>>>>>>> use
> > > > > > >>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> instead
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTxns.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that if we
> > > > > > >>>>>>> have
> > > > > > >>>>>>>>>> some
> > > > > > >>>>>>>>>>>> delay
> > > > > > >>>>>>>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>> client
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> bump
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it could continue to send
> > > > > > >>>>>>>> epoch
> > > > > > >>>>>>>>>> 73
> > > > > > >>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>> those
> > > > > > >>>>>>>>>>>>>>>>>>>> records
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fenced.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Perhaps this is not an
> > > > > > >>>>>>> issue
> > > > > > >>>>>>>> if
> > > > > > >>>>>>>>>> we
> > > > > > >>>>>>>>>>>> don't
> > > > > > >>>>>>>>>>>>>>>> allow
> > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>> next
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> produce
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> go
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through before the EndTxn
> > > > > > >>>>>>>>> request
> > > > > > >>>>>>>>>>>>>>> returns.
> > > > > > >>>>>>>>>>>>>>>> I'm
> > > > > > >>>>>>>>>>>>>>>>>>>> also
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> thinking
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> about
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cases of
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure. I will need to
> > > > > > >>>>>>> think
> > > > > > >>>>>>>>> on
> > > > > > >>>>>>>>>>>> this a
> > > > > > >>>>>>>>>>>>>>> bit.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I wasn't sure if it was
> > > > > > >>>>>>> that
> > > > > > >>>>>>>>>>>> confusing.
> > > > > > >>>>>>>>>>>>>>> But
> > > > > > >>>>>>>>>>>>>>>> if
> > > > > > >>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> think it
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> is,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> investigate other ways.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure these are
> > > > > > >>>>>> the
> > > > > > >>>>>>>> same
> > > > > > >>>>>>>>>>>>>>> purgatories
> > > > > > >>>>>>>>>>>>>>>>>> since
> > > > > > >>>>>>>>>>>>>>>>>>>> one
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> is a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> produce
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory (I was planning
> > > > > > >>>>>>> on
> > > > > > >>>>>>>>>> using a
> > > > > > >>>>>>>>>>>>>>>> callback
> > > > > > >>>>>>>>>>>>>>>>>>>> rather
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> than
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the other is simply a
> > > > > > >>>>>>> request
> > > > > > >>>>>>>>> to
> > > > > > >>>>>>>>>>>> append
> > > > > > >>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>> log.
> > > > > > >>>>>>>>>>>>>>>>>>>>> Not
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> sure
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> structure here for
> > > > > > >>>>>>> ordering,
> > > > > > >>>>>>>>> but
> > > > > > >>>>>>>>>> my
> > > > > > >>>>>>>>>>>>>>>>>> understanding
> > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handle the write request
> > > > > > >>>>>>>> before
> > > > > > >>>>>>>>>> it
> > > > > > >>>>>>>>>>>> hears
> > > > > > >>>>>>>>>>>>>>>> back
> > > > > > >>>>>>>>>>>>>>>>>> from
> > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> Txn
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Coordinator.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if I
> > > > > > >>>>>>>> misunderstood
> > > > > > >>>>>>>>>>>> something
> > > > > > >>>>>>>>>>>>>>> or
> > > > > > >>>>>>>>>>>>>>>>>>>> something
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> was
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> unclear.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 1, 2022 at
> > > > > > >>>>>>> 12:15
> > > > > > >>>>>>>> PM
> > > > > > >>>>>>>>>>>> Matthias
> > > > > > >>>>>>>>>>>>>>> J.
> > > > > > >>>>>>>>>>>>>>>>> Sax
> > > > > > >>>>>>>>>>>>>>>>>> <
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the details
> > > > > > >>>>>>>>> Justine!
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side change
> > > > > > >>>>>>> for
> > > > > > >>>>>>>> 2
> > > > > > >>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>> removing
> > > > > > >>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> addPartitions
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need to
> > > > > > >>>>>>> make
> > > > > > >>>>>>>>>> this
> > > > > > >>>>>>>>>>>> from
> > > > > > >>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>> producer
> > > > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think I did not
> > > > > > >>>>>> express
> > > > > > >>>>>>>>> myself
> > > > > > >>>>>>>>>>>>>>> clearly. I
> > > > > > >>>>>>>>>>>>>>>>>>>>> understand
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (and
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should) change the
> > > > > > >>>>>>> producer
> > > > > > >>>>>>>> to
> > > > > > >>>>>>>>>> not
> > > > > > >>>>>>>>>>>> send
> > > > > > >>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> longer. But I don't
> > > > > > >>>>>> thinks
> > > > > > >>>>>>>>> it's
> > > > > > >>>>>>>>>>>>>>> requirement
> > > > > > >>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>> change
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> broker?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What I am trying to say
> > > > > > >>>>>>> is:
> > > > > > >>>>>>>>> as a
> > > > > > >>>>>>>>>>>>>>> safe-guard
> > > > > > >>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> improvement
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producers, the partition
> > > > > > >>>>>>>>> leader
> > > > > > >>>>>>>>>> can
> > > > > > >>>>>>>>>>>> just
> > > > > > >>>>>>>>>>>>>>>> send
> > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request to the
> > > > > > >>>>>>>> TX-coordinator
> > > > > > >>>>>>>>>> in any
> > > > > > >>>>>>>>>>>>>>> case
> > > > > > >>>>>>>>>>>>>>>> --
> > > > > > >>>>>>>>>>>>>>>>> if
> > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> old
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> correctly did send the
> > > > > > >>>>>>>>>>>> `addPartition`
> > > > > > >>>>>>>>>>>>>>>> request
> > > > > > >>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> already, the
> > > > > > >>>>>>> TX-coordinator
> > > > > > >>>>>>>>> can
> > > > > > >>>>>>>>>> just
> > > > > > >>>>>>>>>>>>>>>> "ignore"
> > > > > > >>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>> as
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> idempotent.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if the old producer has
> > > > > > >>>>>> a
> > > > > > >>>>>>>> bug
> > > > > > >>>>>>>>>> and
> > > > > > >>>>>>>>>>>> did
> > > > > > >>>>>>>>>>>>>>>> forget
> > > > > > >>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>> sent
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `addPartition`
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request, we would now
> > > > > > >>>>>>> ensure
> > > > > > >>>>>>>>>> that
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>> partition
> > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> indeed
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> added
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX and thus fix a
> > > > > > >>>>>>> potential
> > > > > > >>>>>>>>>>>> producer bug
> > > > > > >>>>>>>>>>>>>>>>> (even
> > > > > > >>>>>>>>>>>>>>>>>>>> if we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> get
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fencing via the bump
> > > > > > >>>>>>> epoch).
> > > > > > >>>>>>>>> --
> > > > > > >>>>>>>>>> It
> > > > > > >>>>>>>>>>>>>>> seems to
> > > > > > >>>>>>>>>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>>>>>>>> good
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Or
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there a reason to not do
> > > > > > >>>>>>>> this?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is ongoing
> > > > > > >>>>>> =
> > > > > > >>>>>>>>>> partition
> > > > > > >>>>>>>>>>>> was
> > > > > > >>>>>>>>>>>>>>>> added
> > > > > > >>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> via
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn. We
> > > > > > >>>>>>>> check
> > > > > > >>>>>>>>>> this
> > > > > > >>>>>>>>>>>> with
> > > > > > >>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this wasn't
> > > > > > >>>>>>>>>> sufficiently
> > > > > > >>>>>>>>>>>>>>>>> explained
> > > > > > >>>>>>>>>>>>>>>>>>>> here:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do what I propose
> > > > > > >>>>>> in
> > > > > > >>>>>>>>>> (20), we
> > > > > > >>>>>>>>>>>>>>> don't
> > > > > > >>>>>>>>>>>>>>>>>> really
> > > > > > >>>>>>>>>>>>>>>>>>>>> need
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> make
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `DescribeTransaction`
> > > > > > >>>>>>> call,
> > > > > > >>>>>>>> as
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>> partition
> > > > > > >>>>>>>>>>>>>>>>>>>> leader
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> adds
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for older clients and we
> > > > > > >>>>>>> get
> > > > > > >>>>>>>>>> this
> > > > > > >>>>>>>>>>>> check
> > > > > > >>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>> free.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is that
> > > > > > >>>>>> if
> > > > > > >>>>>>>> any
> > > > > > >>>>>>>>>>>> messages
> > > > > > >>>>>>>>>>>>>>>>> somehow
> > > > > > >>>>>>>>>>>>>>>>>>>> come
> > > > > > >>>>>>>>>>>>>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> before
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the producer,
> > > > > > >>>>>>> they
> > > > > > >>>>>>>>>> will be
> > > > > > >>>>>>>>>>>>>>>> fenced.
> > > > > > >>>>>>>>>>>>>>>>>>>>> However,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it can be
> > > > > > >>>>>>>>>> discussed
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree that we should
> > > > > > >>>>>>> have
> > > > > > >>>>>>>>>> epoch
> > > > > > >>>>>>>>>>>>>>> fencing.
> > > > > > >>>>>>>>>>>>>>>> My
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> question is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> different:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Assume we are at epoch
> > > > > > >>>>>> 73,
> > > > > > >>>>>>>> and
> > > > > > >>>>>>>>>> we
> > > > > > >>>>>>>>>>>> have
> > > > > > >>>>>>>>>>>>>>> an
> > > > > > >>>>>>>>>>>>>>>>>> ongoing
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed. It seems
> > > > > > >>>>>>> natural
> > > > > > >>>>>>>> to
> > > > > > >>>>>>>>>>>> write the
> > > > > > >>>>>>>>>>>>>>>>>> "prepare
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> commit"
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> marker
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
> > > > > > >>>>>>> both
> > > > > > >>>>>>>>> with
> > > > > > >>>>>>>>>>>> epoch
> > > > > > >>>>>>>>>>>>>>> 73,
> > > > > > >>>>>>>>>>>>>>>>> too,
> > > > > > >>>>>>>>>>>>>>>>>>>> as
> > > > > > >>>>>>>>>>>>>>>>>>>>> it
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> belongs
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current transaction. Of
> > > > > > >>>>>>>>> course,
> > > > > > >>>>>>>>>> we
> > > > > > >>>>>>>>>>>> now
> > > > > > >>>>>>>>>>>>>>> also
> > > > > > >>>>>>>>>>>>>>>>>> bump
> > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> expect
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the next requests to
> > > > > > >>>>>> have
> > > > > > >>>>>>>>> epoch
> > > > > > >>>>>>>>>> 74,
> > > > > > >>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>> would
> > > > > > >>>>>>>>>>>>>>>>>>>> reject
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> an
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch 73, as the
> > > > > > >>>>>>>> corresponding
> > > > > > >>>>>>>>>> TX
> > > > > > >>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>> epoch
> > > > > > >>>>>>>>>>>>>>>>> 73
> > > > > > >>>>>>>>>>>>>>>>>>>> was
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> already
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems you propose to
> > > > > > >>>>>>>> write
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>> "prepare
> > > > > > >>>>>>>>>>>>>>>>>> commit
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> marker"
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
> > > > > > >>>>>>> with
> > > > > > >>>>>>>>>> epoch 74
> > > > > > >>>>>>>>>>>>>>>> though,
> > > > > > >>>>>>>>>>>>>>>>>> what
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> work,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems confusing. Is
> > > > > > >>>>>> there
> > > > > > >>>>>>> a
> > > > > > >>>>>>>>>> reason
> > > > > > >>>>>>>>>>>> why
> > > > > > >>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>> would
> > > > > > >>>>>>>>>>>>>>>>>>>> use
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> bumped
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 74
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of the current
> > > > > > >>>>>>> epoch
> > > > > > >>>>>>>>> 73?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are checking if
> > > > > > >>>>>>> the
> > > > > > >>>>>>>>>>>>>>> transaction is
> > > > > > >>>>>>>>>>>>>>>>>>>> ongoing,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the leader
> > > > > > >>>>>>>>> partition
> > > > > > >>>>>>>>>> to
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for this
> > > > > > >>>>>>>>>> message to
> > > > > > >>>>>>>>>>>> come
> > > > > > >>>>>>>>>>>>>>>>> back,
> > > > > > >>>>>>>>>>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> theory
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
> > > > > > >>>>>> that
> > > > > > >>>>>>>>> would
> > > > > > >>>>>>>>>>>> make the
> > > > > > >>>>>>>>>>>>>>>>>> original
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> result
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out of
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why we
> > > > > > >>>>>> can
> > > > > > >>>>>>>>> check
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>> leader
> > > > > > >>>>>>>>>>>>>>>>>> state
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> before
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks. Got it.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, is this really
> > > > > > >>>>>> an
> > > > > > >>>>>>>>>> issue?
> > > > > > >>>>>>>>>>>> We put
> > > > > > >>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>> produce
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory, so how could
> > > > > > >>>>>> we
> > > > > > >>>>>>>>>> process
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> first?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Don't we need to put the
> > > > > > >>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
> > > > > > >>>>>>>>>>>>>>>>>>>>> into
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> too,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for this case, and
> > > > > > >>>>>> process
> > > > > > >>>>>>>>> both
> > > > > > >>>>>>>>>>>> request
> > > > > > >>>>>>>>>>>>>>>>>> in-order?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> (Again,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> my
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> knowledge is limited and
> > > > > > >>>>>>>> maybe
> > > > > > >>>>>>>>>> we
> > > > > > >>>>>>>>>>>> don't
> > > > > > >>>>>>>>>>>>>>>>>> maintain
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> order
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case, what seems to be
> > > > > > >>>>>> an
> > > > > > >>>>>>>>> issue
> > > > > > >>>>>>>>>>>> IMHO,
> > > > > > >>>>>>>>>>>>>>> and I
> > > > > > >>>>>>>>>>>>>>>>> am
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> wondering
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> changing
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request handling to
> > > > > > >>>>>>> preserve
> > > > > > >>>>>>>>>> order
> > > > > > >>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>> case
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> might be
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> cleaner
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution?)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/30/22 3:28 PM,
> > > > > > >>>>>> Artem
> > > > > > >>>>>>>>>> Livshits
> > > > > > >>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think the interesting
> > > > > > >>>>>>>> part
> > > > > > >>>>>>>>> is
> > > > > > >>>>>>>>>>>> not in
> > > > > > >>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>> logic
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> (because
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> tries
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> figure out when
> > > > > > >>>>>>>>>>>> UNKNOWN_PRODUCER_ID is
> > > > > > >>>>>>>>>>>>>>>>>> retriable
> > > > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> it's
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retryable,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's definitely not
> > > > > > >>>>>>> fatal),
> > > > > > >>>>>>>>> but
> > > > > > >>>>>>>>>>>> what
> > > > > > >>>>>>>>>>>>>>>> happens
> > > > > > >>>>>>>>>>>>>>>>>>>> when
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> logic
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'true' and falls
> > > > > > >>>>>> through.
> > > > > > >>>>>>>> In
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>> old
> > > > > > >>>>>>>>>>>>>>>>> clients
> > > > > > >>>>>>>>>>>>>>>>>> it
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> seems
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> keep the behavior in
> > > > > > >>>>>> the
> > > > > > >>>>>>>> new
> > > > > > >>>>>>>>>>>> clients,
> > > > > > >>>>>>>>>>>>>>> I'd
> > > > > > >>>>>>>>>>>>>>>>>>>> expect it
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> well.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 29, 2022 at
> > > > > > >>>>>>>> 11:57
> > > > > > >>>>>>>>>> AM
> > > > > > >>>>>>>>>>>>>>> Justine
> > > > > > >>>>>>>>>>>>>>>>>> Olshan
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>> <jolshan@confluent.io.invalid
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Artem and Jeff,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
> > > > > > >>>>>> look
> > > > > > >>>>>>>> and
> > > > > > >>>>>>>>>>>> sorry for
> > > > > > >>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>> slow
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> response.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You both mentioned the
> > > > > > >>>>>>>>> change
> > > > > > >>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>> handle
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear — this error
> > > > > > >>>>>> code
> > > > > > >>>>>>>> will
> > > > > > >>>>>>>>>> only
> > > > > > >>>>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>> sent
> > > > > > >>>>>>>>>>>>>>>>>> again
> > > > > > >>>>>>>>>>>>>>>>>>>>> when
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> client's
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version is high enough
> > > > > > >>>>>>> to
> > > > > > >>>>>>>>>> ensure
> > > > > > >>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>> handle
> > > > > > >>>>>>>>>>>>>>>>> it
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> correctly.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The current (Java)
> > > > > > >>>>>>> client
> > > > > > >>>>>>>>>> handles
> > > > > > >>>>>>>>>>>>>>> this by
> > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> following
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> (somewhat
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> long)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code snippet:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // An
> > > > > > >>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > >>>>>>>>>> means
> > > > > > >>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>> have
> > > > > > >>>>>>>>>>>>>>>>>>>> lost
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker. Depending on
> > > > > > >>>>>> the
> > > > > > >>>>>>>> log
> > > > > > >>>>>>>>>> start
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // offset, we may want
> > > > > > >>>>>>> to
> > > > > > >>>>>>>>>> retry
> > > > > > >>>>>>>>>>>>>>> these, as
> > > > > > >>>>>>>>>>>>>>>>>>>>> described
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> each
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> case
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below. If
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> none of those apply,
> > > > > > >>>>>>> then
> > > > > > >>>>>>>>> for
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // idempotent
> > > > > > >>>>>> producer,
> > > > > > >>>>>>> we
> > > > > > >>>>>>>>>> will
> > > > > > >>>>>>>>>>>>>>> locally
> > > > > > >>>>>>>>>>>>>>>>> bump
> > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> reset
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence numbers of
> > > > > > >>>>>>>>> in-flight
> > > > > > >>>>>>>>>>>> batches
> > > > > > >>>>>>>>>>>>>>>> from
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // sequence 0, then
> > > > > > >>>>>>> retry
> > > > > > >>>>>>>>> the
> > > > > > >>>>>>>>>>>> failed
> > > > > > >>>>>>>>>>>>>>>> batch,
> > > > > > >>>>>>>>>>>>>>>>>>>> which
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> should
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> now
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> succeed.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the transactional
> > > > > > >>>>>>>> producer,
> > > > > > >>>>>>>>>> allow
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // batch to fail. When
> > > > > > >>>>>>>>>> processing
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>> failed
> > > > > > >>>>>>>>>>>>>>>>>>>>> batch,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transition
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an abortable error and
> > > > > > >>>>>>>> set a
> > > > > > >>>>>>>>>> flag
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // indicating that we
> > > > > > >>>>>>> need
> > > > > > >>>>>>>>> to
> > > > > > >>>>>>>>>>>> bump the
> > > > > > >>>>>>>>>>>>>>>>> epoch
> > > > > > >>>>>>>>>>>>>>>>>>>> (if
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> supported
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> by
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker).
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if (error ==
> > > > > > >>>>>>>>>>>>>>>> Errors.*UNKNOWN_PRODUCER_ID*)
> > > > > > >>>>>>>>>>>>>>>>> {
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > > > > > >>>>>>>>>> (response.logStartOffset
> > > > > > >>>>>>>>>>>> ==
> > > > > > >>>>>>>>>>>>>>> -1)
> > > > > > >>>>>>>>>>>>>>>> {
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // We don't
> > > > > > >>>>>>> know
> > > > > > >>>>>>>>>> the log
> > > > > > >>>>>>>>>>>>>>> start
> > > > > > >>>>>>>>>>>>>>>>>> offset
> > > > > > >>>>>>>>>>>>>>>>>>>>> with
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> response.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the request
> > > > > > >>>>>>>> until
> > > > > > >>>>>>>>>> we
> > > > > > >>>>>>>>>>>> get
> > > > > > >>>>>>>>>>>>>>> it.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // The
> > > > > > >>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > >>>>>>>>>>>>>>>> error
> > > > > > >>>>>>>>>>>>>>>>>> code
> > > > > > >>>>>>>>>>>>>>>>>>>>> was
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> added
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> along
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProduceResponse which
> > > > > > >>>>>>>>>> includes the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              //
> > > > > > >>>>>>>> logStartOffset.
> > > > > > >>>>>>>>>> So
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>> '-1'
> > > > > > >>>>>>>>>>>>>>>>>>>> sentinel
> > > > > > >>>>>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> backward
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatibility.
> > > > > > >>>>>> Instead,
> > > > > > >>>>>>> it
> > > > > > >>>>>>>>> is
> > > > > > >>>>>>>>>>>> possible
> > > > > > >>>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // a broker
> > > > > > >>>>>> to
> > > > > > >>>>>>>> not
> > > > > > >>>>>>>>>> know
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> logStartOffset at
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> when
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returning
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response because
> > > > > > >>>>>> the
> > > > > > >>>>>>>>>> partition
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // may have
> > > > > > >>>>>>>> moved
> > > > > > >>>>>>>>>> away
> > > > > > >>>>>>>>>>>> from
> > > > > > >>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>> broker
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> from
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error was
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initially raised to
> > > > > > >>>>>> the
> > > > > > >>>>>>>> time
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // response
> > > > > > >>>>>>> was
> > > > > > >>>>>>>>>> being
> > > > > > >>>>>>>>>>>>>>>>> constructed.
> > > > > > >>>>>>>>>>>>>>>>>> In
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> these
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> cases,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retry the request: we
> > > > > > >>>>>>> are
> > > > > > >>>>>>>>>>>> guaranteed
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // to
> > > > > > >>>>>>> eventually
> > > > > > >>>>>>>>>> get a
> > > > > > >>>>>>>>>>>>>>>>>> logStartOffset
> > > > > > >>>>>>>>>>>>>>>>>>>>> once
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> things
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> settle
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> down.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > > > > > >>>>>>>>>>>>>>> (batch.sequenceHasBeenReset()) {
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // When the
> > > > > > >>>>>>>> first
> > > > > > >>>>>>>>>>>> inflight
> > > > > > >>>>>>>>>>>>>>>> batch
> > > > > > >>>>>>>>>>>>>>>>>>>> fails
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> due to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> truncation
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> then the sequences of
> > > > > > >>>>>>> all
> > > > > > >>>>>>>>> the
> > > > > > >>>>>>>>>>>> other
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // in flight
> > > > > > >>>>>>>>> batches
> > > > > > >>>>>>>>>>>> would
> > > > > > >>>>>>>>>>>>>>> have
> > > > > > >>>>>>>>>>>>>>>>>> been
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> restarted
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> from
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> beginning.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, when those
> > > > > > >>>>>>>>> responses
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // come back
> > > > > > >>>>>>>> from
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>> broker,
> > > > > > >>>>>>>>>>>>>>>>> they
> > > > > > >>>>>>>>>>>>>>>>>>>> would
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> also
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> come
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > >>>>>>> error.
> > > > > > >>>>>>>>> In
> > > > > > >>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>> case,
> > > > > > >>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>> should
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // reset the
> > > > > > >>>>>>>>>> sequence
> > > > > > >>>>>>>>>>>>>>> numbers
> > > > > > >>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> beginning.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          } else if
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>> (lastAckedOffset(batch.topicPartition).orElse(
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> response.logStartOffset) {
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // The head
> > > > > > >>>>>> of
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>> log
> > > > > > >>>>>>>>>>>> has
> > > > > > >>>>>>>>>>>>>>> been
> > > > > > >>>>>>>>>>>>>>>>>>>> removed,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> probably
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> due
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retention time
> > > > > > >>>>>> elapsing.
> > > > > > >>>>>>>> In
> > > > > > >>>>>>>>>> this
> > > > > > >>>>>>>>>>>> case,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // we expect
> > > > > > >>>>>>> to
> > > > > > >>>>>>>>>> lose the
> > > > > > >>>>>>>>>>>>>>>> producer
> > > > > > >>>>>>>>>>>>>>>>>>>> state.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> For
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer, reset the
> > > > > > >>>>>>>>> sequences
> > > > > > >>>>>>>>>> of
> > > > > > >>>>>>>>>>>> all
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // inflight
> > > > > > >>>>>>>>> batches
> > > > > > >>>>>>>>>> to
> > > > > > >>>>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>> from
> > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> beginning
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> retry
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the transaction
> > > > > > >>>>>>> does
> > > > > > >>>>>>>>> not
> > > > > > >>>>>>>>>>>> need to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // be
> > > > > > >>>>>> aborted.
> > > > > > >>>>>>>> For
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>> idempotent
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> producer,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> bump
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> avoid
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reusing (sequence,
> > > > > > >>>>>>> epoch)
> > > > > > >>>>>>>>>> pairs
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              if
> > > > > > >>>>>>>>>> (isTransactional()) {
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>
> > > > > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>> this.producerIdAndEpoch);
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              } else {
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>
> requestEpochBumpForPartition(batch.topicPartition);
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              }
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > > > > > >>>>>>>>> (!isTransactional())
> > > > > > >>>>>>>>>> {
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // For the
> > > > > > >>>>>>>>>> idempotent
> > > > > > >>>>>>>>>>>>>>> producer,
> > > > > > >>>>>>>>>>>>>>>>>>>> always
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> retry
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors. If the batch
> > > > > > >>>>>> has
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>>>> current
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // producer
> > > > > > >>>>>> ID
> > > > > > >>>>>>>> and
> > > > > > >>>>>>>>>>>> epoch,
> > > > > > >>>>>>>>>>>>>>>>> request a
> > > > > > >>>>>>>>>>>>>>>>>>>> bump
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Otherwise
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the
> > > > > > >>>>>> produce.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> requestEpochBumpForPartition(batch.topicPartition);
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I was considering
> > > > > > >>>>>>> keeping
> > > > > > >>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>> behavior —
> > > > > > >>>>>>>>>>>>>>>>> but
> > > > > > >>>>>>>>>>>>>>>>>>>> am
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> open
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simplifying
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We are leaving changes
> > > > > > >>>>>>> to
> > > > > > >>>>>>>>>> older
> > > > > > >>>>>>>>>>>>>>> clients
> > > > > > >>>>>>>>>>>>>>>> off
> > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> table
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> here
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> since
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> caused
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many issues for
> > > > > > >>>>>> clients
> > > > > > >>>>>>> in
> > > > > > >>>>>>>>> the
> > > > > > >>>>>>>>>>>> past.
> > > > > > >>>>>>>>>>>>>>>>>> Previously
> > > > > > >>>>>>>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> was
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we didn't have the
> > > > > > >>>>>>>>> mechanisms
> > > > > > >>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>>> place to
> > > > > > >>>>>>>>>>>>>>>>>>>> detect
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> when
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> was
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> legitimate
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case vs some bug or
> > > > > > >>>>>> gap
> > > > > > >>>>>>> in
> > > > > > >>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>> protocol.
> > > > > > >>>>>>>>>>>>>>>>>>>> Ensuring
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> each
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> its
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> own epoch should close
> > > > > > >>>>>>>> this
> > > > > > >>>>>>>>>> gap.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> And to address Jeff's
> > > > > > >>>>>>>> second
> > > > > > >>>>>>>>>>>> point:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *does the typical
> > > > > > >>>>>>> produce
> > > > > > >>>>>>>>>> request
> > > > > > >>>>>>>>>>>> path
> > > > > > >>>>>>>>>>>>>>>>> append
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> records
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> local
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> log
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along*
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *with the
> > > > > > >>>>>>>>>> currentTxnFirstOffset
> > > > > > >>>>>>>>>>>>>>>>> information?
> > > > > > >>>>>>>>>>>>>>>>>> I
> > > > > > >>>>>>>>>>>>>>>>>>>>> would
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> like
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand*
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *when the field is
> > > > > > >>>>>>> written
> > > > > > >>>>>>>>> to
> > > > > > >>>>>>>>>>>> disk.*
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, the first produce
> > > > > > >>>>>>>>> request
> > > > > > >>>>>>>>>>>>>>> populates
> > > > > > >>>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>>> field
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> writes
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> offset
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as part of the record
> > > > > > >>>>>>>> batch
> > > > > > >>>>>>>>>> and
> > > > > > >>>>>>>>>>>> also
> > > > > > >>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>> producer
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> state
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> snapshot.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we reload the records
> > > > > > >>>>>> on
> > > > > > >>>>>>>>>> restart
> > > > > > >>>>>>>>>>>>>>> and/or
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> reassignment,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> repopulate
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> field with the
> > > > > > >>>>>> snapshot
> > > > > > >>>>>>>> from
> > > > > > >>>>>>>>>> disk
> > > > > > >>>>>>>>>>>>>>> along
> > > > > > >>>>>>>>>>>>>>>>> with
> > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> rest
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if there
> > > > > > >>>>>> are
> > > > > > >>>>>>>>>> further
> > > > > > >>>>>>>>>>>>>>> comments
> > > > > > >>>>>>>>>>>>>>>>>>>> and/or
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> questions.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > > > > > >>>>>> at
> > > > > > >>>>>>>> 9:00
> > > > > > >>>>>>>>>> PM
> > > > > > >>>>>>>>>>>> Jeff
> > > > > > >>>>>>>>>>>>>>> Kim
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>> <jeff.kim@confluent.io.invalid
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP! I
> > > > > > >>>>>>>> have
> > > > > > >>>>>>>>>> two
> > > > > > >>>>>>>>>>>>>>>> questions:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1) For new clients,
> > > > > > >>>>>> we
> > > > > > >>>>>>>> can
> > > > > > >>>>>>>>>> once
> > > > > > >>>>>>>>>>>> again
> > > > > > >>>>>>>>>>>>>>>>> return
> > > > > > >>>>>>>>>>>>>>>>>>>> an
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> error
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for sequences
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that are non-zero
> > > > > > >>>>>> when
> > > > > > >>>>>>>>> there
> > > > > > >>>>>>>>>> is
> > > > > > >>>>>>>>>>>> no
> > > > > > >>>>>>>>>>>>>>>>> producer
> > > > > > >>>>>>>>>>>>>>>>>>>> state
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> present
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> on
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This will indicate we
> > > > > > >>>>>>>>> missed
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>> 0
> > > > > > >>>>>>>>>>>>>>>>> sequence
> > > > > > >>>>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> yet
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> want
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the log.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would like to
> > > > > > >>>>>>>> understand
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>> current
> > > > > > >>>>>>>>>>>>>>>>>>>> behavior
> > > > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> handle
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and if there are any
> > > > > > >>>>>>>>> changes
> > > > > > >>>>>>>>>> we
> > > > > > >>>>>>>>>>>> are
> > > > > > >>>>>>>>>>>>>>>>> making.
> > > > > > >>>>>>>>>>>>>>>>>>>> Maybe
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> I'm
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> missing
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but we would want to
> > > > > > >>>>>>>>> identify
> > > > > > >>>>>>>>>>>>>>> whether we
> > > > > > >>>>>>>>>>>>>>>>>>>> missed
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> the 0
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> sequence
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients, no?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2) Upon returning
> > > > > > >>>>>> from
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> coordinator, we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> set
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as ongoing on the
> > > > > > >>>>>>> leader
> > > > > > >>>>>>>> by
> > > > > > >>>>>>>>>>>>>>> populating
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> currentTxnFirstOffset
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through the typical
> > > > > > >>>>>>>> produce
> > > > > > >>>>>>>>>>>> request
> > > > > > >>>>>>>>>>>>>>>>>> handling.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does the typical
> > > > > > >>>>>>> produce
> > > > > > >>>>>>>>>> request
> > > > > > >>>>>>>>>>>> path
> > > > > > >>>>>>>>>>>>>>>>> append
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> records
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> local
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> log
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> > > > > > >>>>>>>>>> currentTxnFirstOffset
> > > > > > >>>>>>>>>>>>>>>>> information?
> > > > > > >>>>>>>>>>>>>>>>>> I
> > > > > > >>>>>>>>>>>>>>>>>>>>> would
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> like
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when the field is
> > > > > > >>>>>>> written
> > > > > > >>>>>>>>> to
> > > > > > >>>>>>>>>>>> disk.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeff
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > > > > > >>>>>> at
> > > > > > >>>>>>>>> 4:44
> > > > > > >>>>>>>>>> PM
> > > > > > >>>>>>>>>>>> Artem
> > > > > > >>>>>>>>>>>>>>>>>> Livshits
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
> > > > > > >>>>>>> alivshits@confluent.io
> > > > > > >>>>>>>>>> .invalid>
> > > > > > >>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for the
> > > > > > >>>>>> KIP.
> > > > > > >>>>>>>> I
> > > > > > >>>>>>>>>> have
> > > > > > >>>>>>>>>>>> one
> > > > > > >>>>>>>>>>>>>>>>>> question.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 5) For new clients,
> > > > > > >>>>>> we
> > > > > > >>>>>>>> can
> > > > > > >>>>>>>>>> once
> > > > > > >>>>>>>>>>>>>>> again
> > > > > > >>>>>>>>>>>>>>>>>> return
> > > > > > >>>>>>>>>>>>>>>>>>>> an
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> error
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe we had
> > > > > > >>>>>>>> problems
> > > > > > >>>>>>>>>> in the
> > > > > > >>>>>>>>>>>>>>> past
> > > > > > >>>>>>>>>>>>>>>>> with
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> returning
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because it was
> > > > > > >>>>>>>> considered
> > > > > > >>>>>>>>>> fatal
> > > > > > >>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>> required
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> client
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> restart.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> It
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good to spell out
> > > > > > >>>>>> the
> > > > > > >>>>>>>> new
> > > > > > >>>>>>>>>> client
> > > > > > >>>>>>>>>>>>>>>> behavior
> > > > > > >>>>>>>>>>>>>>>>>>>> when
> > > > > > >>>>>>>>>>>>>>>>>>>>> it
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> receives
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > > > > > >>>>>>> at
> > > > > > >>>>>>>>>> 10:00 AM
> > > > > > >>>>>>>>>>>>>>>> Justine
> > > > > > >>>>>>>>>>>>>>>>>>>> Olshan
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > > >>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
> > > > > > >>>>>>>> look
> > > > > > >>>>>>>>>>>> Matthias.
> > > > > > >>>>>>>>>>>>>>>> I've
> > > > > > >>>>>>>>>>>>>>>>>>>> tried
> > > > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> answer
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> your
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> questions
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 10)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Right — so the
> > > > > > >>>>>>> hanging
> > > > > > >>>>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>>> only
> > > > > > >>>>>>>>>>>>>>>>>>>> occurs
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> when
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> come in and the
> > > > > > >>>>>>>> partition
> > > > > > >>>>>>>>>> is
> > > > > > >>>>>>>>>>>> never
> > > > > > >>>>>>>>>>>>>>>> added
> > > > > > >>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> again.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never add the
> > > > > > >>>>>>> partition
> > > > > > >>>>>>>>> to
> > > > > > >>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>>>> transaction,
> > > > > > >>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>> will
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> never
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> marker
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never advance the
> > > > > > >>>>>>> LSO.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do end up
> > > > > > >>>>>>> adding
> > > > > > >>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>> partition
> > > > > > >>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (I
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suppose
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happen before or
> > > > > > >>>>>>> after
> > > > > > >>>>>>>>> the
> > > > > > >>>>>>>>>> late
> > > > > > >>>>>>>>>>>>>>>> message
> > > > > > >>>>>>>>>>>>>>>>>>>> comes
> > > > > > >>>>>>>>>>>>>>>>>>>>> in)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> then
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> include
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late message in the
> > > > > > >>>>>>>> next
> > > > > > >>>>>>>>>>>>>>> (incorrect)
> > > > > > >>>>>>>>>>>>>>>>>>>>> transaction.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> So perhaps it is
> > > > > > >>>>>>>> clearer
> > > > > > >>>>>>>>> to
> > > > > > >>>>>>>>>>>> make
> > > > > > >>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>> distinction
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> between
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> eventually get
> > > > > > >>>>>> added
> > > > > > >>>>>>> to
> > > > > > >>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>>>>> (but
> > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> wrong
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> one)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> or
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that never get
> > > > > > >>>>>> added
> > > > > > >>>>>>>> and
> > > > > > >>>>>>>>>> become
> > > > > > >>>>>>>>>>>>>>>> hanging.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side
> > > > > > >>>>>>> change
> > > > > > >>>>>>>>> for
> > > > > > >>>>>>>>>> 2 is
> > > > > > >>>>>>>>>>>>>>>> removing
> > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> addPartitions
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need
> > > > > > >>>>>>> to
> > > > > > >>>>>>>>> make
> > > > > > >>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>> from
> > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In my opinion, the
> > > > > > >>>>>>>> issue
> > > > > > >>>>>>>>>> with
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> call
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is that we don't
> > > > > > >>>>>> have
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>> epoch
> > > > > > >>>>>>>>>>>>>>> bump,
> > > > > > >>>>>>>>>>>>>>>> so
> > > > > > >>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>> don't
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> know
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> belongs to the
> > > > > > >>>>>>> previous
> > > > > > >>>>>>>>>>>>>>> transaction or
> > > > > > >>>>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>>> one.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> We
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition has been
> > > > > > >>>>>>>> added
> > > > > > >>>>>>>>> to
> > > > > > >>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>> transaction.
> > > > > > >>>>>>>>>>>>>>>>>>>>> Of
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> course,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> means
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't completely
> > > > > > >>>>>>> cover
> > > > > > >>>>>>>>> the
> > > > > > >>>>>>>>>> case
> > > > > > >>>>>>>>>>>>>>> where
> > > > > > >>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>> have a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> really
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have added the
> > > > > > >>>>>>>> partition
> > > > > > >>>>>>>>> to
> > > > > > >>>>>>>>>>>> the new
> > > > > > >>>>>>>>>>>>>>>>>>>>> transaction,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> that's
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something we will
> > > > > > >>>>>>> need
> > > > > > >>>>>>>>> the
> > > > > > >>>>>>>>>> new
> > > > > > >>>>>>>>>>>>>>> clients
> > > > > > >>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>> cover.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is
> > > > > > >>>>>>> ongoing
> > > > > > >>>>>>>> =
> > > > > > >>>>>>>>>>>> partition
> > > > > > >>>>>>>>>>>>>>> was
> > > > > > >>>>>>>>>>>>>>>>>>>> added to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> via
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn.
> > > > > > >>>>>>> We
> > > > > > >>>>>>>>>> check
> > > > > > >>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>> with
> > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this
> > > > > > >>>>>>> wasn't
> > > > > > >>>>>>>>>>>> sufficiently
> > > > > > >>>>>>>>>>>>>>>>>>>> explained
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> here:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>
> > > > > > >>>>
> > > > > > >>
> > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is
> > > > > > >>>>>> that
> > > > > > >>>>>>>> if
> > > > > > >>>>>>>>>> any
> > > > > > >>>>>>>>>>>>>>> messages
> > > > > > >>>>>>>>>>>>>>>>>>>> somehow
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> come
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> before
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the
> > > > > > >>>>>>> producer,
> > > > > > >>>>>>>>> they
> > > > > > >>>>>>>>>>>> will be
> > > > > > >>>>>>>>>>>>>>>>>> fenced.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> However,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it
> > > > > > >>>>>> can
> > > > > > >>>>>>> be
> > > > > > >>>>>>>>>>>> discussed
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 50)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It should be
> > > > > > >>>>>>>> synchronous
> > > > > > >>>>>>>>>>>> because
> > > > > > >>>>>>>>>>>>>>> if we
> > > > > > >>>>>>>>>>>>>>>>>> have
> > > > > > >>>>>>>>>>>>>>>>>>>> an
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> event
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (ie,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> an
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes us to need
> > > > > > >>>>>> to
> > > > > > >>>>>>>>> abort
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>> transaction,
> > > > > > >>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> need
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions to send
> > > > > > >>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>> markers
> > > > > > >>>>>>>>>>>>>>>>> to.
> > > > > > >>>>>>>>>>>>>>>>>>>> We
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> know
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we added them to
> > > > > > >>>>>> the
> > > > > > >>>>>>>>>>>> coordinator
> > > > > > >>>>>>>>>>>>>>> via
> > > > > > >>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Previously we have
> > > > > > >>>>>>> had
> > > > > > >>>>>>>>>>>> asynchronous
> > > > > > >>>>>>>>>>>>>>>>> calls
> > > > > > >>>>>>>>>>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> past
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (ie,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit markers when
> > > > > > >>>>>>> the
> > > > > > >>>>>>>>>>>>>>> transaction is
> > > > > > >>>>>>>>>>>>>>>>>>>>> completed)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> often
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes confusion as
> > > > > > >>>>>>> we
> > > > > > >>>>>>>>>> need to
> > > > > > >>>>>>>>>>>> wait
> > > > > > >>>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>> some
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> operations
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complete.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing commit
> > > > > > >>>>>>> markers
> > > > > > >>>>>>>>>> case,
> > > > > > >>>>>>>>>>>>>>> clients
> > > > > > >>>>>>>>>>>>>>>>> often
> > > > > > >>>>>>>>>>>>>>>>>>>> see
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>> CONCURRENT_TRANSACTIONs
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error messages and
> > > > > > >>>>>>> that
> > > > > > >>>>>>>>>> can be
> > > > > > >>>>>>>>>>>>>>>>> confusing.
> > > > > > >>>>>>>>>>>>>>>>>>>> For
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> reason,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> may
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simpler to just
> > > > > > >>>>>> have
> > > > > > >>>>>>>>>>>> synchronous
> > > > > > >>>>>>>>>>>>>>>> calls —
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> especially
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> block
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some operation's
> > > > > > >>>>>>>>> completion
> > > > > > >>>>>>>>>>>> anyway
> > > > > > >>>>>>>>>>>>>>>>> before
> > > > > > >>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>> can
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> start
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> next
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. And
> > > > > > >>>>>>> yes, I
> > > > > > >>>>>>>>>> meant
> > > > > > >>>>>>>>>>>>>>>>>> coordinator. I
> > > > > > >>>>>>>>>>>>>>>>>>>>> will
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> fix
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> that.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are
> > > > > > >>>>>> checking
> > > > > > >>>>>>> if
> > > > > > >>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> ongoing,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the
> > > > > > >>>>>> leader
> > > > > > >>>>>>>>>> partition
> > > > > > >>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for
> > > > > > >>>>>>> this
> > > > > > >>>>>>>>>>>> message to
> > > > > > >>>>>>>>>>>>>>>> come
> > > > > > >>>>>>>>>>>>>>>>>>>> back,
> > > > > > >>>>>>>>>>>>>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> theory
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
> > > > > > >>>>>>>> that
> > > > > > >>>>>>>>>> would
> > > > > > >>>>>>>>>>>> make
> > > > > > >>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>> original
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> result
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why
> > > > > > >>>>>> we
> > > > > > >>>>>>>> can
> > > > > > >>>>>>>>>> check
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>> leader
> > > > > > >>>>>>>>>>>>>>>>>>>> state
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> before
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm happy to update
> > > > > > >>>>>>> the
> > > > > > >>>>>>>>>> KIP if
> > > > > > >>>>>>>>>>>>>>> some of
> > > > > > >>>>>>>>>>>>>>>>>> these
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> things
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> were
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Nov 21,
> > > > > > >>>>>> 2022
> > > > > > >>>>>>> at
> > > > > > >>>>>>>>>> 7:11 PM
> > > > > > >>>>>>>>>>>>>>>> Matthias
> > > > > > >>>>>>>>>>>>>>>>>> J.
> > > > > > >>>>>>>>>>>>>>>>>>>>> Sax <
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the
> > > > > > >>>>>> KIP.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Couple of
> > > > > > >>>>>>>> clarification
> > > > > > >>>>>>>>>>>> questions
> > > > > > >>>>>>>>>>>>>>> (I
> > > > > > >>>>>>>>>>>>>>>> am
> > > > > > >>>>>>>>>>>>>>>>>>>> not a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> broker
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> expert
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> do
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some question are
> > > > > > >>>>>>>>> obvious
> > > > > > >>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>> others,
> > > > > > >>>>>>>>>>>>>>>>> but
> > > > > > >>>>>>>>>>>>>>>>>>>> not
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> me
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> my
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lack
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker knowledge).
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (10)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What happens if
> > > > > > >>>>>> the
> > > > > > >>>>>>>>>> message
> > > > > > >>>>>>>>>>>> come
> > > > > > >>>>>>>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>>>>>> before
> > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> next
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request? It seems
> > > > > > >>>>>>> the
> > > > > > >>>>>>>>>> broker
> > > > > > >>>>>>>>>>>>>>> hosting
> > > > > > >>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>> data
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anything about it
> > > > > > >>>>>>> and
> > > > > > >>>>>>>>>> append
> > > > > > >>>>>>>>>>>> it to
> > > > > > >>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> partition,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> too?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> What
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> difference between
> > > > > > >>>>>>>> both
> > > > > > >>>>>>>>>> cases?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, it seems a
> > > > > > >>>>>> TX
> > > > > > >>>>>>>>> would
> > > > > > >>>>>>>>>> only
> > > > > > >>>>>>>>>>>>>>> hang,
> > > > > > >>>>>>>>>>>>>>>> if
> > > > > > >>>>>>>>>>>>>>>>>>>> there
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> is no
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> following
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> either committer
> > > > > > >>>>>> or
> > > > > > >>>>>>>>>> aborted?
> > > > > > >>>>>>>>>>>> Thus,
> > > > > > >>>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>> case
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> above,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> TX
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> might
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually not hang
> > > > > > >>>>>>> (of
> > > > > > >>>>>>>>>> course,
> > > > > > >>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>> might
> > > > > > >>>>>>>>>>>>>>>>>> get
> > > > > > >>>>>>>>>>>>>>>>>>>> an
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> EOS
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> violation
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX was aborted and
> > > > > > >>>>>>> the
> > > > > > >>>>>>>>>> second
> > > > > > >>>>>>>>>>>>>>>>> committed,
> > > > > > >>>>>>>>>>>>>>>>>> or
> > > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> other
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> way
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around).
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (20)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Of course, 1 and
> > > > > > >>>>>> 2
> > > > > > >>>>>>>>>> require
> > > > > > >>>>>>>>>>>>>>>> client-side
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> changes, so
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> those approaches
> > > > > > >>>>>>> won’t
> > > > > > >>>>>>>>>> apply.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For (1) I
> > > > > > >>>>>> understand
> > > > > > >>>>>>>>> why a
> > > > > > >>>>>>>>>>>> client
> > > > > > >>>>>>>>>>>>>>>>> change
> > > > > > >>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> necessary,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we need a client
> > > > > > >>>>>>>> change
> > > > > > >>>>>>>>>> for
> > > > > > >>>>>>>>>>>> (2).
> > > > > > >>>>>>>>>>>>>>> Can
> > > > > > >>>>>>>>>>>>>>>>> you
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> --
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Later
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> explain
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that we should
> > > > > > >>>>>> send
> > > > > > >>>>>>> a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> DescribeTransactionRequest,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> but I
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> am
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Can't we not just
> > > > > > >>>>>> do
> > > > > > >>>>>>>> an
> > > > > > >>>>>>>>>>>> implicit
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> AddPartiitonToTx,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> too?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> If
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> old
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer correctly
> > > > > > >>>>>>>>>> registered
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>> partition
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> already,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can just ignore it
> > > > > > >>>>>>> as
> > > > > > >>>>>>>>>> it's an
> > > > > > >>>>>>>>>>>>>>>>> idempotent
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> operation?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (30)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To cover older
> > > > > > >>>>>>>> clients,
> > > > > > >>>>>>>>>> we
> > > > > > >>>>>>>>>>>> will
> > > > > > >>>>>>>>>>>>>>>>> ensure a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ongoing
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> before
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we write to a
> > > > > > >>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Not sure what you
> > > > > > >>>>>>> mean
> > > > > > >>>>>>>>> by
> > > > > > >>>>>>>>>>>> this?
> > > > > > >>>>>>>>>>>>>>> Can
> > > > > > >>>>>>>>>>>>>>>> you
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (40)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [the
> > > > > > >>>>>>> TX-coordinator]
> > > > > > >>>>>>>>> will
> > > > > > >>>>>>>>>>>> write
> > > > > > >>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>> prepare
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> commit
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch and send
> > > > > > >>>>>>>>>>>>>>> WriteTxnMarkerRequests
> > > > > > >>>>>>>>>>>>>>>>>> with
> > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> bumped
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why do we use the
> > > > > > >>>>>>>> bumped
> > > > > > >>>>>>>>>>>> epoch for
> > > > > > >>>>>>>>>>>>>>>>> both?
> > > > > > >>>>>>>>>>>>>>>>>> It
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> seems
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> more
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> intuitive
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the current epoch,
> > > > > > >>>>>>> and
> > > > > > >>>>>>>>>> only
> > > > > > >>>>>>>>>>>> return
> > > > > > >>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>> bumped
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (50) "Implicit
> > > > > > >>>>>>>>>>>>>>>>> AddPartitionToTransaction"
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why does the
> > > > > > >>>>>>>> implicitly
> > > > > > >>>>>>>>>> sent
> > > > > > >>>>>>>>>>>>>>> request
> > > > > > >>>>>>>>>>>>>>>>> need
> > > > > > >>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> synchronous?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also says
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in case we need
> > > > > > >>>>>> to
> > > > > > >>>>>>>>> abort
> > > > > > >>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>> need to
> > > > > > >>>>>>>>>>>>>>>>>> know
> > > > > > >>>>>>>>>>>>>>>>>>>>> which
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What do you mean
> > > > > > >>>>>> by
> > > > > > >>>>>>>>> this?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we don’t want to
> > > > > > >>>>>>>> write
> > > > > > >>>>>>>>>> to it
> > > > > > >>>>>>>>>>>>>>> before
> > > > > > >>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>> store
> > > > > > >>>>>>>>>>>>>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> manager
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Do you mean
> > > > > > >>>>>>>>> TX-coordinator
> > > > > > >>>>>>>>>>>>>>> instead of
> > > > > > >>>>>>>>>>>>>>>>>>>>> "manager"?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (60)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For older clients
> > > > > > >>>>>>> and
> > > > > > >>>>>>>>>> ensuring
> > > > > > >>>>>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>> TX
> > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> ongoing,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> describe a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> race condition. I
> > > > > > >>>>>> am
> > > > > > >>>>>>>> not
> > > > > > >>>>>>>>>> sure
> > > > > > >>>>>>>>>>>> if I
> > > > > > >>>>>>>>>>>>>>>> can
> > > > > > >>>>>>>>>>>>>>>>>>>> follow
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> here.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> Can
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/18/22 1:21
> > > > > > >>>>>> PM,
> > > > > > >>>>>>>>>> Justine
> > > > > > >>>>>>>>>>>>>>> Olshan
> > > > > > >>>>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey all!
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to
> > > > > > >>>>>> start a
> > > > > > >>>>>>>>>>>> discussion
> > > > > > >>>>>>>>>>>>>>> on my
> > > > > > >>>>>>>>>>>>>>>>>>>> proposal
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> add
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> some
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checks on
> > > > > > >>>>>>>> transactions
> > > > > > >>>>>>>>> to
> > > > > > >>>>>>>>>>>> avoid
> > > > > > >>>>>>>>>>>>>>>>> hanging
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> transactions.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> been
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an issue for some
> > > > > > >>>>>>>> time,
> > > > > > >>>>>>>>>> so I
> > > > > > >>>>>>>>>>>>>>> really
> > > > > > >>>>>>>>>>>>>>>>> hope
> > > > > > >>>>>>>>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> KIP
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> helpful
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users of EOS.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The KIP includes
> > > > > > >>>>>>>>> changes
> > > > > > >>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>> will
> > > > > > >>>>>>>>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> compatible
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> old
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> changes to
> > > > > > >>>>>> improve
> > > > > > >>>>>>>>>>>> performance
> > > > > > >>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>> correctness
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> on
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Please take a
> > > > > > >>>>>> look
> > > > > > >>>>>>>> and
> > > > > > >>>>>>>>>> leave
> > > > > > >>>>>>>>>>>> any
> > > > > > >>>>>>>>>>>>>>>>>> comments
> > > > > > >>>>>>>>>>>>>>>>>>>> you
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> may
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> have!
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>
> > > > > > >>>>
> > > > > > >>
> > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> JIRA:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-14402
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>
> > > > > > >>>>
> > > > > > >>>
> > > > > > >>
> > > > > > >
> > > > >
> > > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Artem Livshits <al...@confluent.io.INVALID>.
>  I'd like to check that, under this KIP's circumstances, if we could ever
have "partial committed" txns

The goal of this KIP is to harden transaction implementation to never have
partially committed txns.

-Artem

On Fri, Jan 27, 2023 at 10:04 AM Guozhang Wang <gu...@gmail.com>
wrote:

> Thanks folks, I agree with all of your points.
>
> I think in the past we did have some cases where abortable txns are
> handled as fatal ones which we are fixing, but maybe there are some
> vice versa as well. I'd like to check that, under this KIP's
> circumstances, if we could ever have "partial committed" txns where
> some of the data are considered as committed while some others are
> aborted later. And if there were, could we recover from it. If the
> answer is "yes and yes", then I think we can bite the complexity
> bullet inside the client and leave users in comfort and peace :)
>
> Guozhang
>
> On Fri, Jan 27, 2023 at 9:42 AM Artem Livshits
> <al...@confluent.io.invalid> wrote:
> >
> > I agree.  I also think we should go a little further and be more
> > prescriptive with our recommendations for error handling (i.e. not just
> > think if there is a chance, but propose design patterns for typical
> > cases).  For example, if a transaction is aborted, the application must
> > either keep enough state to be able to redo the operation from the
> > beginning, or reset all state and fully recover from Kafka (i.e.
> re-consume
> > data from the last committed offsets).
> >
> > For failed commits, we need a way for the application to either learn the
> > result (if commit went through, then continue, otherwise redo operation)
> or
> > just reset all state and fully recover from Kafka (i.e. re-consume data
> > from the last committed offsets).
> >
> > -Artem
> >
> > On Fri, Jan 27, 2023 at 9:31 AM Justine Olshan
> <jo...@confluent.io.invalid>
> > wrote:
> >
> > > Thanks for clarifying Guozhang,
> > >
> > > I like the framing:
> > > * “if we did abort the txn and continue, could the app ever has a
> chance to
> > > recover and not have any side effects violating EOS guarantees”.*
> > >
> > > With this principle in mind, if we could convince ourselves that we
> won't
> > > have side effects, then we could move forward with a non-fatal error. I
> > > appreciate laying out the cases we've discussed. I think I also want to
> > > take a look in very close detail and convince myself of the right
> option.
> > > I'll get back to this thread with the final conclusion as well as
> update
> > > the KIP.
> > >
> > > Thanks for also bringing up KIP-691. I should take a look there as
> well. :)
> > >
> > > Justine
> > >
> > > On Thu, Jan 26, 2023 at 5:28 AM Guozhang Wang <
> guozhang.wang.us@gmail.com>
> > > wrote:
> > >
> > > > Hello Justine,
> > > >
> > > > Regarding which errors should be fatal v.s. abortable, I think the
> > > > principle would be “if we did abort the txn and continue, could the
> > > > app ever has a chance to recover and not have any side effects
> > > > violating EOS guarantees”.Of course we can discuss whether that’s a
> > > > good principle  e.g. one can argue that maybe it’s okay to violate
> EOS
> > > > some times rather than killing the whole app, whereas my rationale is
> > > > that since we want to make EOS the default config, it’s crucial that
> > > > we try to guarantee it as much as possible and whenever we detected
> if
> > > > there’s a chance that it maybe broken, we should let users know. And
> > > > if we just log an error in app and try to abort and continue, it’s
> > > > very likely that users would not be notified and probably only
> realize
> > > > that after a while, which may cause an even harder scenario to
> > > > debug.If people agree on that principle, we can go back and check
> this
> > > > KIP’s scenario: there are several scenarios causing a partition
> leader
> > > > detects an out of date epoch from a produce requests:
> > > >
> > > > * The old produce requests arrive late after a network partition, and
> > > > the old producer is already gone.
> > > > * The old produce requests arrive late after a network partition, but
> > > > the old producer is still around.
> > > > * A buggy producer client that did not follow the protocol.
> > > >
> > > > For the third case, we should try to let user know asap, and as clear
> > > > as possible, and hence it’s best to just stop the client app; for the
> > > > first case, it does not matter since the producer is already gone;
> > > > only for the second, probably the least likely case, we need to
> > > > ponder, and there my rationale again is that at that time, we may
> have
> > > > already violated the EOS guarantees since there are some partial txn
> > > > records that should be rejected while the txn itself has been
> > > > committed. In this case I think it’s better to let clients/users know
> > > > as soon as possible as well.
> > > >
> > > > Regarding the past scenarios where a fatal error killing the whole
> > > > apps, I believe part of that reason is that we were doing an
> > > > sub-optimal job on clients side handling various error cases and
> > > > that’s what KIP-691 is trying to resolve, and hence personally I
> would
> > > > suggest we do not weight in too much on that if we can trust that
> > > > KIP-691 will be successfully eliminate those not-necessary-hard-fail
> > > > scenarios.
> > > >
> > > >
> > > > Guozhang
> > > >
> > > > On Wed, Jan 25, 2023 at 5:51 PM Matthias J. Sax <mj...@apache.org>
> > > wrote:
> > > > >
> > > > > So the timestamp would be set when the write happens and thus no
> > > > > out-of-order data (base in time) can be introduced with
> "append_time"
> > > > > config even if a request sits in purgatory first while we check
> the TX
> > > > > status.
> > > > >
> > > > > That does make sense. Thanks for confirming, that there is no
> > > > > out-of-order issue for this case.
> > > > >
> > > > >
> > > > > -Matthias
> > > > >
> > > > > On 1/25/23 5:04 PM, Justine Olshan wrote:
> > > > > > Hey Matthias,
> > > > > > Let me put it this way, if a producer is checking if a
> transaction is
> > > > > > ongoing, then no writes to the partition from the producer will
> go
> > > > through
> > > > > > until the transaction is confirmed ongoing.
> > > > > >  From then, I think I can apply the writes in the order they
> came in.
> > > > Does
> > > > > > that make sense?
> > > > > >
> > > > > > Let me know if I'm missing something.
> > > > > > Justine
> > > > > >
> > > > > > On Wed, Jan 25, 2023 at 4:57 PM Matthias J. Sax <
> mjsax@apache.org>
> > > > wrote:
> > > > > >
> > > > > >>> would it build an offset map with just the latest timestamp
> for a
> > > > key?
> > > > > >>
> > > > > >> Cannot remember the details without reading the KIP, but yes,
> > > > something
> > > > > >> like this (I believe it actually needs to track both, offset and
> > > > > >> timestamp per key).
> > > > > >>
> > > > > >>> I wonder if ordering assumptions are baked in there, why not
> use
> > > > > >> offset-based compaction.
> > > > > >>
> > > > > >> The use case is a compacted topic that does contain out-of-order
> > > data.
> > > > > >> If you set key k1=v1 @ 5 offset 100 and later key1 = v0 @ 3 at
> > > offset
> > > > > >> 200 we want to cleanup v0 with higher offset because it's
> > > out-of-order
> > > > > >> based on time, but keep v1 what is the actual latest version of
> k1.
> > > > > >>
> > > > > >>
> > > > > >>> I was also not aware of this "guarantee" with regards to broker
> > > side
> > > > > >> time.
> > > > > >>
> > > > > >> As already said: I am not sure if it's a public contract, but
> based
> > > on
> > > > > >> my experience, people might reply on it as "implicit contract".
> --
> > > > Maybe
> > > > > >> somebody else knows if it's public or not, and if it would be
> ok to
> > > > > >> "break" it.
> > > > > >>
> > > > > >>> Let me know if you have any concerns here.
> > > > > >>
> > > > > >> My understanding is: While we cannot make an offset-order
> guarantee
> > > > for
> > > > > >> interleaved writes of different producer, if the topic is
> configures
> > > > > >> with "append_time", we "guarantee" (cf. my comment above")
> timestamp
> > > > > >> order... If that's the case, it would be an issue if we break
> this
> > > > > >> "guarantee".
> > > > > >>
> > > > > >> I am not sure when the broker sets the timestamp for
> "append_time"
> > > > > >> config? If we do it before putting the request into purgatory,
> we
> > > > have a
> > > > > >> problem. However, if we set the timestamp when we actually
> process
> > > the
> > > > > >> request and do the actual append, it seems there is no issue,
> as the
> > > > > >> request that was waiting in purgatory get the "newest"
> timestamp and
> > > > > >> thus cannot introduce out-of-order data.
> > > > > >>
> > > > > >>
> > > > > >> -Matthias
> > > > > >>
> > > > > >>
> > > > > >> On 1/24/23 10:44 AM, Justine Olshan wrote:
> > > > > >>> Hey Matthias,
> > > > > >>>
> > > > > >>> I have actually never heard of KIP-280 so thanks for bringing
> it
> > > up.
> > > > That
> > > > > >>> seems interesting. I wonder how it would work though -- would
> it
> > > > build an
> > > > > >>> offset map with just the latest timestamp for a key? I wonder
> if
> > > > ordering
> > > > > >>> assumptions are baked in there, why not use offset-based
> > > compaction.
> > > > > >>>
> > > > > >>> I was also not aware of this "guarantee" with regards to broker
> > > side
> > > > > >> time.
> > > > > >>> I think that we can do in order handling for a given producer,
> but
> > > > not
> > > > > >>> across all producers. However, we can't guarantee that anyway.
> > > > > >>>
> > > > > >>> Let me know if you have any concerns here.
> > > > > >>>
> > > > > >>> Thanks,
> > > > > >>> Justine
> > > > > >>>
> > > > > >>> On Mon, Jan 23, 2023 at 6:33 PM Matthias J. Sax <
> mjsax@apache.org>
> > > > > >> wrote:
> > > > > >>>
> > > > > >>>> Just a side note about Guozhang comments about timestamps.
> > > > > >>>>
> > > > > >>>> If the producer sets the timestamp, putting the record into
> > > > purgatory
> > > > > >>>> seems not to be an issue (as already said: for this case we
> don't
> > > > > >>>> guarantee timestamp order between writes of different
> producers
> > > > anyway).
> > > > > >>>> However, if the broker sets the timestamp, the expectation is
> that
> > > > there
> > > > > >>>> is no out-of-order data in the partition ever; if we would
> > > introduce
> > > > > >>>> out-of-order data for this case (for interleaved writes of
> > > different
> > > > > >>>> producers), it seems we would violate the current contract?
> (To be
> > > > fair:
> > > > > >>>> I don't know if that's an official contract, but I assume
> people
> > > > rely on
> > > > > >>>> this behavior -- and it "advertised" in many public talks...)
> > > > > >>>>
> > > > > >>>> About compaction: there is actually KIP-280 that adds
> timestamp
> > > > based
> > > > > >>>> compaction what is a very useful feature for Kafka Streams
> with
> > > > regard
> > > > > >>>> to out-of-order data handling. So the impact if we introduce
> > > > > >>>> out-of-order data could be larger scoped.
> > > > > >>>>
> > > > > >>>>
> > > > > >>>> -Matthias
> > > > > >>>>
> > > > > >>>>
> > > > > >>>> On 1/20/23 4:48 PM, Justine Olshan wrote:
> > > > > >>>>> Hey Artem,
> > > > > >>>>>
> > > > > >>>>> I see there is a check for transactional producers. I'm
> wondering
> > > > if we
> > > > > >>>>> don't handle the epoch overflow case. I'm also not sure it
> will
> > > be
> > > > a
> > > > > >> huge
> > > > > >>>>> issue to extend to transactional producers, but maybe I'm
> missing
> > > > > >>>> something.
> > > > > >>>>>
> > > > > >>>>> As for the recovery path -- I think Guozhang's point was if
> we
> > > > have a
> > > > > >> bad
> > > > > >>>>> client that repeatedly tries to produce without adding to the
> > > > > >> transaction
> > > > > >>>>> we would do the following:
> > > > > >>>>> a) if not fatal, we just fail the produce request over and
> over
> > > > > >>>>> b) if fatal, we fence the producer
> > > > > >>>>>
> > > > > >>>>> Here with B, the issue with the client would be made clear
> more
> > > > > >> quickly.
> > > > > >>>> I
> > > > > >>>>> suppose there are some intermediate cases where the issue
> only
> > > > occurs
> > > > > >>>>> sometimes, but I wonder if we should consider how to recover
> with
> > > > > >> clients
> > > > > >>>>> who don't behave as expected anyway.
> > > > > >>>>>
> > > > > >>>>> I think there is a place for the abortable error that we are
> > > > adding --
> > > > > >>>> just
> > > > > >>>>> abort and try again. But I think there are also some cases
> where
> > > > trying
> > > > > >>>> to
> > > > > >>>>> recover overcomplicates some logic. Especially if we are
> > > > considering
> > > > > >>>> older
> > > > > >>>>> clients -- there I'm not sure if there's a ton we can do
> besides
> > > > fail
> > > > > >> the
> > > > > >>>>> batch or fence the producer. With newer clients, we can
> consider
> > > > more
> > > > > >>>>> options for what can just be recovered after aborting. But
> epochs
> > > > might
> > > > > >>>> be
> > > > > >>>>> a hard one unless we also want to reset producer ID.
> > > > > >>>>>
> > > > > >>>>> Thanks,
> > > > > >>>>> Justine
> > > > > >>>>>
> > > > > >>>>>
> > > > > >>>>>
> > > > > >>>>> On Fri, Jan 20, 2023 at 3:59 PM Artem Livshits
> > > > > >>>>> <al...@confluent.io.invalid> wrote:
> > > > > >>>>>
> > > > > >>>>>>>     besides the poorly written client case
> > > > > >>>>>>
> > > > > >>>>>> A poorly written client could create a lot of grief to
> people
> > > who
> > > > run
> > > > > >>>> Kafka
> > > > > >>>>>> brokers :-), so when deciding to make an error fatal I
> would see
> > > > if
> > > > > >>>> there
> > > > > >>>>>> is a reasonable recovery path rather than how often it could
> > > > happen.
> > > > > >>>> If we
> > > > > >>>>>> have solid implementation of transactions (which I hope
> we'll do
> > > > as a
> > > > > >>>>>> result of this KIP), it would help to recover from a large
> class
> > > > of
> > > > > >>>> errors
> > > > > >>>>>> by just aborting a transaction, even if the cause of error
> is a
> > > > race
> > > > > >>>>>> condition or etc.
> > > > > >>>>>>
> > > > > >>>>>> -Artem
> > > > > >>>>>>
> > > > > >>>>>> On Fri, Jan 20, 2023 at 3:26 PM Justine Olshan
> > > > > >>>>>> <jo...@confluent.io.invalid>
> > > > > >>>>>> wrote:
> > > > > >>>>>>
> > > > > >>>>>>> Artem --
> > > > > >>>>>>> I guess the discussion path we were going down is when we
> > > expect
> > > > to
> > > > > >> see
> > > > > >>>>>>> this error. I mentioned that it was hard to come up with
> cases
> > > > for
> > > > > >> when
> > > > > >>>>>> the
> > > > > >>>>>>> producer would still be around to receive the error
> besides the
> > > > > >> poorly
> > > > > >>>>>>> written client case.
> > > > > >>>>>>> If we don't expect to have a producer to receive the
> response,
> > > it
> > > > > >> sort
> > > > > >>>> of
> > > > > >>>>>>> makes sense for it to be fatal.
> > > > > >>>>>>>
> > > > > >>>>>>> I had some discussion with Jason offline about the epoch
> being
> > > > off
> > > > > >>>> cases
> > > > > >>>>>>> and I'm not sure we could find a ton (outside of produce
> > > > requests)
> > > > > >>>> where
> > > > > >>>>>> we
> > > > > >>>>>>> could/should recover. I'd be happy to hear some examples
> > > though,
> > > > > >> maybe
> > > > > >>>>>> I'm
> > > > > >>>>>>> missing something.
> > > > > >>>>>>>
> > > > > >>>>>>> Thanks,
> > > > > >>>>>>> Justine
> > > > > >>>>>>>
> > > > > >>>>>>> On Fri, Jan 20, 2023 at 3:19 PM Artem Livshits
> > > > > >>>>>>> <al...@confluent.io.invalid> wrote:
> > > > > >>>>>>>
> > > > > >>>>>>>> In general, I'd like to avoid fatal errors as much as
> > > possible,
> > > > in
> > > > > >>>> some
> > > > > >>>>>>>> sense fatal errors just push out recovery logic to the
> > > > application
> > > > > >>>>>> which
> > > > > >>>>>>>> either complicates the application or leads to disruption
> > > (we've
> > > > > >> seen
> > > > > >>>>>>> cases
> > > > > >>>>>>>> when a transient broker error could lead to work stoppage
> when
> > > > > >>>>>>> applications
> > > > > >>>>>>>> need to be manually restarted).  I think we should strive
> to
> > > > define
> > > > > >>>>>>>> recovery logic for most errors (and/or encapsulate it in
> the
> > > > Kafka
> > > > > >>>>>> client
> > > > > >>>>>>>> as much as possible).
> > > > > >>>>>>>>
> > > > > >>>>>>>> One benefit of transactions is that they simplify recovery
> > > from
> > > > > >>>> errors,
> > > > > >>>>>>>> pretty much any error (that's not handled transparently by
> > > > retries
> > > > > >> in
> > > > > >>>>>>> Kafka
> > > > > >>>>>>>> client) can be handled by the application via aborting the
> > > > > >> transaction
> > > > > >>>>>>> and
> > > > > >>>>>>>> repeating the transactional logic again.  One tricky
> error is
> > > an
> > > > > >> error
> > > > > >>>>>>>> during commit, because we don't know the outcome.  For
> commit
> > > > > >> errors,
> > > > > >>>>>> the
> > > > > >>>>>>>> recommendation should be to retry the commit until it
> returns
> > > > the
> > > > > >>>>>>> specific
> > > > > >>>>>>>> result (committed or aborted).
> > > > > >>>>>>>>
> > > > > >>>>>>>> -Artem
> > > > > >>>>>>>>
> > > > > >>>>>>>> On Fri, Jan 20, 2023 at 2:52 PM Justine Olshan
> > > > > >>>>>>>> <jo...@confluent.io.invalid>
> > > > > >>>>>>>> wrote:
> > > > > >>>>>>>>
> > > > > >>>>>>>>> That's a fair point about other clients.
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> I think the abortable error case is interesting because
> I'm
> > > > curious
> > > > > >>>>>> how
> > > > > >>>>>>>>> other clients would handle this. I assume they would
> need to
> > > > > >>>>>> implement
> > > > > >>>>>>>>> handling for the error code unless they did something
> like
> > > "any
> > > > > >>>>>> unknown
> > > > > >>>>>>>>> error codes/any codes that aren't x,y,z are retriable." I
> > > would
> > > > > >> hope
> > > > > >>>>>>> that
> > > > > >>>>>>>>> unknown error codes were fatal, and if the code was
> > > > implemented it
> > > > > >>>>>>> would
> > > > > >>>>>>>>> abort the transaction. But I will think on this too.
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> As for InvalidRecord -- you mentioned it was not fatal,
> but
> > > I'm
> > > > > >>>>>> taking
> > > > > >>>>>>> a
> > > > > >>>>>>>>> look through the code. We would see this on handling the
> > > > produce
> > > > > >>>>>>>> response.
> > > > > >>>>>>>>> If I recall correctly, we check if errors are retriable.
> I
> > > > think
> > > > > >> this
> > > > > >>>>>>>> error
> > > > > >>>>>>>>> would not be retriable. But I guess the concern here is
> that
> > > > it is
> > > > > >>>>>> not
> > > > > >>>>>>>>> enough for just that batch to fail. I guess I hadn't
> > > considered
> > > > > >> fully
> > > > > >>>>>>>>> fencing the old producer but there are valid arguments
> here
> > > > why we
> > > > > >>>>>>> would
> > > > > >>>>>>>>> want to.
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> Thanks,
> > > > > >>>>>>>>> Justine
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> On Fri, Jan 20, 2023 at 2:35 PM Guozhang Wang <
> > > > > >>>>>>>> guozhang.wang.us@gmail.com>
> > > > > >>>>>>>>> wrote:
> > > > > >>>>>>>>>
> > > > > >>>>>>>>>> Thanks Justine for the replies! I agree with most of
> your
> > > > > >> thoughts.
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> Just for 3/7), though I agree for our own AK producer,
> since
> > > > we do
> > > > > >>>>>>>>>> "nextRequest(boolean hasIncompleteBatches)", we
> guarantee
> > > the
> > > > > >>>>>> end-txn
> > > > > >>>>>>>>>> would not be sent until we've effectively flushed, but
> I was
> > > > > >>>>>>> referring
> > > > > >>>>>>>>>> to any future bugs or other buggy clients that the same
> > > > client may
> > > > > >>>>>>> get
> > > > > >>>>>>>>>> into this situation, in which case we should give the
> > > client a
> > > > > >>>>>> clear
> > > > > >>>>>>>>>> msg that "you did something wrong, and hence now you
> should
> > > > > >> fatally
> > > > > >>>>>>>>>> close yourself". What I'm concerned about is that, by
> seeing
> > > > an
> > > > > >>>>>>>>>> "abortable error" or in some rare cases an "invalid
> record",
> > > > the
> > > > > >>>>>>>>>> client could not realize "something that's really bad
> > > > happened".
> > > > > >> So
> > > > > >>>>>>>>>> it's not about adding a new error, it's mainly about
> those
> > > > real
> > > > > >>>>>> buggy
> > > > > >>>>>>>>>> situations causing such "should never happen" cases, the
> > > > errors
> > > > > >>>>>>> return
> > > > > >>>>>>>>>> would not be informative enough.
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> Thinking in other ways, if we believe that for most
> cases
> > > such
> > > > > >>>>>> error
> > > > > >>>>>>>>>> codes would not reach the original clients since they
> would
> > > be
> > > > > >>>>>>>>>> disconnected or even gone by that time, and only in some
> > > rare
> > > > > >> cases
> > > > > >>>>>>>>>> they would still be seen by the sending clients, then
> why
> > > not
> > > > make
> > > > > >>>>>>>>>> them more fatal and more specific than generic.
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> Guozhang
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> On Fri, Jan 20, 2023 at 1:59 PM Justine Olshan
> > > > > >>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> Hey Guozhang. Thanks for taking a look and for the
> detailed
> > > > > >>>>>>> comments!
> > > > > >>>>>>>>>> I'll
> > > > > >>>>>>>>>>> do my best to address below.
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> 1. I see what you are saying here, but I think I need
> to
> > > look
> > > > > >>>>>>> through
> > > > > >>>>>>>>> the
> > > > > >>>>>>>>>>> sequence of events you mention. Typically we've seen
> this
> > > > issue
> > > > > >>>>>> in
> > > > > >>>>>>> a
> > > > > >>>>>>>>> few
> > > > > >>>>>>>>>>> cases.
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>>     One is when we have a producer disconnect when
> trying
> > > to
> > > > > >>>>>> produce.
> > > > > >>>>>>>>>>> Typically in these cases, we abort the transaction.
> We've
> > > > seen
> > > > > >>>>>> that
> > > > > >>>>>>>>> after
> > > > > >>>>>>>>>>> the markers are written, the disconnection can
> sometimes
> > > > cause
> > > > > >>>>>> the
> > > > > >>>>>>>>>> request
> > > > > >>>>>>>>>>> to get flushed to the broker. In this case, we don't
> need
> > > > client
> > > > > >>>>>>>>> handling
> > > > > >>>>>>>>>>> because the producer we are responding to is gone. We
> just
> > > > needed
> > > > > >>>>>>> to
> > > > > >>>>>>>>> make
> > > > > >>>>>>>>>>> sure we didn't write to the log on the broker side. I'm
> > > > trying to
> > > > > >>>>>>>> think
> > > > > >>>>>>>>>> of
> > > > > >>>>>>>>>>> a case where we do have the client to return to. I'd
> think
> > > > the
> > > > > >>>>>> same
> > > > > >>>>>>>>>> client
> > > > > >>>>>>>>>>> couldn't progress to committing the transaction unless
> the
> > > > > >>>>>> produce
> > > > > >>>>>>>>>> request
> > > > > >>>>>>>>>>> returned right? Of course, there is the incorrectly
> written
> > > > > >>>>>> clients
> > > > > >>>>>>>>> case.
> > > > > >>>>>>>>>>> I'll think on this a bit more and let you know if I
> come up
> > > > with
> > > > > >>>>>>>>> another
> > > > > >>>>>>>>>>> scenario when we would return to an active client when
> the
> > > > > >>>>>>>> transaction
> > > > > >>>>>>>>> is
> > > > > >>>>>>>>>>> no longer ongoing.
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> I was not aware that we checked the result of a send
> after
> > > we
> > > > > >>>>>>> commit
> > > > > >>>>>>>>>>> though. I'll need to look into that a bit more.
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> 2. There were some questions about this in the
> discussion.
> > > > The
> > > > > >>>>>> plan
> > > > > >>>>>>>> is
> > > > > >>>>>>>>> to
> > > > > >>>>>>>>>>> handle overflow with the mechanism we currently have
> in the
> > > > > >>>>>>> producer.
> > > > > >>>>>>>>> If
> > > > > >>>>>>>>>> we
> > > > > >>>>>>>>>>> try to bump and the epoch will overflow, we actually
> > > > allocate a
> > > > > >>>>>> new
> > > > > >>>>>>>>>>> producer ID. I need to confirm the fencing logic on the
> > > last
> > > > > >>>>>> epoch
> > > > > >>>>>>>> (ie,
> > > > > >>>>>>>>>> we
> > > > > >>>>>>>>>>> probably shouldn't allow any records to be produced
> with
> > > the
> > > > > >>>>>> final
> > > > > >>>>>>>>> epoch
> > > > > >>>>>>>>>>> since we can never properly fence that one).
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> 3. I can agree with you that the current error
> handling is
> > > > > >>>>>> messy. I
> > > > > >>>>>>>>>> recall
> > > > > >>>>>>>>>>> taking a look at your KIP a while back, but I think I
> > > mostly
> > > > saw
> > > > > >>>>>>> the
> > > > > >>>>>>>>>>> section about how the errors were wrapped. Maybe I
> need to
> > > > take
> > > > > >>>>>>>> another
> > > > > >>>>>>>>>>> look. As for abortable error, the idea was that the
> > > handling
> > > > > >>>>>> would
> > > > > >>>>>>> be
> > > > > >>>>>>>>>>> simple -- if this error is seen, the transaction
> should be
> > > > > >>>>>> aborted
> > > > > >>>>>>> --
> > > > > >>>>>>>>> no
> > > > > >>>>>>>>>>> other logic about previous state or requests
> necessary. Is
> > > > your
> > > > > >>>>>>>> concern
> > > > > >>>>>>>>>>> simply about adding new errors? We were hoping to have
> an
> > > > error
> > > > > >>>>>>> that
> > > > > >>>>>>>>>> would
> > > > > >>>>>>>>>>> have one meaning and many of the current errors have a
> > > > history of
> > > > > >>>>>>>>> meaning
> > > > > >>>>>>>>>>> different things on different client versions. That
> was the
> > > > main
> > > > > >>>>>>>>>> motivation
> > > > > >>>>>>>>>>> for adding a new error.
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> 4. This is a good point about record timestamp
> reordering.
> > > > > >>>>>>> Timestamps
> > > > > >>>>>>>>>> don't
> > > > > >>>>>>>>>>> affect compaction, but they do affect retention
> deletion.
> > > For
> > > > > >>>>>> that,
> > > > > >>>>>>>>> kafka
> > > > > >>>>>>>>>>> considers the largest timestamp in the segment, so I
> think
> > > a
> > > > > >>>>>> small
> > > > > >>>>>>>>> amount
> > > > > >>>>>>>>>>> of reordering (hopefully on the order of milliseconds
> or
> > > even
> > > > > >>>>>>>> seconds)
> > > > > >>>>>>>>>> will
> > > > > >>>>>>>>>>> be ok. We take timestamps from clients so there is
> already
> > > a
> > > > > >>>>>>>>> possibility
> > > > > >>>>>>>>>>> for some drift and non-monotonically increasing
> timestamps.
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> 5. Thanks for catching. The error is there, but it's
> > > actually
> > > > > >>>>>> that
> > > > > >>>>>>>>> those
> > > > > >>>>>>>>>>> fields should be 4+! Due to how the message generator
> > > works,
> > > > I
> > > > > >>>>>>>> actually
> > > > > >>>>>>>>>>> have to redefine those fields inside the
> > > > > >>>>>>>>> `"AddPartitionsToTxnTransaction`
> > > > > >>>>>>>>>>> block for it to build correctly. I'll fix it to be
> correct.
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> 6. Correct -- we will only add the request to
> purgatory if
> > > > the
> > > > > >>>>>>> cache
> > > > > >>>>>>>>> has
> > > > > >>>>>>>>>> no
> > > > > >>>>>>>>>>> ongoing transaction. I can change the wording to make
> that
> > > > > >>>>>> clearer
> > > > > >>>>>>>> that
> > > > > >>>>>>>>>> we
> > > > > >>>>>>>>>>> only place the request in purgatory if we need to
> contact
> > > the
> > > > > >>>>>>>>> transaction
> > > > > >>>>>>>>>>> coordinator.
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> 7. We did take a look at some of the errors and it was
> hard
> > > > to
> > > > > >>>>>> come
> > > > > >>>>>>>> up
> > > > > >>>>>>>>>> with
> > > > > >>>>>>>>>>> a good one. I agree that InvalidTxnStateException is
> ideal
> > > > except
> > > > > >>>>>>> for
> > > > > >>>>>>>>> the
> > > > > >>>>>>>>>>> fact that it hasn't been returned on Produce requests
> > > > before. The
> > > > > >>>>>>>> error
> > > > > >>>>>>>>>>> handling for clients is a bit vague (which is why I
> opened
> > > > > >>>>>>>> KAFKA-14439
> > > > > >>>>>>>>>>> <https://issues.apache.org/jira/browse/KAFKA-14439>),
> but
> > > > the
> > > > > >>>>>>>> decision
> > > > > >>>>>>>>>> we
> > > > > >>>>>>>>>>> made here was to only return errors that have been
> > > previously
> > > > > >>>>>>>> returned
> > > > > >>>>>>>>> to
> > > > > >>>>>>>>>>> producers. As for not being fatal, I think part of the
> > > > theory was
> > > > > >>>>>>>> that
> > > > > >>>>>>>>> in
> > > > > >>>>>>>>>>> many cases, the producer would be disconnected. (See
> point
> > > > 1) and
> > > > > >>>>>>>> this
> > > > > >>>>>>>>>>> would just be an error to return from the server. I did
> > > plan
> > > > to
> > > > > >>>>>>> think
> > > > > >>>>>>>>>> about
> > > > > >>>>>>>>>>> other cases, so let me know if you think of any as
> well!
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> Lots to say! Let me know if you have further thoughts!
> > > > > >>>>>>>>>>> Justine
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang <
> > > > > >>>>>>>>>> guozhang.wang.us@gmail.com>
> > > > > >>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>>> Hello Justine,
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> Thanks for the great write-up! I made a quick pass
> through
> > > > it
> > > > > >>>>>> and
> > > > > >>>>>>>>> here
> > > > > >>>>>>>>>>>> are some thoughts (I have not been able to read
> through
> > > this
> > > > > >>>>>>> thread
> > > > > >>>>>>>>> so
> > > > > >>>>>>>>>>>> pardon me if they have overlapped or subsumed by
> previous
> > > > > >>>>>>>> comments):
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> First are some meta ones:
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> 1. I think we need to also improve the client's
> experience
> > > > once
> > > > > >>>>>>> we
> > > > > >>>>>>>>>>>> have this defence in place. More concretely, say a
> user's
> > > > > >>>>>>> producer
> > > > > >>>>>>>>>>>> code is like following:
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> future = producer.send();
> > > > > >>>>>>>>>>>> // producer.flush();
> > > > > >>>>>>>>>>>> producer.commitTransaction();
> > > > > >>>>>>>>>>>> future.get();
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> Which resulted in the order of a) produce-request
> sent by
> > > > > >>>>>>> producer,
> > > > > >>>>>>>>> b)
> > > > > >>>>>>>>>>>> end-txn-request sent by producer, c) end-txn-response
> sent
> > > > > >>>>>> back,
> > > > > >>>>>>> d)
> > > > > >>>>>>>>>>>> txn-marker-request sent from coordinator to partition
> > > > leader,
> > > > > >>>>>> e)
> > > > > >>>>>>>>>>>> produce-request finally received by the partition
> leader,
> > > > > >>>>>> before
> > > > > >>>>>>>> this
> > > > > >>>>>>>>>>>> KIP e) step would be accepted causing a dangling txn;
> now
> > > it
> > > > > >>>>>>> would
> > > > > >>>>>>>> be
> > > > > >>>>>>>>>>>> rejected in step e) which is good. But from the
> client's
> > > > point
> > > > > >>>>>> of
> > > > > >>>>>>>>> view
> > > > > >>>>>>>>>>>> now it becomes confusing since the
> `commitTransaction()`
> > > > > >>>>>> returns
> > > > > >>>>>>>>>>>> successfully, but the "future" throws an invalid-epoch
> > > > error,
> > > > > >>>>>> and
> > > > > >>>>>>>>> they
> > > > > >>>>>>>>>>>> are not sure if the transaction did succeed or not. In
> > > > fact, it
> > > > > >>>>>>>>>>>> "partially succeeded" with some msgs being rejected
> but
> > > > others
> > > > > >>>>>>>>>>>> committed successfully.
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> Of course the easy way to avoid this is, always call
> > > > > >>>>>>>>>>>> "producer.flush()" before commitTxn and that's what
> we do
> > > > > >>>>>>>> ourselves,
> > > > > >>>>>>>>>>>> and what we recommend users do. But I suspect not
> everyone
> > > > does
> > > > > >>>>>>> it.
> > > > > >>>>>>>>> In
> > > > > >>>>>>>>>>>> fact I just checked the javadoc in KafkaProducer and
> our
> > > > code
> > > > > >>>>>>>> snippet
> > > > > >>>>>>>>>>>> does not include a `flush()` call. So I'm thinking
> maybe
> > > we
> > > > can
> > > > > >>>>>>> in
> > > > > >>>>>>>>>>>> side the `commitTxn` code to enforce flushing before
> > > sending
> > > > > >>>>>> the
> > > > > >>>>>>>>>>>> end-txn request.
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> 2. I'd like to clarify a bit details on "just add
> > > > partitions to
> > > > > >>>>>>> the
> > > > > >>>>>>>>>>>> transaction on the first produce request during a
> > > > transaction".
> > > > > >>>>>>> My
> > > > > >>>>>>>>>>>> understanding is that the partition leader's cache
> has the
> > > > > >>>>>>> producer
> > > > > >>>>>>>>> id
> > > > > >>>>>>>>>>>> / sequence / epoch for the latest txn, either
> on-going or
> > > is
> > > > > >>>>>>>>> completed
> > > > > >>>>>>>>>>>> (upon receiving the marker request from coordinator).
> > > When a
> > > > > >>>>>>>> produce
> > > > > >>>>>>>>>>>> request is received, if
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> * producer's epoch < cached epoch, or producer's
> epoch ==
> > > > > >>>>>> cached
> > > > > >>>>>>>>> epoch
> > > > > >>>>>>>>>>>> but the latest txn is completed, leader directly
> reject
> > > with
> > > > > >>>>>>>>>>>> invalid-epoch.
> > > > > >>>>>>>>>>>> * producer's epoch > cached epoch, park the the
> request
> > > and
> > > > > >>>>>> send
> > > > > >>>>>>>>>>>> add-partitions request to coordinator.
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> In order to do it, does the coordinator need to bump
> the
> > > > > >>>>>> sequence
> > > > > >>>>>>>> and
> > > > > >>>>>>>>>>>> reset epoch to 0 when the next epoch is going to
> overflow?
> > > > If
> > > > > >>>>>> no
> > > > > >>>>>>>> need
> > > > > >>>>>>>>>>>> to do so, then how we handle the (admittedly rare, but
> > > still
> > > > > >>>>>> may
> > > > > >>>>>>>>>>>> happen) epoch overflow situation?
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> 3. I'm a bit concerned about adding a generic
> > > > "ABORTABLE_ERROR"
> > > > > >>>>>>>> given
> > > > > >>>>>>>>>>>> we already have a pretty messy error classification
> and
> > > > error
> > > > > >>>>>>>>> handling
> > > > > >>>>>>>>>>>> on the producer clients side --- I have a summary
> about
> > > the
> > > > > >>>>>>> issues
> > > > > >>>>>>>>> and
> > > > > >>>>>>>>>>>> a proposal to address this in
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>
> > > > > >>>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>
> > > > > >>>>
> > > > > >>
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
> > > > > >>>>>>>>>>>> -- I understand we do not want to use
> > > "UNKNOWN_PRODUCER_ID"
> > > > > >>>>>>> anymore
> > > > > >>>>>>>>>>>> and in fact we intend to deprecate it in KIP-360 and
> > > > eventually
> > > > > >>>>>>>>> remove
> > > > > >>>>>>>>>>>> it; but I'm wondering can we still use specific error
> > > codes.
> > > > > >>>>>> E.g.
> > > > > >>>>>>>>> what
> > > > > >>>>>>>>>>>> about "InvalidProducerEpochException" since for new
> > > clients,
> > > > > >>>>>> the
> > > > > >>>>>>>>>>>> actual reason this would actually be rejected is
> indeed
> > > > because
> > > > > >>>>>>> the
> > > > > >>>>>>>>>>>> epoch on the coordinator caused the
> add-partitions-request
> > > > from
> > > > > >>>>>>> the
> > > > > >>>>>>>>>>>> brokers to be rejected anyways?
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> 4. It seems we put the producer request into purgatory
> > > > before
> > > > > >>>>>> we
> > > > > >>>>>>>> ever
> > > > > >>>>>>>>>>>> append the records, while other producer's records may
> > > > still be
> > > > > >>>>>>>>>>>> appended during the time; and that potentially may
> result
> > > in
> > > > > >>>>>> some
> > > > > >>>>>>>>>>>> re-ordering compared with reception order. I'm not
> super
> > > > > >>>>>>> concerned
> > > > > >>>>>>>>>>>> about it since Kafka does not guarantee reception
> ordering
> > > > > >>>>>> across
> > > > > >>>>>>>>>>>> producers anyways, but it may make the timestamps of
> > > records
> > > > > >>>>>>>> inside a
> > > > > >>>>>>>>>>>> partition to be more out-of-ordered. Are we aware of
> any
> > > > > >>>>>>> scenarios
> > > > > >>>>>>>>>>>> such as future enhancements on log compactions that
> may be
> > > > > >>>>>>> affected
> > > > > >>>>>>>>> by
> > > > > >>>>>>>>>>>> this effect?
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> Below are just minor comments:
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> 5. In "AddPartitionsToTxnTransaction" field of
> > > > > >>>>>>>>>>>> "AddPartitionsToTxnRequest" RPC, the versions of those
> > > inner
> > > > > >>>>>>> fields
> > > > > >>>>>>>>>>>> are "0-3" while I thought they should be "0+" still?
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> 6. Regarding "we can place the request in a purgatory
> of
> > > > sorts
> > > > > >>>>>>> and
> > > > > >>>>>>>>>>>> check if there is any state for the transaction on the
> > > > > >>>>>> broker": i
> > > > > >>>>>>>>>>>> think at this time when we just do the checks against
> the
> > > > > >>>>>> cached
> > > > > >>>>>>>>>>>> state, we do not need to put the request to purgatory
> yet?
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> 7. This is related to 3) above. I feel using
> > > > > >>>>>>>> "InvalidRecordException"
> > > > > >>>>>>>>>>>> for older clients may also be a bit confusing, and
> also it
> > > > is
> > > > > >>>>>> not
> > > > > >>>>>>>>>>>> fatal -- for old clients, it better to be fatal since
> this
> > > > > >>>>>>>> indicates
> > > > > >>>>>>>>>>>> the clients is doing something wrong and hence it
> should
> > > be
> > > > > >>>>>>> closed.
> > > > > >>>>>>>>>>>> And in general I'd prefer to use slightly more
> specific
> > > > meaning
> > > > > >>>>>>>> error
> > > > > >>>>>>>>>>>> codes for clients. That being said, I also feel
> > > > > >>>>>>>>>>>> "InvalidProducerEpochException" is not suitable for
> old
> > > > > >>>>>> versioned
> > > > > >>>>>>>>>>>> clients, and we'd have to pick one that old clients
> > > > recognize.
> > > > > >>>>>>> I'd
> > > > > >>>>>>>>>>>> prefer "InvalidTxnStateException" but that one is
> supposed
> > > > to
> > > > > >>>>>> be
> > > > > >>>>>>>>>>>> returned from txn coordinators only today. I'd
> suggest we
> > > > do a
> > > > > >>>>>>>> quick
> > > > > >>>>>>>>>>>> check in the current client's code path and see if
> that
> > > one
> > > > > >>>>>> would
> > > > > >>>>>>>> be
> > > > > >>>>>>>>>>>> handled if it's from a produce-response, and if yes,
> use
> > > > this
> > > > > >>>>>>> one;
> > > > > >>>>>>>>>>>> otherwise, use "ProducerFencedException" which is much
> > > less
> > > > > >>>>>>>>> meaningful
> > > > > >>>>>>>>>>>> but it's still a fatal error.
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> Thanks,
> > > > > >>>>>>>>>>>> Guozhang
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
> > > > > >>>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> Yeah -- looks like we already have code to handle
> bumping
> > > > the
> > > > > >>>>>>>> epoch
> > > > > >>>>>>>>>> and
> > > > > >>>>>>>>>>>>> when the epoch is Short.MAX_VALUE, we get a new
> producer
> > > > ID.
> > > > > >>>>>>>> Since
> > > > > >>>>>>>>>> this
> > > > > >>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>> already the behavior, do we want to change it
> further?
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> Justine
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <
> > > > > >>>>>>>>> jolshan@confluent.io
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> Hey all, just wanted to quickly update and say I've
> > > > > >>>>>> modified
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>> KIP to
> > > > > >>>>>>>>>>>>>> explicitly mention that AddOffsetCommitsToTxnRequest
> > > will
> > > > > >>>>>> be
> > > > > >>>>>>>>>> replaced
> > > > > >>>>>>>>>>>> by
> > > > > >>>>>>>>>>>>>> a coordinator-side (inter-broker) AddPartitionsToTxn
> > > > > >>>>>> implicit
> > > > > >>>>>>>>>> request.
> > > > > >>>>>>>>>>>> This
> > > > > >>>>>>>>>>>>>> mirrors the user partitions and will implicitly add
> > > offset
> > > > > >>>>>>>>>> partitions
> > > > > >>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>> transactions when we commit offsets on them. We will
> > > > > >>>>>>> deprecate
> > > > > >>>>>>>>>>>> AddOffsetCommitsToTxnRequest
> > > > > >>>>>>>>>>>>>> for new clients.
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> Also to address Artem's comments --
> > > > > >>>>>>>>>>>>>> I'm a bit unsure if the changes here will change the
> > > > > >>>>>> previous
> > > > > >>>>>>>>>> behavior
> > > > > >>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>> fencing producers. In the case you mention in the
> first
> > > > > >>>>>>>>> paragraph,
> > > > > >>>>>>>>>> are
> > > > > >>>>>>>>>>>> you
> > > > > >>>>>>>>>>>>>> saying we bump the epoch before we try to abort the
> > > > > >>>>>>>> transaction?
> > > > > >>>>>>>>> I
> > > > > >>>>>>>>>>>> think I
> > > > > >>>>>>>>>>>>>> need to understand the scenarios you mention a bit
> > > better.
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> As for the second part -- I think it makes sense to
> have
> > > > > >>>>>> some
> > > > > >>>>>>>>> sort
> > > > > >>>>>>>>>> of
> > > > > >>>>>>>>>>>>>> "sentinel" epoch to signal epoch is about to
> overflow (I
> > > > > >>>>>>> think
> > > > > >>>>>>>> we
> > > > > >>>>>>>>>> sort
> > > > > >>>>>>>>>>>> of
> > > > > >>>>>>>>>>>>>> have this value in place in some ways) so we can
> codify
> > > it
> > > > > >>>>>> in
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>> KIP.
> > > > > >>>>>>>>>>>> I'll
> > > > > >>>>>>>>>>>>>> look into that and try to update soon.
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> Thanks,
> > > > > >>>>>>>>>>>>>> Justine.
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
> > > > > >>>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> It's good to know that KIP-588 addressed some of
> the
> > > > > >>>>>> issues.
> > > > > >>>>>>>>>> Looking
> > > > > >>>>>>>>>>>> at
> > > > > >>>>>>>>>>>>>>> the code, it still looks like there are some cases
> that
> > > > > >>>>>>> would
> > > > > >>>>>>>>>> result
> > > > > >>>>>>>>>>>> in
> > > > > >>>>>>>>>>>>>>> fatal error, e.g. PRODUCER_FENCED is issued by the
> > > > > >>>>>>> transaction
> > > > > >>>>>>>>>>>> coordinator
> > > > > >>>>>>>>>>>>>>> if epoch doesn't match, and the client treats it
> as a
> > > > > >>>>>> fatal
> > > > > >>>>>>>>> error
> > > > > >>>>>>>>>>>> (code in
> > > > > >>>>>>>>>>>>>>> TransactionManager request handling).  If we
> consider,
> > > > for
> > > > > >>>>>>>>>> example,
> > > > > >>>>>>>>>>>>>>> committing a transaction that returns a timeout,
> but
> > > > > >>>>>>> actually
> > > > > >>>>>>>>>>>> succeeds,
> > > > > >>>>>>>>>>>>>>> trying to abort it or re-commit may result in
> > > > > >>>>>>> PRODUCER_FENCED
> > > > > >>>>>>>>>> error
> > > > > >>>>>>>>>>>>>>> (because of epoch bump).
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> For failed commits, specifically, we need to know
> the
> > > > > >>>>>> actual
> > > > > >>>>>>>>>> outcome,
> > > > > >>>>>>>>>>>>>>> because if we return an error the application may
> think
> > > > > >>>>>> that
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>>>>>>> transaction is aborted and redo the work, leading
> to
> > > > > >>>>>>>> duplicates.
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> Re: overflowing epoch.  We could either do it on
> the TC
> > > > > >>>>>> and
> > > > > >>>>>>>>> return
> > > > > >>>>>>>>>>>> both
> > > > > >>>>>>>>>>>>>>> producer id and epoch (e.g. change the protocol),
> or
> > > > > >>>>>> signal
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>> client
> > > > > >>>>>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>> it needs to get a new producer id.  Checking for
> max
> > > > epoch
> > > > > >>>>>>>> could
> > > > > >>>>>>>>>> be a
> > > > > >>>>>>>>>>>>>>> reasonable signal, the value to check should
> probably
> > > be
> > > > > >>>>>>>> present
> > > > > >>>>>>>>>> in
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>> KIP
> > > > > >>>>>>>>>>>>>>> as this is effectively a part of the contract.
> Also,
> > > the
> > > > > >>>>>> TC
> > > > > >>>>>>>>>> should
> > > > > >>>>>>>>>>>>>>> probably return an error if the client didn't
> change
> > > > > >>>>>>> producer
> > > > > >>>>>>>> id
> > > > > >>>>>>>>>> after
> > > > > >>>>>>>>>>>>>>> hitting max epoch.
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> -Artem
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
> > > > > >>>>>>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> Thanks for the discussion Artem.
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> With respect to the handling of fenced producers,
> we
> > > > > >>>>>> have
> > > > > >>>>>>>> some
> > > > > >>>>>>>>>>>> behavior
> > > > > >>>>>>>>>>>>>>>> already in place. As of KIP-588:
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>
> > > > > >>>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>
> > > > > >>>>
> > > > > >>
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> > > > > >>>>>>>>>>>>>>>> ,
> > > > > >>>>>>>>>>>>>>>> we handle timeouts more gracefully. The producer
> can
> > > > > >>>>>>>> recover.
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> Produce requests can also recover from epoch
> fencing
> > > by
> > > > > >>>>>>>>>> aborting the
> > > > > >>>>>>>>>>>>>>>> transaction and starting over.
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> What other cases were you considering that would
> cause
> > > > > >>>>>> us
> > > > > >>>>>>> to
> > > > > >>>>>>>>>> have a
> > > > > >>>>>>>>>>>>>>> fenced
> > > > > >>>>>>>>>>>>>>>> epoch but we'd want to recover?
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> The first point about handling epoch overflows is
> > > fair.
> > > > > >>>>>> I
> > > > > >>>>>>>>> think
> > > > > >>>>>>>>>>>> there is
> > > > > >>>>>>>>>>>>>>>> some logic we'd need to consider. (ie, if we are
> one
> > > > > >>>>>> away
> > > > > >>>>>>>> from
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>> max
> > > > > >>>>>>>>>>>>>>>> epoch, we need to reset the producer ID.) I'm
> still
> > > > > >>>>>>>> wondering
> > > > > >>>>>>>>> if
> > > > > >>>>>>>>>>>> there
> > > > > >>>>>>>>>>>>>>> is a
> > > > > >>>>>>>>>>>>>>>> way to direct this from the response, or if
> everything
> > > > > >>>>>>>> should
> > > > > >>>>>>>>> be
> > > > > >>>>>>>>>>>> done on
> > > > > >>>>>>>>>>>>>>>> the client side. Let me know if you have any
> thoughts
> > > > > >>>>>>> here.
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> Thanks,
> > > > > >>>>>>>>>>>>>>>> Justine
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
> > > > > >>>>>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> There are some workflows in the client that are
> > > > > >>>>>> implied
> > > > > >>>>>>> by
> > > > > >>>>>>>>>>>> protocol
> > > > > >>>>>>>>>>>>>>>>> changes, e.g.:
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> - for new clients, epoch changes with every
> > > > > >>>>>> transaction
> > > > > >>>>>>>> and
> > > > > >>>>>>>>>> can
> > > > > >>>>>>>>>>>>>>> overflow,
> > > > > >>>>>>>>>>>>>>>>> in old clients this condition was handled
> > > > > >>>>>> transparently,
> > > > > >>>>>>>>>> because
> > > > > >>>>>>>>>>>> epoch
> > > > > >>>>>>>>>>>>>>>> was
> > > > > >>>>>>>>>>>>>>>>> bumped in InitProducerId and it would return a
> new
> > > > > >>>>>>>> producer
> > > > > >>>>>>>>>> id if
> > > > > >>>>>>>>>>>>>>> epoch
> > > > > >>>>>>>>>>>>>>>>> overflows, the new clients would need to
> implement
> > > > > >>>>>> some
> > > > > >>>>>>>>>> workflow
> > > > > >>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>> refresh
> > > > > >>>>>>>>>>>>>>>>> producer id
> > > > > >>>>>>>>>>>>>>>>> - how to handle fenced producers, for new clients
> > > > > >>>>>> epoch
> > > > > >>>>>>>>>> changes
> > > > > >>>>>>>>>>>> with
> > > > > >>>>>>>>>>>>>>>> every
> > > > > >>>>>>>>>>>>>>>>> transaction, so in presence of failures during
> > > > > >>>>>> commits /
> > > > > >>>>>>>>>> aborts,
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>> producer could get easily fenced, old clients
> would
> > > > > >>>>>>> pretty
> > > > > >>>>>>>>>> much
> > > > > >>>>>>>>>>>> would
> > > > > >>>>>>>>>>>>>>> get
> > > > > >>>>>>>>>>>>>>>>> fenced when a new incarnation of the producer was
> > > > > >>>>>>>>> initialized
> > > > > >>>>>>>>>> with
> > > > > >>>>>>>>>>>>>>>>> InitProducerId so it's ok to treat as a fatal
> error,
> > > > > >>>>>> the
> > > > > >>>>>>>> new
> > > > > >>>>>>>>>>>> clients
> > > > > >>>>>>>>>>>>>>>> would
> > > > > >>>>>>>>>>>>>>>>> need to implement some workflow to handle that
> error,
> > > > > >>>>>>>>>> otherwise
> > > > > >>>>>>>>>>>> they
> > > > > >>>>>>>>>>>>>>>> could
> > > > > >>>>>>>>>>>>>>>>> get fenced by themselves
> > > > > >>>>>>>>>>>>>>>>> - in particular (as a subset of the previous
> issue),
> > > > > >>>>>>> what
> > > > > >>>>>>>>>> would
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>> client
> > > > > >>>>>>>>>>>>>>>>> do if it got a timeout during commit?  commit
> > > could've
> > > > > >>>>>>>>>> succeeded
> > > > > >>>>>>>>>>>> or
> > > > > >>>>>>>>>>>>>>>> failed
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> Not sure if this has to be defined in the KIP as
> > > > > >>>>>>>>> implementing
> > > > > >>>>>>>>>>>> those
> > > > > >>>>>>>>>>>>>>>>> probably wouldn't require protocol changes, but
> we
> > > > > >>>>>> have
> > > > > >>>>>>>>>> multiple
> > > > > >>>>>>>>>>>>>>>>> implementations of Kafka clients, so probably
> would
> > > be
> > > > > >>>>>>>> good
> > > > > >>>>>>>>> to
> > > > > >>>>>>>>>>>> have
> > > > > >>>>>>>>>>>>>>> some
> > > > > >>>>>>>>>>>>>>>>> client implementation guidance.  Could also be
> done
> > > > > >>>>>> as a
> > > > > >>>>>>>>>> separate
> > > > > >>>>>>>>>>>> doc.
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> -Artem
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
> > > > > >>>>>>>>>>>>>>>> <jolshan@confluent.io.invalid
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> Hey all, I've updated the KIP to incorporate
> Jason's
> > > > > >>>>>>>>>>>> suggestions.
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>
> > > > > >>>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>
> > > > > >>>>
> > > > > >>
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> 1. Use AddPartitionsToTxn + verify flag to
> check on
> > > > > >>>>>>> old
> > > > > >>>>>>>>>> clients
> > > > > >>>>>>>>>>>>>>>>>> 2. Updated AddPartitionsToTxn API to support
> > > > > >>>>>>> transaction
> > > > > >>>>>>>>>>>> batching
> > > > > >>>>>>>>>>>>>>>>>> 3. Mention IBP bump
> > > > > >>>>>>>>>>>>>>>>>> 4. Mention auth change on new AddPartitionsToTxn
> > > > > >>>>>>>> version.
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> I'm planning on opening a vote soon.
> > > > > >>>>>>>>>>>>>>>>>> Thanks,
> > > > > >>>>>>>>>>>>>>>>>> Justine
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <
> > > > > >>>>>>>>>>>> jolshan@confluent.io
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>> Thanks Jason. Those changes make sense to me. I
> > > > > >>>>>> will
> > > > > >>>>>>>>>> update
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>> KIP.
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
> > > > > >>>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
> > > > > >>>>>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>> Hey Justine,
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>> I was wondering about compatibility here.
> When
> > > > > >>>>>> we
> > > > > >>>>>>>>> send
> > > > > >>>>>>>>>>>> requests
> > > > > >>>>>>>>>>>>>>>>>>>> between brokers, we want to ensure that the
> > > > > >>>>>>> receiving
> > > > > >>>>>>>>>> broker
> > > > > >>>>>>>>>>>>>>>>> understands
> > > > > >>>>>>>>>>>>>>>>>>>> the request (specifically the new fields).
> > > > > >>>>>>> Typically
> > > > > >>>>>>>>>> this is
> > > > > >>>>>>>>>>>> done
> > > > > >>>>>>>>>>>>>>>> via
> > > > > >>>>>>>>>>>>>>>>>>>> IBP/metadata version.
> > > > > >>>>>>>>>>>>>>>>>>>> I'm trying to think if there is a way around
> it
> > > > > >>>>>> but
> > > > > >>>>>>>> I'm
> > > > > >>>>>>>>>> not
> > > > > >>>>>>>>>>>> sure
> > > > > >>>>>>>>>>>>>>>> there
> > > > > >>>>>>>>>>>>>>>>>> is.
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>> Yes. I think we would gate usage of this
> behind
> > > > > >>>>>> an
> > > > > >>>>>>>> IBP
> > > > > >>>>>>>>>> bump.
> > > > > >>>>>>>>>>>> Does
> > > > > >>>>>>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>>>>>>> seem
> > > > > >>>>>>>>>>>>>>>>>>>> reasonable?
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>> As for the improvements -- can you clarify
> how
> > > > > >>>>>>> the
> > > > > >>>>>>>>>> multiple
> > > > > >>>>>>>>>>>>>>>>>>>> transactional
> > > > > >>>>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking of a
> case
> > > > > >>>>>>>> where
> > > > > >>>>>>>>> we
> > > > > >>>>>>>>>>>>>>> wait/batch
> > > > > >>>>>>>>>>>>>>>>>>>> multiple produce requests together? My
> > > > > >>>>>>> understanding
> > > > > >>>>>>>>> for
> > > > > >>>>>>>>>> now
> > > > > >>>>>>>>>>>> was
> > > > > >>>>>>>>>>>>>>> 1
> > > > > >>>>>>>>>>>>>>>>>>>> transactional ID and one validation per 1
> produce
> > > > > >>>>>>>>>> request.
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>> Each call to `AddPartitionsToTxn` is
> essentially
> > > > > >>>>>> a
> > > > > >>>>>>>>> write
> > > > > >>>>>>>>>> to
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>>>>>>> log and must block on replication. The more we
> > > > > >>>>>> can
> > > > > >>>>>>>> fit
> > > > > >>>>>>>>>> into a
> > > > > >>>>>>>>>>>>>>> single
> > > > > >>>>>>>>>>>>>>>>>>>> request, the more writes we can do in
> parallel.
> > > > > >>>>>> The
> > > > > >>>>>>>>>>>> alternative
> > > > > >>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>> make
> > > > > >>>>>>>>>>>>>>>>>>>> use of more connections, but usually we prefer
> > > > > >>>>>>>> batching
> > > > > >>>>>>>>>>>> since the
> > > > > >>>>>>>>>>>>>>>>>> network
> > > > > >>>>>>>>>>>>>>>>>>>> stack is not really optimized for high
> > > > > >>>>>>>>> connection/request
> > > > > >>>>>>>>>>>> loads.
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>> Finally with respect to the authorizations, I
> > > > > >>>>>>> think
> > > > > >>>>>>>>> it
> > > > > >>>>>>>>>>>> makes
> > > > > >>>>>>>>>>>>>>> sense
> > > > > >>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>> skip
> > > > > >>>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit confused
> by
> > > > > >>>>>> the
> > > > > >>>>>>>>>> "leader
> > > > > >>>>>>>>>>>> ID"
> > > > > >>>>>>>>>>>>>>>> field.
> > > > > >>>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the request as
> > > > > >>>>>> from a
> > > > > >>>>>>>>>> broker
> > > > > >>>>>>>>>>>> (does
> > > > > >>>>>>>>>>>>>>> it
> > > > > >>>>>>>>>>>>>>>>>> matter
> > > > > >>>>>>>>>>>>>>>>>>>> which one?).
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>> We could also make it version-based. For the
> next
> > > > > >>>>>>>>>> version, we
> > > > > >>>>>>>>>>>>>>> could
> > > > > >>>>>>>>>>>>>>>>>>>> require
> > > > > >>>>>>>>>>>>>>>>>>>> CLUSTER auth. So clients would not be able to
> use
> > > > > >>>>>>> the
> > > > > >>>>>>>>> API
> > > > > >>>>>>>>>>>>>>> anymore,
> > > > > >>>>>>>>>>>>>>>>> which
> > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>> probably what we want.
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>> -Jason
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> > > > > >>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>> As a follow up, I was just thinking about the
> > > > > >>>>>>>>> batching
> > > > > >>>>>>>>>> a
> > > > > >>>>>>>>>>>> bit
> > > > > >>>>>>>>>>>>>>> more.
> > > > > >>>>>>>>>>>>>>>>>>>>> I suppose if we have one request in flight
> and
> > > > > >>>>>> we
> > > > > >>>>>>>>>> queue up
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>> other
> > > > > >>>>>>>>>>>>>>>>>>>>> produce requests in some sort of purgatory,
> we
> > > > > >>>>>>>> could
> > > > > >>>>>>>>>> send
> > > > > >>>>>>>>>>>>>>>>> information
> > > > > >>>>>>>>>>>>>>>>>>>> out
> > > > > >>>>>>>>>>>>>>>>>>>>> for all of them rather than one by one. So
> that
> > > > > >>>>>>>> would
> > > > > >>>>>>>>>> be a
> > > > > >>>>>>>>>>>>>>> benefit
> > > > > >>>>>>>>>>>>>>>>> of
> > > > > >>>>>>>>>>>>>>>>>>>>> batching partitions to add per transaction.
> > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>> I'll need to think a bit more on the design
> of
> > > > > >>>>>>> this
> > > > > >>>>>>>>>> part
> > > > > >>>>>>>>>>>> of the
> > > > > >>>>>>>>>>>>>>>> KIP,
> > > > > >>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>> will update the KIP in the next few days.
> > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > >>>>>>>>>>>>>>>>>>>>> Justine
> > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:22 AM Justine
> Olshan
> > > > > >>>>>> <
> > > > > >>>>>>>>>>>>>>>>> jolshan@confluent.io>
> > > > > >>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>> Hey Jason -- thanks for the input -- I was
> > > > > >>>>>> just
> > > > > >>>>>>>>>> digging
> > > > > >>>>>>>>>>>> a bit
> > > > > >>>>>>>>>>>>>>>>> deeper
> > > > > >>>>>>>>>>>>>>>>>>>> into
> > > > > >>>>>>>>>>>>>>>>>>>>>> the design + implementation of the
> validation
> > > > > >>>>>>>> calls
> > > > > >>>>>>>>>> here
> > > > > >>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>> what
> > > > > >>>>>>>>>>>>>>>>>> you
> > > > > >>>>>>>>>>>>>>>>>>>> say
> > > > > >>>>>>>>>>>>>>>>>>>>>> makes sense.
> > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>> I was wondering about compatibility here.
> > > > > >>>>>> When
> > > > > >>>>>>> we
> > > > > >>>>>>>>>> send
> > > > > >>>>>>>>>>>>>>> requests
> > > > > >>>>>>>>>>>>>>>>>>>>>> between brokers, we want to ensure that the
> > > > > >>>>>>>>> receiving
> > > > > >>>>>>>>>>>> broker
> > > > > >>>>>>>>>>>>>>>>>>>> understands
> > > > > >>>>>>>>>>>>>>>>>>>>>> the request (specifically the new fields).
> > > > > >>>>>>>>> Typically
> > > > > >>>>>>>>>>>> this is
> > > > > >>>>>>>>>>>>>>>> done
> > > > > >>>>>>>>>>>>>>>>>> via
> > > > > >>>>>>>>>>>>>>>>>>>>>> IBP/metadata version.
> > > > > >>>>>>>>>>>>>>>>>>>>>> I'm trying to think if there is a way around
> > > > > >>>>>> it
> > > > > >>>>>>>> but
> > > > > >>>>>>>>>> I'm
> > > > > >>>>>>>>>>>> not
> > > > > >>>>>>>>>>>>>>> sure
> > > > > >>>>>>>>>>>>>>>>>> there
> > > > > >>>>>>>>>>>>>>>>>>>>> is.
> > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>> As for the improvements -- can you clarify
> > > > > >>>>>> how
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>>>> multiple
> > > > > >>>>>>>>>>>>>>>>>>>> transactional
> > > > > >>>>>>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking of a
> > > > > >>>>>>> case
> > > > > >>>>>>>>>> where we
> > > > > >>>>>>>>>>>>>>>>> wait/batch
> > > > > >>>>>>>>>>>>>>>>>>>>>> multiple produce requests together? My
> > > > > >>>>>>>>> understanding
> > > > > >>>>>>>>>> for
> > > > > >>>>>>>>>>>> now
> > > > > >>>>>>>>>>>>>>>> was 1
> > > > > >>>>>>>>>>>>>>>>>>>>>> transactional ID and one validation per 1
> > > > > >>>>>>> produce
> > > > > >>>>>>>>>>>> request.
> > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>> Finally with respect to the authorizations,
> I
> > > > > >>>>>>>> think
> > > > > >>>>>>>>>> it
> > > > > >>>>>>>>>>>> makes
> > > > > >>>>>>>>>>>>>>>> sense
> > > > > >>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>> skip
> > > > > >>>>>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit confused
> > > > > >>>>>> by
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>>>> "leader
> > > > > >>>>>>>>>>>>>>> ID"
> > > > > >>>>>>>>>>>>>>>>>> field.
> > > > > >>>>>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the request as
> > > > > >>>>>>>> from a
> > > > > >>>>>>>>>>>> broker
> > > > > >>>>>>>>>>>>>>> (does
> > > > > >>>>>>>>>>>>>>>>> it
> > > > > >>>>>>>>>>>>>>>>>>>>> matter
> > > > > >>>>>>>>>>>>>>>>>>>>>> which one?).
> > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>> I think I want to adopt these suggestions,
> > > > > >>>>>> just
> > > > > >>>>>>>> had
> > > > > >>>>>>>>>> a few
> > > > > >>>>>>>>>>>>>>>>> questions
> > > > > >>>>>>>>>>>>>>>>>> on
> > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>> details.
> > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > >>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>> On Thu, Jan 5, 2023 at 5:05 PM Jason
> > > > > >>>>>> Gustafson
> > > > > >>>>>>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
> > > > > >>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>> Thanks for the proposal.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>> I was thinking about the implementation a
> > > > > >>>>>>> little
> > > > > >>>>>>>>>> bit.
> > > > > >>>>>>>>>>>> In the
> > > > > >>>>>>>>>>>>>>>>>> current
> > > > > >>>>>>>>>>>>>>>>>>>>>>> proposal, the behavior depends on whether
> we
> > > > > >>>>>>>> have
> > > > > >>>>>>>>> an
> > > > > >>>>>>>>>>>> old or
> > > > > >>>>>>>>>>>>>>> new
> > > > > >>>>>>>>>>>>>>>>>>>> client.
> > > > > >>>>>>>>>>>>>>>>>>>>>>> For
> > > > > >>>>>>>>>>>>>>>>>>>>>>> old clients, we send `DescribeTransactions`
> > > > > >>>>>>> and
> > > > > >>>>>>>>>> verify
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>> result
> > > > > >>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>>>>>>> new clients, we send `AddPartitionsToTxn`.
> > > > > >>>>>> We
> > > > > >>>>>>>>> might
> > > > > >>>>>>>>>> be
> > > > > >>>>>>>>>>>> able
> > > > > >>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>> simplify
> > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> implementation if we can use the same
> > > > > >>>>>> request
> > > > > >>>>>>>>> type.
> > > > > >>>>>>>>>> For
> > > > > >>>>>>>>>>>>>>>> example,
> > > > > >>>>>>>>>>>>>>>>>>>> what if
> > > > > >>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>> bump the protocol version for
> > > > > >>>>>>>> `AddPartitionsToTxn`
> > > > > >>>>>>>>>> and
> > > > > >>>>>>>>>>>> add a
> > > > > >>>>>>>>>>>>>>>>>>>>>>> `validateOnly`
> > > > > >>>>>>>>>>>>>>>>>>>>>>> flag? For older versions, we can set
> > > > > >>>>>>>>>>>> `validateOnly=true` so
> > > > > >>>>>>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> request only returns successfully if the
> > > > > >>>>>>>> partition
> > > > > >>>>>>>>>> had
> > > > > >>>>>>>>>>>>>>> already
> > > > > >>>>>>>>>>>>>>>>> been
> > > > > >>>>>>>>>>>>>>>>>>>>> added.
> > > > > >>>>>>>>>>>>>>>>>>>>>>> For new versions, we can set
> > > > > >>>>>>>> `validateOnly=false`
> > > > > >>>>>>>>>> and
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>> partition
> > > > > >>>>>>>>>>>>>>>>>>>> will
> > > > > >>>>>>>>>>>>>>>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>>>>>>>>>> added to the transaction. The other
> slightly
> > > > > >>>>>>>>>> annoying
> > > > > >>>>>>>>>>>> thing
> > > > > >>>>>>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > > >>>>>>>>>>>>>>>>>>>>>>> get around is the need to collect the
> > > > > >>>>>>>> transaction
> > > > > >>>>>>>>>> state
> > > > > >>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>> all
> > > > > >>>>>>>>>>>>>>>>>>>>> partitions
> > > > > >>>>>>>>>>>>>>>>>>>>>>> even when we only care about a subset.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>> Some additional improvements to consider:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>> - We can give `AddPartitionsToTxn` better
> > > > > >>>>>>> batch
> > > > > >>>>>>>>>> support
> > > > > >>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>>>> inter-broker
> > > > > >>>>>>>>>>>>>>>>>>>>>>> usage. Currently we only allow one
> > > > > >>>>>>>>>> `TransactionalId` to
> > > > > >>>>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>>>>> specified,
> > > > > >>>>>>>>>>>>>>>>>>>>> but
> > > > > >>>>>>>>>>>>>>>>>>>>>>> the broker may get some benefit being able
> > > > > >>>>>> to
> > > > > >>>>>>>>> batch
> > > > > >>>>>>>>>>>> across
> > > > > >>>>>>>>>>>>>>>>> multiple
> > > > > >>>>>>>>>>>>>>>>>>>>>>> transactions.
> > > > > >>>>>>>>>>>>>>>>>>>>>>> - Another small improvement is skipping
> > > > > >>>>>> topic
> > > > > >>>>>>>>>>>> authorization
> > > > > >>>>>>>>>>>>>>>>> checks
> > > > > >>>>>>>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>>>>>>> `AddPartitionsToTxn` when the request is
> > > > > >>>>>> from
> > > > > >>>>>>> a
> > > > > >>>>>>>>>> broker.
> > > > > >>>>>>>>>>>>>>> Perhaps
> > > > > >>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>> can
> > > > > >>>>>>>>>>>>>>>>>>>>> add
> > > > > >>>>>>>>>>>>>>>>>>>>>>> a field for the `LeaderId` or something
> like
> > > > > >>>>>>>> that
> > > > > >>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>> require
> > > > > >>>>>>>>>>>>>>>>>> CLUSTER
> > > > > >>>>>>>>>>>>>>>>>>>>>>> permission when set.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>> Best,
> > > > > >>>>>>>>>>>>>>>>>>>>>>> Jason
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
> > > > > >>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation. It makes sense
> > > > > >>>>>>> to
> > > > > >>>>>>>> me
> > > > > >>>>>>>>>> now.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 1:42 PM Justine
> > > > > >>>>>>> Olshan
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> My understanding of the mechanism is
> > > > > >>>>>> that
> > > > > >>>>>>>> when
> > > > > >>>>>>>>>> we
> > > > > >>>>>>>>>>>> get to
> > > > > >>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>> last
> > > > > >>>>>>>>>>>>>>>>>>>>>>> epoch,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> increment to the fencing/last epoch and
> > > > > >>>>>> if
> > > > > >>>>>>>> any
> > > > > >>>>>>>>>>>> further
> > > > > >>>>>>>>>>>>>>>>> requests
> > > > > >>>>>>>>>>>>>>>>>>>> come
> > > > > >>>>>>>>>>>>>>>>>>>>>>> in
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> this producer ID they are fenced. Then
> > > > > >>>>>> the
> > > > > >>>>>>>>>> producer
> > > > > >>>>>>>>>>>>>>> gets a
> > > > > >>>>>>>>>>>>>>>>> new
> > > > > >>>>>>>>>>>>>>>>>> ID
> > > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> restarts with epoch/sequence 0. The
> > > > > >>>>>> fenced
> > > > > >>>>>>>>> epoch
> > > > > >>>>>>>>>>>> sticks
> > > > > >>>>>>>>>>>>>>>>> around
> > > > > >>>>>>>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> duration of producer.id.expiration.ms
> > > > > >>>>>> and
> > > > > >>>>>>>>>> blocks
> > > > > >>>>>>>>>>>> any
> > > > > >>>>>>>>>>>>>>> late
> > > > > >>>>>>>>>>>>>>>>>>>> messages
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> there.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> The new ID will get to take advantage of
> > > > > >>>>>>> the
> > > > > >>>>>>>>>>>> improved
> > > > > >>>>>>>>>>>>>>>>> semantics
> > > > > >>>>>>>>>>>>>>>>>>>>> around
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> non-zero start sequences. So I think we
> > > > > >>>>>>> are
> > > > > >>>>>>>>>> covered.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> The only potential issue is overloading
> > > > > >>>>>>> the
> > > > > >>>>>>>>>> cache,
> > > > > >>>>>>>>>>>> but
> > > > > >>>>>>>>>>>>>>>>>> hopefully
> > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> improvements (lowered
> > > > > >>>>>>>>> producer.id.expiration.ms
> > > > > >>>>>>>>>> )
> > > > > >>>>>>>>>>>> will
> > > > > >>>>>>>>>>>>>>> help
> > > > > >>>>>>>>>>>>>>>>>> with
> > > > > >>>>>>>>>>>>>>>>>>>>> that.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> Let
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> me know if you still have concerns.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
> > > > > >>>>>>>>>>>>>>>>>>>> <ju...@confluent.io.invalid>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> 70. The proposed fencing logic doesn't
> > > > > >>>>>>>> apply
> > > > > >>>>>>>>>> when
> > > > > >>>>>>>>>>>> pid
> > > > > >>>>>>>>>>>>>>>>>> changes,
> > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> right? If so, I am not sure how
> > > > > >>>>>> complete
> > > > > >>>>>>>> we
> > > > > >>>>>>>>>> are
> > > > > >>>>>>>>>>>>>>>> addressing
> > > > > >>>>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>>>>> issue
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the pid changes more frequently.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 16, 2022 at 9:16 AM
> > > > > >>>>>> Justine
> > > > > >>>>>>>>> Olshan
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for replying!
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> 70.We already do the overflow
> > > > > >>>>>>> mechanism,
> > > > > >>>>>>>>> so
> > > > > >>>>>>>>>> my
> > > > > >>>>>>>>>>>>>>> change
> > > > > >>>>>>>>>>>>>>>>> would
> > > > > >>>>>>>>>>>>>>>>>>>> just
> > > > > >>>>>>>>>>>>>>>>>>>>>>> make
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> happen more often.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I was also not suggesting a new
> > > > > >>>>>> field
> > > > > >>>>>>> in
> > > > > >>>>>>>>> the
> > > > > >>>>>>>>>>>> log,
> > > > > >>>>>>>>>>>>>>> but
> > > > > >>>>>>>>>>>>>>>> in
> > > > > >>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> response,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> which would be gated by the client
> > > > > >>>>>>>>> version.
> > > > > >>>>>>>>>>>> Sorry if
> > > > > >>>>>>>>>>>>>>>>>>>> something
> > > > > >>>>>>>>>>>>>>>>>>>>>>> there
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> unclear. I think we are starting to
> > > > > >>>>>>>>> diverge.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> The goal of this KIP is to not
> > > > > >>>>>> change
> > > > > >>>>>>> to
> > > > > >>>>>>>>> the
> > > > > >>>>>>>>>>>> marker
> > > > > >>>>>>>>>>>>>>>>> format
> > > > > >>>>>>>>>>>>>>>>>> at
> > > > > >>>>>>>>>>>>>>>>>>>>> all.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> 71. Yes, I guess I was going under
> > > > > >>>>>> the
> > > > > >>>>>>>>>>>> assumption
> > > > > >>>>>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>> log
> > > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> just
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> look at its last epoch and treat it
> > > > > >>>>>> as
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>>>> current
> > > > > >>>>>>>>>>>>>>>>> epoch. I
> > > > > >>>>>>>>>>>>>>>>>>>>>>> suppose
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> have some special logic that if the
> > > > > >>>>>>> last
> > > > > >>>>>>>>>> epoch
> > > > > >>>>>>>>>>>> was
> > > > > >>>>>>>>>>>>>>> on a
> > > > > >>>>>>>>>>>>>>>>>>>> marker
> > > > > >>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> actually
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> expect the next epoch or something
> > > > > >>>>>>> like
> > > > > >>>>>>>>>> that. We
> > > > > >>>>>>>>>>>>>>> just
> > > > > >>>>>>>>>>>>>>>>> need
> > > > > >>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> distinguish
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> based on whether we had a
> > > > > >>>>>> commit/abort
> > > > > >>>>>>>>>> marker.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> 72.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> if the producer epoch hasn't been
> > > > > >>>>>>>> bumped
> > > > > >>>>>>>>>> on
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
> > > > > >>>>>>>> message
> > > > > >>>>>>>>>> will
> > > > > >>>>>>>>>>>> fail
> > > > > >>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>> sequence
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> validation
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the producer
> > > > > >>>>>>>> epoch
> > > > > >>>>>>>>>> has
> > > > > >>>>>>>>>>>> been
> > > > > >>>>>>>>>>>>>>>>> bumped,
> > > > > >>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>> ignore
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck message
> > > > > >>>>>>>> could
> > > > > >>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>> appended
> > > > > >>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>> log.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> So,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
> > > > > >>>>>> guard?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I follow that "the
> > > > > >>>>>>> message
> > > > > >>>>>>>>> will
> > > > > >>>>>>>>>>>> fail
> > > > > >>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>> sequence
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> validation".
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> In some of these cases, we had an
> > > > > >>>>>>> abort
> > > > > >>>>>>>>>> marker
> > > > > >>>>>>>>>>>> (due
> > > > > >>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>> an
> > > > > >>>>>>>>>>>>>>>>>>>> error)
> > > > > >>>>>>>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> then
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the late message comes in with the
> > > > > >>>>>>>> correct
> > > > > >>>>>>>>>>>> sequence
> > > > > >>>>>>>>>>>>>>>>> number.
> > > > > >>>>>>>>>>>>>>>>>>>> This
> > > > > >>>>>>>>>>>>>>>>>>>>>>> is a
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> case
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> covered by the KIP.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> The latter case is actually not
> > > > > >>>>>>>> something
> > > > > >>>>>>>>>> we've
> > > > > >>>>>>>>>>>>>>>>> considered
> > > > > >>>>>>>>>>>>>>>>>>>>> here. I
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> think
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> generally when we bump the epoch, we
> > > > > >>>>>>> are
> > > > > >>>>>>>>>>>> accepting
> > > > > >>>>>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> sequence
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> does
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> not need to be checked anymore. My
> > > > > >>>>>>>>>>>> understanding is
> > > > > >>>>>>>>>>>>>>>> also
> > > > > >>>>>>>>>>>>>>>>>>>> that we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> typically bump epoch mid transaction
> > > > > >>>>>>>>> (based
> > > > > >>>>>>>>>> on a
> > > > > >>>>>>>>>>>>>>> quick
> > > > > >>>>>>>>>>>>>>>>> look
> > > > > >>>>>>>>>>>>>>>>>>>> at
> > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> code)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but let me know if that is the case.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 15, 2022 at 12:23 PM Jun
> > > > > >>>>>>> Rao
> > > > > >>>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the reply.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Assigning a new pid on int
> > > > > >>>>>>>> overflow
> > > > > >>>>>>>>>> seems
> > > > > >>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>> bit
> > > > > >>>>>>>>>>>>>>>>>> hacky.
> > > > > >>>>>>>>>>>>>>>>>>>> If
> > > > > >>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> need a
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> level id, it will be better to
> > > > > >>>>>> model
> > > > > >>>>>>>>> this
> > > > > >>>>>>>>>>>>>>> explicitly.
> > > > > >>>>>>>>>>>>>>>>>>>> Adding a
> > > > > >>>>>>>>>>>>>>>>>>>>>>> new
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> field
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> would require a bit more work
> > > > > >>>>>> since
> > > > > >>>>>>> it
> > > > > >>>>>>>>>>>> requires a
> > > > > >>>>>>>>>>>>>>> new
> > > > > >>>>>>>>>>>>>>>>> txn
> > > > > >>>>>>>>>>>>>>>>>>>>> marker
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> format
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the log. So, we probably need to
> > > > > >>>>>>> guard
> > > > > >>>>>>>>> it
> > > > > >>>>>>>>>>>> with an
> > > > > >>>>>>>>>>>>>>> IBP
> > > > > >>>>>>>>>>>>>>>>> or
> > > > > >>>>>>>>>>>>>>>>>>>>>>> metadata
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> version
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and document the impact on
> > > > > >>>>>> downgrade
> > > > > >>>>>>>>> once
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>> new
> > > > > >>>>>>>>>>>>>>>>> format
> > > > > >>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>>> written
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 71. Hmm, once the marker is
> > > > > >>>>>> written,
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>>>> partition
> > > > > >>>>>>>>>>>>>>>> will
> > > > > >>>>>>>>>>>>>>>>>>>> expect
> > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> next
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> append to be on the next epoch.
> > > > > >>>>>> Does
> > > > > >>>>>>>>> that
> > > > > >>>>>>>>>>>> cover
> > > > > >>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>> case
> > > > > >>>>>>>>>>>>>>>>>>>> you
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> mentioned?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 72. Also, just to be clear on the
> > > > > >>>>>>>>> stucked
> > > > > >>>>>>>>>>>> message
> > > > > >>>>>>>>>>>>>>>> issue
> > > > > >>>>>>>>>>>>>>>>>>>>>>> described
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> motivation. With EoS, we also
> > > > > >>>>>>> validate
> > > > > >>>>>>>>> the
> > > > > >>>>>>>>>>>>>>> sequence
> > > > > >>>>>>>>>>>>>>>> id
> > > > > >>>>>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> idempotency.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> So,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> with the current logic, if the
> > > > > >>>>>>>> producer
> > > > > >>>>>>>>>> epoch
> > > > > >>>>>>>>>>>>>>> hasn't
> > > > > >>>>>>>>>>>>>>>>> been
> > > > > >>>>>>>>>>>>>>>>>>>>>>> bumped on
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
> > > > > >>>>>>>>> message
> > > > > >>>>>>>>>> will
> > > > > >>>>>>>>>>>>>>> fail
> > > > > >>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>> sequence
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> validation
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the
> > > > > >>>>>> producer
> > > > > >>>>>>>>>> epoch has
> > > > > >>>>>>>>>>>>>>> been
> > > > > >>>>>>>>>>>>>>>>>>>> bumped, we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> ignore
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck
> > > > > >>>>>> message
> > > > > >>>>>>>>>> could be
> > > > > >>>>>>>>>>>>>>>> appended
> > > > > >>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> log.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> So,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
> > > > > >>>>>>> guard?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 10:44 AM
> > > > > >>>>>>>> Justine
> > > > > >>>>>>>>>>>> Olshan
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > >>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias — thanks again for
> > > > > >>>>>> taking
> > > > > >>>>>>>>> time
> > > > > >>>>>>>>>> to
> > > > > >>>>>>>>>>>> look
> > > > > >>>>>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>>>> this.
> > > > > >>>>>>>>>>>>>>>>>>>> You
> > > > > >>>>>>>>>>>>>>>>>>>>>>> said:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My proposal was only focusing
> > > > > >>>>>> to
> > > > > >>>>>>>>> avoid
> > > > > >>>>>>>>>>>>>>> dangling
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
> > > > > >>>>>> added
> > > > > >>>>>>>>>> without
> > > > > >>>>>>>>>>>>>>>> registered
> > > > > >>>>>>>>>>>>>>>>>>>>>>> partition.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> --
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more details
> > > > > >>>>>> to
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>> KIP
> > > > > >>>>>>>>>>>> about
> > > > > >>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>>> scenario
> > > > > >>>>>>>>>>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> better
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I understand what
> > > > > >>>>>> you
> > > > > >>>>>>>>> mean
> > > > > >>>>>>>>>>>> here.
> > > > > >>>>>>>>>>>>>>> The
> > > > > >>>>>>>>>>>>>>>>>>>> motivation
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> section
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> describes two scenarios about
> > > > > >>>>>> how
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>> record
> > > > > >>>>>>>>>>>>>>> can be
> > > > > >>>>>>>>>>>>>>>>>> added
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> without a
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> registered partition:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Another way hanging
> > > > > >>>>>> transactions
> > > > > >>>>>>>> can
> > > > > >>>>>>>>>>>> occur is
> > > > > >>>>>>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>>>>>>>> client
> > > > > >>>>>>>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> buggy
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> may somehow try to write to a
> > > > > >>>>>>>>> partition
> > > > > >>>>>>>>>>>> before
> > > > > >>>>>>>>>>>>>>> it
> > > > > >>>>>>>>>>>>>>>>> adds
> > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> partition
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> For the first example of this
> > > > > >>>>>>> would
> > > > > >>>>>>>> it
> > > > > >>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>> helpful
> > > > > >>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>> say
> > > > > >>>>>>>>>>>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> comes in after the abort, but
> > > > > >>>>>>> before
> > > > > >>>>>>>>> the
> > > > > >>>>>>>>>>>>>>> partition
> > > > > >>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>> added
> > > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> next
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction so it becomes
> > > > > >>>>>>> "hanging."
> > > > > >>>>>>>>>>>> Perhaps the
> > > > > >>>>>>>>>>>>>>>> next
> > > > > >>>>>>>>>>>>>>>>>>>>> sentence
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> describing
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the message becoming part of the
> > > > > >>>>>>>> next
> > > > > >>>>>>>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>>> (a
> > > > > >>>>>>>>>>>>>>>>>>>>> different
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> case)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> was
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not properly differentiated.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun — thanks for reading the
> > > > > >>>>>> KIP.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. The int typing was a
> > > > > >>>>>> concern.
> > > > > >>>>>>>>>> Currently
> > > > > >>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>> have a
> > > > > >>>>>>>>>>>>>>>>>>>>>>> mechanism
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> place
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> fence the final epoch when the
> > > > > >>>>>>> epoch
> > > > > >>>>>>>>> is
> > > > > >>>>>>>>>>>> about to
> > > > > >>>>>>>>>>>>>>>>>> overflow
> > > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> assign
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer ID with epoch 0. Of
> > > > > >>>>>>> course,
> > > > > >>>>>>>>>> this
> > > > > >>>>>>>>>>>> is a
> > > > > >>>>>>>>>>>>>>> bit
> > > > > >>>>>>>>>>>>>>>>>> tricky
> > > > > >>>>>>>>>>>>>>>>>>>>>>> when it
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> comes
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response back to the client.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Making this a long could be
> > > > > >>>>>>> another
> > > > > >>>>>>>>>> option,
> > > > > >>>>>>>>>>>> but
> > > > > >>>>>>>>>>>>>>> I
> > > > > >>>>>>>>>>>>>>>>>> wonder
> > > > > >>>>>>>>>>>>>>>>>>>> are
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> there
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> implications on changing this
> > > > > >>>>>>> field
> > > > > >>>>>>>> if
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>> epoch is
> > > > > >>>>>>>>>>>>>>>>>>>>> persisted
> > > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> disk?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to check the usages.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 71.This was something Matthias
> > > > > >>>>>>> asked
> > > > > >>>>>>>>>> about
> > > > > >>>>>>>>>>>> as
> > > > > >>>>>>>>>>>>>>>> well. I
> > > > > >>>>>>>>>>>>>>>>>> was
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> considering a
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible edge case where a
> > > > > >>>>>> produce
> > > > > >>>>>>>>>> request
> > > > > >>>>>>>>>>>> from
> > > > > >>>>>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>>> new
> > > > > >>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> somehow
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> gets sent right after the marker
> > > > > >>>>>>> is
> > > > > >>>>>>>>>>>> written, but
> > > > > >>>>>>>>>>>>>>>>> before
> > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> alerted of the newly bumped
> > > > > >>>>>> epoch.
> > > > > >>>>>>>> In
> > > > > >>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>> case, we
> > > > > >>>>>>>>>>>>>>>>> may
> > > > > >>>>>>>>>>>>>>>>>>>>>>> include
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> record
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we don't want to. I suppose
> > > > > >>>>>>> we
> > > > > >>>>>>>>>> could
> > > > > >>>>>>>>>>>> try
> > > > > >>>>>>>>>>>>>>> to do
> > > > > >>>>>>>>>>>>>>>>>>>>> something
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> client
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> side
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to bump the epoch after sending
> > > > > >>>>>> an
> > > > > >>>>>>>>>> endTxn as
> > > > > >>>>>>>>>>>>>>> well
> > > > > >>>>>>>>>>>>>>>> in
> > > > > >>>>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> scenario
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> —
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> I
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wonder how it would work when
> > > > > >>>>>> the
> > > > > >>>>>>>>>> server is
> > > > > >>>>>>>>>>>>>>>> aborting
> > > > > >>>>>>>>>>>>>>>>>>>> based
> > > > > >>>>>>>>>>>>>>>>>>>>> on
> > > > > >>>>>>>>>>>>>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> error. I could also be missing
> > > > > >>>>>>>>>> something and
> > > > > >>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>> scenario
> > > > > >>>>>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> actually
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again to everyone reading
> > > > > >>>>>>> and
> > > > > >>>>>>>>>>>> commenting.
> > > > > >>>>>>>>>>>>>>>> Let
> > > > > >>>>>>>>>>>>>>>>> me
> > > > > >>>>>>>>>>>>>>>>>>>> know
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> about
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> further questions or comments.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 9:41 AM
> > > > > >>>>>>> Jun
> > > > > >>>>>>>>> Rao
> > > > > >>>>>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP. A couple
> > > > > >>>>>> of
> > > > > >>>>>>>>>> comments.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Currently, the producer
> > > > > >>>>>>> epoch
> > > > > >>>>>>>> is
> > > > > >>>>>>>>>> an
> > > > > >>>>>>>>>>>> int.
> > > > > >>>>>>>>>>>>>>> I am
> > > > > >>>>>>>>>>>>>>>>> not
> > > > > >>>>>>>>>>>>>>>>>>>> sure
> > > > > >>>>>>>>>>>>>>>>>>>>>>> if
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> it's
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> enough
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to accommodate all
> > > > > >>>>>> transactions
> > > > > >>>>>>> in
> > > > > >>>>>>>>> the
> > > > > >>>>>>>>>>>>>>> lifetime
> > > > > >>>>>>>>>>>>>>>> of
> > > > > >>>>>>>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>>>>>>>>>> producer.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Should
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> change that to a long or add a
> > > > > >>>>>>> new
> > > > > >>>>>>>>>> long
> > > > > >>>>>>>>>>>> field
> > > > > >>>>>>>>>>>>>>>> like
> > > > > >>>>>>>>>>>>>>>>>>>> txnId?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 71. "it will write the prepare
> > > > > >>>>>>>>> commit
> > > > > >>>>>>>>>>>> message
> > > > > >>>>>>>>>>>>>>>> with
> > > > > >>>>>>>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>>>>>>>> bumped
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> send
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WriteTxnMarkerRequests with
> > > > > >>>>>> the
> > > > > >>>>>>>>> bumped
> > > > > >>>>>>>>>>>> epoch."
> > > > > >>>>>>>>>>>>>>>> Hmm,
> > > > > >>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> associated
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the current txn right?
> > > > > >>>>>> So,
> > > > > >>>>>>> it
> > > > > >>>>>>>>>> seems
> > > > > >>>>>>>>>>>>>>> weird to
> > > > > >>>>>>>>>>>>>>>>>>>> write a
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> commit
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with a bumped epoch. Should we
> > > > > >>>>>>>> only
> > > > > >>>>>>>>>> bump
> > > > > >>>>>>>>>>>> up
> > > > > >>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>> epoch
> > > > > >>>>>>>>>>>>>>>>>>>> in
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> EndTxnResponse
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rename the field to sth like
> > > > > >>>>>>>>>>>>>>> nextProducerEpoch?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 12, 2022 at 8:54
> > > > > >>>>>> PM
> > > > > >>>>>>>>>> Matthias
> > > > > >>>>>>>>>>>> J.
> > > > > >>>>>>>>>>>>>>> Sax <
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the background.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30: SGTM. My proposal was
> > > > > >>>>>>>> only
> > > > > >>>>>>>>>>>> focusing
> > > > > >>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>> avoid
> > > > > >>>>>>>>>>>>>>>>>>>>>>> dangling
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
> > > > > >>>>>>>> added
> > > > > >>>>>>>>>>>> without
> > > > > >>>>>>>>>>>>>>>>>> registered
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> partition.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> --
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more
> > > > > >>>>>> details
> > > > > >>>>>>>> to
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>> KIP
> > > > > >>>>>>>>>>>>>>> about
> > > > > >>>>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>>>>> scenario
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40: I think you hit a fair
> > > > > >>>>>>> point
> > > > > >>>>>>>>>> about
> > > > > >>>>>>>>>>>> race
> > > > > >>>>>>>>>>>>>>>>>>>> conditions
> > > > > >>>>>>>>>>>>>>>>>>>>> or
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> client
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> bugs
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (incorrectly not bumping the
> > > > > >>>>>>>>>> epoch). The
> > > > > >>>>>>>>>>>>>>>>>>>>>>> complexity/confusion
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> using
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the bumped epoch I see, is
> > > > > >>>>>>>> mainly
> > > > > >>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>> internal
> > > > > >>>>>>>>>>>>>>>>>>>>> debugging,
> > > > > >>>>>>>>>>>>>>>>>>>>>>> ie,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> inspecting
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log segment dumps -- it
> > > > > >>>>>> seems
> > > > > >>>>>>>>>> harder to
> > > > > >>>>>>>>>>>>>>> reason
> > > > > >>>>>>>>>>>>>>>>>> about
> > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> system
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> us
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> humans. But if we get better
> > > > > >>>>>>>>>>>> guarantees, it
> > > > > >>>>>>>>>>>>>>>> would
> > > > > >>>>>>>>>>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>>>>>>>>>> worth to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> use
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped epoch.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60: as I mentioned already,
> > > > > >>>>>> I
> > > > > >>>>>>>>> don't
> > > > > >>>>>>>>>>>> know the
> > > > > >>>>>>>>>>>>>>>>> broker
> > > > > >>>>>>>>>>>>>>>>>>>>>>> internals
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> provide
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more input. So if nobody
> > > > > >>>>>> else
> > > > > >>>>>>>>> chimes
> > > > > >>>>>>>>>>>> in, we
> > > > > >>>>>>>>>>>>>>>>> should
> > > > > >>>>>>>>>>>>>>>>>>>> just
> > > > > >>>>>>>>>>>>>>>>>>>>>>> move
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> forward
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with your proposal.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 12/6/22 4:22 PM, Justine
> > > > > >>>>>>>> Olshan
> > > > > >>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi all,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> After Artem's questions
> > > > > >>>>>>> about
> > > > > >>>>>>>>>> error
> > > > > >>>>>>>>>>>>>>> behavior,
> > > > > >>>>>>>>>>>>>>>>>> I've
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> re-evaluated
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unknown producer ID
> > > > > >>>>>>> exception
> > > > > >>>>>>>>> and
> > > > > >>>>>>>>>> had
> > > > > >>>>>>>>>>>> some
> > > > > >>>>>>>>>>>>>>>>>>>> discussions
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> offline.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think generally it makes
> > > > > >>>>>>>> sense
> > > > > >>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>> simplify
> > > > > >>>>>>>>>>>>>>>>>> error
> > > > > >>>>>>>>>>>>>>>>>>>>>>> handling
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> cases
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this and the
> > > > > >>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > >>>>>>>>>> error
> > > > > >>>>>>>>>>>>>>> has a
> > > > > >>>>>>>>>>>>>>>>>> pretty
> > > > > >>>>>>>>>>>>>>>>>>>>> long
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complicated
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> history. Because of this,
> > > > > >>>>>> I
> > > > > >>>>>>>>>> propose
> > > > > >>>>>>>>>>>>>>> adding a
> > > > > >>>>>>>>>>>>>>>>> new
> > > > > >>>>>>>>>>>>>>>>>>>> error
> > > > > >>>>>>>>>>>>>>>>>>>>>>> code
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ABORTABLE_ERROR
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that when encountered by
> > > > > >>>>>> new
> > > > > >>>>>>>>>> clients
> > > > > >>>>>>>>>>>>>>> (gated
> > > > > >>>>>>>>>>>>>>>> by
> > > > > >>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> produce
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will simply abort the
> > > > > >>>>>>>>> transaction.
> > > > > >>>>>>>>>>>> This
> > > > > >>>>>>>>>>>>>>>> allows
> > > > > >>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> server
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> some
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> say
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in whether the client
> > > > > >>>>>> aborts
> > > > > >>>>>>>> and
> > > > > >>>>>>>>>> makes
> > > > > >>>>>>>>>>>>>>>> handling
> > > > > >>>>>>>>>>>>>>>>>>>> much
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> simpler.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> future, we can also use
> > > > > >>>>>> this
> > > > > >>>>>>>>>> error in
> > > > > >>>>>>>>>>>>>>> other
> > > > > >>>>>>>>>>>>>>>>>>>> situations
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> where
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> want
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> abort the transactions. We
> > > > > >>>>>>> can
> > > > > >>>>>>>>>> even
> > > > > >>>>>>>>>>>> use on
> > > > > >>>>>>>>>>>>>>>>> other
> > > > > >>>>>>>>>>>>>>>>>>>> apis.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've added this to the
> > > > > >>>>>> KIP.
> > > > > >>>>>>>> Let
> > > > > >>>>>>>>> me
> > > > > >>>>>>>>>>>> know if
> > > > > >>>>>>>>>>>>>>>>> there
> > > > > >>>>>>>>>>>>>>>>>>>> are
> > > > > >>>>>>>>>>>>>>>>>>>>> any
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> questions
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> or
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 2, 2022 at
> > > > > >>>>>> 10:22
> > > > > >>>>>>>> AM
> > > > > >>>>>>>>>>>> Justine
> > > > > >>>>>>>>>>>>>>>> Olshan
> > > > > >>>>>>>>>>>>>>>>> <
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> jolshan@confluent.io
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey Matthias,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30 — Maybe I also
> > > > > >>>>>> didn't
> > > > > >>>>>>>>>> express
> > > > > >>>>>>>>>>>>>>> myself
> > > > > >>>>>>>>>>>>>>>>>>>> clearly.
> > > > > >>>>>>>>>>>>>>>>>>>>> For
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't have a way to
> > > > > >>>>>>>> distinguish
> > > > > >>>>>>>>>>>> between a
> > > > > >>>>>>>>>>>>>>>>>> previous
> > > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> current
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction since we
> > > > > >>>>>> don't
> > > > > >>>>>>>> have
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>> epoch
> > > > > >>>>>>>>>>>>>>>>> bump.
> > > > > >>>>>>>>>>>>>>>>>>>> This
> > > > > >>>>>>>>>>>>>>>>>>>>>>> means
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message from the previous
> > > > > >>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>> may be
> > > > > >>>>>>>>>>>>>>>>>>>> added to
> > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> one.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> With
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older clients — we can't
> > > > > >>>>>>>>>> guarantee
> > > > > >>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>> won't
> > > > > >>>>>>>>>>>>>>>>>>>> happen
> > > > > >>>>>>>>>>>>>>>>>>>>>>> if we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> already
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call
> > > > > >>>>>>> (why
> > > > > >>>>>>>> we
> > > > > >>>>>>>>>> make
> > > > > >>>>>>>>>>>>>>> changes
> > > > > >>>>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> newer
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> client)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can at least gate some by
> > > > > >>>>>>>>>> ensuring
> > > > > >>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>> partition
> > > > > >>>>>>>>>>>>>>>>>>>>>>> has
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> been
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> added
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. The
> > > > > >>>>>> rationale
> > > > > >>>>>>>> here
> > > > > >>>>>>>>>> is
> > > > > >>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>>> there
> > > > > >>>>>>>>>>>>>>>>>> are
> > > > > >>>>>>>>>>>>>>>>>>>>>>> likely
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> LESS
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrivals
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as time goes on, so
> > > > > >>>>>>> hopefully
> > > > > >>>>>>>>>> most
> > > > > >>>>>>>>>>>> late
> > > > > >>>>>>>>>>>>>>>>> arrivals
> > > > > >>>>>>>>>>>>>>>>>>>> will
> > > > > >>>>>>>>>>>>>>>>>>>>>>> come
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> BEFORE
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call.
> > > > > >>>>>>>> Those
> > > > > >>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>> arrive
> > > > > >>>>>>>>>>>>>>>>>> before
> > > > > >>>>>>>>>>>>>>>>>>>>> will
> > > > > >>>>>>>>>>>>>>>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> properly
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> gated
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> > > > > >>>>>>> describeTransactions
> > > > > >>>>>>>>>>>> approach.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we take the approach
> > > > > >>>>>> you
> > > > > >>>>>>>>>>>> suggested,
> > > > > >>>>>>>>>>>>>>> ANY
> > > > > >>>>>>>>>>>>>>>>> late
> > > > > >>>>>>>>>>>>>>>>>>>>> arrival
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> from a
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> previous
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction will be
> > > > > >>>>>> added.
> > > > > >>>>>>>> And
> > > > > >>>>>>>>> we
> > > > > >>>>>>>>>>>> don't
> > > > > >>>>>>>>>>>>>>> want
> > > > > >>>>>>>>>>>>>>>>>>>> that. I
> > > > > >>>>>>>>>>>>>>>>>>>>>>> also
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> see
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> benefit in sending
> > > > > >>>>>>>>>> addPartitionsToTxn
> > > > > >>>>>>>>>>>>>>> over
> > > > > >>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> describeTxns
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> They
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> both be one extra RPC to
> > > > > >>>>>>> the
> > > > > >>>>>>>>> Txn
> > > > > >>>>>>>>>>>>>>>> coordinator.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To be clear — newer
> > > > > >>>>>> clients
> > > > > >>>>>>>>> will
> > > > > >>>>>>>>>> use
> > > > > >>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> instead
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTxns.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that if we
> > > > > >>>>>>> have
> > > > > >>>>>>>>>> some
> > > > > >>>>>>>>>>>> delay
> > > > > >>>>>>>>>>>>>>> in
> > > > > >>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>> client
> > > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> bump
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it could continue to send
> > > > > >>>>>>>> epoch
> > > > > >>>>>>>>>> 73
> > > > > >>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>> those
> > > > > >>>>>>>>>>>>>>>>>>>> records
> > > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fenced.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Perhaps this is not an
> > > > > >>>>>>> issue
> > > > > >>>>>>>> if
> > > > > >>>>>>>>>> we
> > > > > >>>>>>>>>>>> don't
> > > > > >>>>>>>>>>>>>>>> allow
> > > > > >>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>> next
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> produce
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> go
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through before the EndTxn
> > > > > >>>>>>>>> request
> > > > > >>>>>>>>>>>>>>> returns.
> > > > > >>>>>>>>>>>>>>>> I'm
> > > > > >>>>>>>>>>>>>>>>>>>> also
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> thinking
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> about
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cases of
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure. I will need to
> > > > > >>>>>>> think
> > > > > >>>>>>>>> on
> > > > > >>>>>>>>>>>> this a
> > > > > >>>>>>>>>>>>>>> bit.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I wasn't sure if it was
> > > > > >>>>>>> that
> > > > > >>>>>>>>>>>> confusing.
> > > > > >>>>>>>>>>>>>>> But
> > > > > >>>>>>>>>>>>>>>> if
> > > > > >>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>> think it
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> is,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> investigate other ways.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure these are
> > > > > >>>>>> the
> > > > > >>>>>>>> same
> > > > > >>>>>>>>>>>>>>> purgatories
> > > > > >>>>>>>>>>>>>>>>>> since
> > > > > >>>>>>>>>>>>>>>>>>>> one
> > > > > >>>>>>>>>>>>>>>>>>>>>>> is a
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> produce
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory (I was planning
> > > > > >>>>>>> on
> > > > > >>>>>>>>>> using a
> > > > > >>>>>>>>>>>>>>>> callback
> > > > > >>>>>>>>>>>>>>>>>>>> rather
> > > > > >>>>>>>>>>>>>>>>>>>>>>> than
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the other is simply a
> > > > > >>>>>>> request
> > > > > >>>>>>>>> to
> > > > > >>>>>>>>>>>> append
> > > > > >>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>> log.
> > > > > >>>>>>>>>>>>>>>>>>>>> Not
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> sure
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> structure here for
> > > > > >>>>>>> ordering,
> > > > > >>>>>>>>> but
> > > > > >>>>>>>>>> my
> > > > > >>>>>>>>>>>>>>>>>> understanding
> > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handle the write request
> > > > > >>>>>>>> before
> > > > > >>>>>>>>>> it
> > > > > >>>>>>>>>>>> hears
> > > > > >>>>>>>>>>>>>>>> back
> > > > > >>>>>>>>>>>>>>>>>> from
> > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> Txn
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Coordinator.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if I
> > > > > >>>>>>>> misunderstood
> > > > > >>>>>>>>>>>> something
> > > > > >>>>>>>>>>>>>>> or
> > > > > >>>>>>>>>>>>>>>>>>>> something
> > > > > >>>>>>>>>>>>>>>>>>>>>>> was
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> unclear.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 1, 2022 at
> > > > > >>>>>>> 12:15
> > > > > >>>>>>>> PM
> > > > > >>>>>>>>>>>> Matthias
> > > > > >>>>>>>>>>>>>>> J.
> > > > > >>>>>>>>>>>>>>>>> Sax
> > > > > >>>>>>>>>>>>>>>>>> <
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the details
> > > > > >>>>>>>>> Justine!
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side change
> > > > > >>>>>>> for
> > > > > >>>>>>>> 2
> > > > > >>>>>>>>> is
> > > > > >>>>>>>>>>>>>>> removing
> > > > > >>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> addPartitions
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need to
> > > > > >>>>>>> make
> > > > > >>>>>>>>>> this
> > > > > >>>>>>>>>>>> from
> > > > > >>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>> producer
> > > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think I did not
> > > > > >>>>>> express
> > > > > >>>>>>>>> myself
> > > > > >>>>>>>>>>>>>>> clearly. I
> > > > > >>>>>>>>>>>>>>>>>>>>> understand
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (and
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should) change the
> > > > > >>>>>>> producer
> > > > > >>>>>>>> to
> > > > > >>>>>>>>>> not
> > > > > >>>>>>>>>>>> send
> > > > > >>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> longer. But I don't
> > > > > >>>>>> thinks
> > > > > >>>>>>>>> it's
> > > > > >>>>>>>>>>>>>>> requirement
> > > > > >>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>> change
> > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> broker?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What I am trying to say
> > > > > >>>>>>> is:
> > > > > >>>>>>>>> as a
> > > > > >>>>>>>>>>>>>>> safe-guard
> > > > > >>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> improvement
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producers, the partition
> > > > > >>>>>>>>> leader
> > > > > >>>>>>>>>> can
> > > > > >>>>>>>>>>>> just
> > > > > >>>>>>>>>>>>>>>> send
> > > > > >>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request to the
> > > > > >>>>>>>> TX-coordinator
> > > > > >>>>>>>>>> in any
> > > > > >>>>>>>>>>>>>>> case
> > > > > >>>>>>>>>>>>>>>> --
> > > > > >>>>>>>>>>>>>>>>> if
> > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> old
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> correctly did send the
> > > > > >>>>>>>>>>>> `addPartition`
> > > > > >>>>>>>>>>>>>>>> request
> > > > > >>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> already, the
> > > > > >>>>>>> TX-coordinator
> > > > > >>>>>>>>> can
> > > > > >>>>>>>>>> just
> > > > > >>>>>>>>>>>>>>>> "ignore"
> > > > > >>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>> as
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> idempotent.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if the old producer has
> > > > > >>>>>> a
> > > > > >>>>>>>> bug
> > > > > >>>>>>>>>> and
> > > > > >>>>>>>>>>>> did
> > > > > >>>>>>>>>>>>>>>> forget
> > > > > >>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>> sent
> > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `addPartition`
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request, we would now
> > > > > >>>>>>> ensure
> > > > > >>>>>>>>>> that
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>> partition
> > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>>> indeed
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> added
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX and thus fix a
> > > > > >>>>>>> potential
> > > > > >>>>>>>>>>>> producer bug
> > > > > >>>>>>>>>>>>>>>>> (even
> > > > > >>>>>>>>>>>>>>>>>>>> if we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> get
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fencing via the bump
> > > > > >>>>>>> epoch).
> > > > > >>>>>>>>> --
> > > > > >>>>>>>>>> It
> > > > > >>>>>>>>>>>>>>> seems to
> > > > > >>>>>>>>>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>>>>>>>> good
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Or
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there a reason to not do
> > > > > >>>>>>>> this?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is ongoing
> > > > > >>>>>> =
> > > > > >>>>>>>>>> partition
> > > > > >>>>>>>>>>>> was
> > > > > >>>>>>>>>>>>>>>> added
> > > > > >>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> via
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn. We
> > > > > >>>>>>>> check
> > > > > >>>>>>>>>> this
> > > > > >>>>>>>>>>>> with
> > > > > >>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this wasn't
> > > > > >>>>>>>>>> sufficiently
> > > > > >>>>>>>>>>>>>>>>> explained
> > > > > >>>>>>>>>>>>>>>>>>>> here:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do what I propose
> > > > > >>>>>> in
> > > > > >>>>>>>>>> (20), we
> > > > > >>>>>>>>>>>>>>> don't
> > > > > >>>>>>>>>>>>>>>>>> really
> > > > > >>>>>>>>>>>>>>>>>>>>> need
> > > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> make
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `DescribeTransaction`
> > > > > >>>>>>> call,
> > > > > >>>>>>>> as
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>> partition
> > > > > >>>>>>>>>>>>>>>>>>>> leader
> > > > > >>>>>>>>>>>>>>>>>>>>>>> adds
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for older clients and we
> > > > > >>>>>>> get
> > > > > >>>>>>>>>> this
> > > > > >>>>>>>>>>>> check
> > > > > >>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>> free.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is that
> > > > > >>>>>> if
> > > > > >>>>>>>> any
> > > > > >>>>>>>>>>>> messages
> > > > > >>>>>>>>>>>>>>>>> somehow
> > > > > >>>>>>>>>>>>>>>>>>>> come
> > > > > >>>>>>>>>>>>>>>>>>>>> in
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> before
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the producer,
> > > > > >>>>>>> they
> > > > > >>>>>>>>>> will be
> > > > > >>>>>>>>>>>>>>>> fenced.
> > > > > >>>>>>>>>>>>>>>>>>>>> However,
> > > > > >>>>>>>>>>>>>>>>>>>>>>> if
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it can be
> > > > > >>>>>>>>>> discussed
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree that we should
> > > > > >>>>>>> have
> > > > > >>>>>>>>>> epoch
> > > > > >>>>>>>>>>>>>>> fencing.
> > > > > >>>>>>>>>>>>>>>> My
> > > > > >>>>>>>>>>>>>>>>>>>>>>> question is
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> different:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Assume we are at epoch
> > > > > >>>>>> 73,
> > > > > >>>>>>>> and
> > > > > >>>>>>>>>> we
> > > > > >>>>>>>>>>>> have
> > > > > >>>>>>>>>>>>>>> an
> > > > > >>>>>>>>>>>>>>>>>> ongoing
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed. It seems
> > > > > >>>>>>> natural
> > > > > >>>>>>>> to
> > > > > >>>>>>>>>>>> write the
> > > > > >>>>>>>>>>>>>>>>>> "prepare
> > > > > >>>>>>>>>>>>>>>>>>>>>>> commit"
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> marker
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
> > > > > >>>>>>> both
> > > > > >>>>>>>>> with
> > > > > >>>>>>>>>>>> epoch
> > > > > >>>>>>>>>>>>>>> 73,
> > > > > >>>>>>>>>>>>>>>>> too,
> > > > > >>>>>>>>>>>>>>>>>>>> as
> > > > > >>>>>>>>>>>>>>>>>>>>> it
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> belongs
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current transaction. Of
> > > > > >>>>>>>>> course,
> > > > > >>>>>>>>>> we
> > > > > >>>>>>>>>>>> now
> > > > > >>>>>>>>>>>>>>> also
> > > > > >>>>>>>>>>>>>>>>>> bump
> > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> expect
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the next requests to
> > > > > >>>>>> have
> > > > > >>>>>>>>> epoch
> > > > > >>>>>>>>>> 74,
> > > > > >>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>> would
> > > > > >>>>>>>>>>>>>>>>>>>> reject
> > > > > >>>>>>>>>>>>>>>>>>>>>>> an
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch 73, as the
> > > > > >>>>>>>> corresponding
> > > > > >>>>>>>>>> TX
> > > > > >>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>> epoch
> > > > > >>>>>>>>>>>>>>>>> 73
> > > > > >>>>>>>>>>>>>>>>>>>> was
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> already
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems you propose to
> > > > > >>>>>>>> write
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>> "prepare
> > > > > >>>>>>>>>>>>>>>>>> commit
> > > > > >>>>>>>>>>>>>>>>>>>>>>> marker"
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
> > > > > >>>>>>> with
> > > > > >>>>>>>>>> epoch 74
> > > > > >>>>>>>>>>>>>>>> though,
> > > > > >>>>>>>>>>>>>>>>>> what
> > > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> work,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems confusing. Is
> > > > > >>>>>> there
> > > > > >>>>>>> a
> > > > > >>>>>>>>>> reason
> > > > > >>>>>>>>>>>> why
> > > > > >>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>> would
> > > > > >>>>>>>>>>>>>>>>>>>> use
> > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> bumped
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 74
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of the current
> > > > > >>>>>>> epoch
> > > > > >>>>>>>>> 73?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are checking if
> > > > > >>>>>>> the
> > > > > >>>>>>>>>>>>>>> transaction is
> > > > > >>>>>>>>>>>>>>>>>>>> ongoing,
> > > > > >>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the leader
> > > > > >>>>>>>>> partition
> > > > > >>>>>>>>>> to
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for this
> > > > > >>>>>>>>>> message to
> > > > > >>>>>>>>>>>> come
> > > > > >>>>>>>>>>>>>>>>> back,
> > > > > >>>>>>>>>>>>>>>>>> in
> > > > > >>>>>>>>>>>>>>>>>>>>>>> theory
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
> > > > > >>>>>> that
> > > > > >>>>>>>>> would
> > > > > >>>>>>>>>>>> make the
> > > > > >>>>>>>>>>>>>>>>>> original
> > > > > >>>>>>>>>>>>>>>>>>>>>>> result
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out of
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why we
> > > > > >>>>>> can
> > > > > >>>>>>>>> check
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>> leader
> > > > > >>>>>>>>>>>>>>>>>> state
> > > > > >>>>>>>>>>>>>>>>>>>>>>> before
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks. Got it.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, is this really
> > > > > >>>>>> an
> > > > > >>>>>>>>>> issue?
> > > > > >>>>>>>>>>>> We put
> > > > > >>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>> produce
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory, so how could
> > > > > >>>>>> we
> > > > > >>>>>>>>>> process
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> first?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Don't we need to put the
> > > > > >>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
> > > > > >>>>>>>>>>>>>>>>>>>>> into
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> too,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for this case, and
> > > > > >>>>>> process
> > > > > >>>>>>>>> both
> > > > > >>>>>>>>>>>> request
> > > > > >>>>>>>>>>>>>>>>>> in-order?
> > > > > >>>>>>>>>>>>>>>>>>>>>>> (Again,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> my
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> knowledge is limited and
> > > > > >>>>>>>> maybe
> > > > > >>>>>>>>>> we
> > > > > >>>>>>>>>>>> don't
> > > > > >>>>>>>>>>>>>>>>>> maintain
> > > > > >>>>>>>>>>>>>>>>>>>>>>> request
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> order
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case, what seems to be
> > > > > >>>>>> an
> > > > > >>>>>>>>> issue
> > > > > >>>>>>>>>>>> IMHO,
> > > > > >>>>>>>>>>>>>>> and I
> > > > > >>>>>>>>>>>>>>>>> am
> > > > > >>>>>>>>>>>>>>>>>>>>>>> wondering
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> changing
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request handling to
> > > > > >>>>>>> preserve
> > > > > >>>>>>>>>> order
> > > > > >>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>> case
> > > > > >>>>>>>>>>>>>>>>>>>>>>> might be
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> cleaner
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution?)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/30/22 3:28 PM,
> > > > > >>>>>> Artem
> > > > > >>>>>>>>>> Livshits
> > > > > >>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think the interesting
> > > > > >>>>>>>> part
> > > > > >>>>>>>>> is
> > > > > >>>>>>>>>>>> not in
> > > > > >>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>> logic
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> (because
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> tries
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> figure out when
> > > > > >>>>>>>>>>>> UNKNOWN_PRODUCER_ID is
> > > > > >>>>>>>>>>>>>>>>>> retriable
> > > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>>> if
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> it's
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retryable,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's definitely not
> > > > > >>>>>>> fatal),
> > > > > >>>>>>>>> but
> > > > > >>>>>>>>>>>> what
> > > > > >>>>>>>>>>>>>>>> happens
> > > > > >>>>>>>>>>>>>>>>>>>> when
> > > > > >>>>>>>>>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> logic
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'true' and falls
> > > > > >>>>>> through.
> > > > > >>>>>>>> In
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>> old
> > > > > >>>>>>>>>>>>>>>>> clients
> > > > > >>>>>>>>>>>>>>>>>> it
> > > > > >>>>>>>>>>>>>>>>>>>>>>> seems
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> keep the behavior in
> > > > > >>>>>> the
> > > > > >>>>>>>> new
> > > > > >>>>>>>>>>>> clients,
> > > > > >>>>>>>>>>>>>>> I'd
> > > > > >>>>>>>>>>>>>>>>>>>> expect it
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> would
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> well.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 29, 2022 at
> > > > > >>>>>>>> 11:57
> > > > > >>>>>>>>>> AM
> > > > > >>>>>>>>>>>>>>> Justine
> > > > > >>>>>>>>>>>>>>>>>> Olshan
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>> <jolshan@confluent.io.invalid
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Artem and Jeff,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
> > > > > >>>>>> look
> > > > > >>>>>>>> and
> > > > > >>>>>>>>>>>> sorry for
> > > > > >>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>> slow
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> response.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You both mentioned the
> > > > > >>>>>>>>> change
> > > > > >>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>> handle
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear — this error
> > > > > >>>>>> code
> > > > > >>>>>>>> will
> > > > > >>>>>>>>>> only
> > > > > >>>>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>> sent
> > > > > >>>>>>>>>>>>>>>>>> again
> > > > > >>>>>>>>>>>>>>>>>>>>> when
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> client's
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version is high enough
> > > > > >>>>>>> to
> > > > > >>>>>>>>>> ensure
> > > > > >>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>> handle
> > > > > >>>>>>>>>>>>>>>>> it
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> correctly.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The current (Java)
> > > > > >>>>>>> client
> > > > > >>>>>>>>>> handles
> > > > > >>>>>>>>>>>>>>> this by
> > > > > >>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> following
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> (somewhat
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> long)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code snippet:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // An
> > > > > >>>>>>> UNKNOWN_PRODUCER_ID
> > > > > >>>>>>>>>> means
> > > > > >>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>> have
> > > > > >>>>>>>>>>>>>>>>>>>> lost
> > > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker. Depending on
> > > > > >>>>>> the
> > > > > >>>>>>>> log
> > > > > >>>>>>>>>> start
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // offset, we may want
> > > > > >>>>>>> to
> > > > > >>>>>>>>>> retry
> > > > > >>>>>>>>>>>>>>> these, as
> > > > > >>>>>>>>>>>>>>>>>>>>> described
> > > > > >>>>>>>>>>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> each
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> case
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below. If
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> none of those apply,
> > > > > >>>>>>> then
> > > > > >>>>>>>>> for
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // idempotent
> > > > > >>>>>> producer,
> > > > > >>>>>>> we
> > > > > >>>>>>>>>> will
> > > > > >>>>>>>>>>>>>>> locally
> > > > > >>>>>>>>>>>>>>>>> bump
> > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> reset
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence numbers of
> > > > > >>>>>>>>> in-flight
> > > > > >>>>>>>>>>>> batches
> > > > > >>>>>>>>>>>>>>>> from
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // sequence 0, then
> > > > > >>>>>>> retry
> > > > > >>>>>>>>> the
> > > > > >>>>>>>>>>>> failed
> > > > > >>>>>>>>>>>>>>>> batch,
> > > > > >>>>>>>>>>>>>>>>>>>> which
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> should
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> now
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> succeed.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the transactional
> > > > > >>>>>>>> producer,
> > > > > >>>>>>>>>> allow
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // batch to fail. When
> > > > > >>>>>>>>>> processing
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>> failed
> > > > > >>>>>>>>>>>>>>>>>>>>> batch,
> > > > > >>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transition
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an abortable error and
> > > > > >>>>>>>> set a
> > > > > >>>>>>>>>> flag
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // indicating that we
> > > > > >>>>>>> need
> > > > > >>>>>>>>> to
> > > > > >>>>>>>>>>>> bump the
> > > > > >>>>>>>>>>>>>>>>> epoch
> > > > > >>>>>>>>>>>>>>>>>>>> (if
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> supported
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> by
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker).
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if (error ==
> > > > > >>>>>>>>>>>>>>>> Errors.*UNKNOWN_PRODUCER_ID*)
> > > > > >>>>>>>>>>>>>>>>> {
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > > > > >>>>>>>>>> (response.logStartOffset
> > > > > >>>>>>>>>>>> ==
> > > > > >>>>>>>>>>>>>>> -1)
> > > > > >>>>>>>>>>>>>>>> {
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // We don't
> > > > > >>>>>>> know
> > > > > >>>>>>>>>> the log
> > > > > >>>>>>>>>>>>>>> start
> > > > > >>>>>>>>>>>>>>>>>> offset
> > > > > >>>>>>>>>>>>>>>>>>>>> with
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> response.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the request
> > > > > >>>>>>>> until
> > > > > >>>>>>>>>> we
> > > > > >>>>>>>>>>>> get
> > > > > >>>>>>>>>>>>>>> it.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // The
> > > > > >>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > >>>>>>>>>>>>>>>> error
> > > > > >>>>>>>>>>>>>>>>>> code
> > > > > >>>>>>>>>>>>>>>>>>>>> was
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> added
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> along
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProduceResponse which
> > > > > >>>>>>>>>> includes the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              //
> > > > > >>>>>>>> logStartOffset.
> > > > > >>>>>>>>>> So
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>> '-1'
> > > > > >>>>>>>>>>>>>>>>>>>> sentinel
> > > > > >>>>>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> backward
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatibility.
> > > > > >>>>>> Instead,
> > > > > >>>>>>> it
> > > > > >>>>>>>>> is
> > > > > >>>>>>>>>>>> possible
> > > > > >>>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // a broker
> > > > > >>>>>> to
> > > > > >>>>>>>> not
> > > > > >>>>>>>>>> know
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> logStartOffset at
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> when
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returning
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response because
> > > > > >>>>>> the
> > > > > >>>>>>>>>> partition
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // may have
> > > > > >>>>>>>> moved
> > > > > >>>>>>>>>> away
> > > > > >>>>>>>>>>>> from
> > > > > >>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>> broker
> > > > > >>>>>>>>>>>>>>>>>>>>>>> from
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error was
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initially raised to
> > > > > >>>>>> the
> > > > > >>>>>>>> time
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // response
> > > > > >>>>>>> was
> > > > > >>>>>>>>>> being
> > > > > >>>>>>>>>>>>>>>>> constructed.
> > > > > >>>>>>>>>>>>>>>>>> In
> > > > > >>>>>>>>>>>>>>>>>>>>>>> these
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> cases,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retry the request: we
> > > > > >>>>>>> are
> > > > > >>>>>>>>>>>> guaranteed
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // to
> > > > > >>>>>>> eventually
> > > > > >>>>>>>>>> get a
> > > > > >>>>>>>>>>>>>>>>>> logStartOffset
> > > > > >>>>>>>>>>>>>>>>>>>>> once
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> things
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> settle
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> down.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > > > > >>>>>>>>>>>>>>> (batch.sequenceHasBeenReset()) {
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // When the
> > > > > >>>>>>>> first
> > > > > >>>>>>>>>>>> inflight
> > > > > >>>>>>>>>>>>>>>> batch
> > > > > >>>>>>>>>>>>>>>>>>>> fails
> > > > > >>>>>>>>>>>>>>>>>>>>>>> due to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> truncation
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> then the sequences of
> > > > > >>>>>>> all
> > > > > >>>>>>>>> the
> > > > > >>>>>>>>>>>> other
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // in flight
> > > > > >>>>>>>>> batches
> > > > > >>>>>>>>>>>> would
> > > > > >>>>>>>>>>>>>>> have
> > > > > >>>>>>>>>>>>>>>>>> been
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> restarted
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> from
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> beginning.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, when those
> > > > > >>>>>>>>> responses
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // come back
> > > > > >>>>>>>> from
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>> broker,
> > > > > >>>>>>>>>>>>>>>>> they
> > > > > >>>>>>>>>>>>>>>>>>>> would
> > > > > >>>>>>>>>>>>>>>>>>>>>>> also
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> come
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > >>>>>>> error.
> > > > > >>>>>>>>> In
> > > > > >>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>> case,
> > > > > >>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>> should
> > > > > >>>>>>>>>>>>>>>>>>>>>>> not
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // reset the
> > > > > >>>>>>>>>> sequence
> > > > > >>>>>>>>>>>>>>> numbers
> > > > > >>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> beginning.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          } else if
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>> (lastAckedOffset(batch.topicPartition).orElse(
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> response.logStartOffset) {
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // The head
> > > > > >>>>>> of
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>> log
> > > > > >>>>>>>>>>>> has
> > > > > >>>>>>>>>>>>>>> been
> > > > > >>>>>>>>>>>>>>>>>>>> removed,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> probably
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> due
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retention time
> > > > > >>>>>> elapsing.
> > > > > >>>>>>>> In
> > > > > >>>>>>>>>> this
> > > > > >>>>>>>>>>>> case,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // we expect
> > > > > >>>>>>> to
> > > > > >>>>>>>>>> lose the
> > > > > >>>>>>>>>>>>>>>> producer
> > > > > >>>>>>>>>>>>>>>>>>>> state.
> > > > > >>>>>>>>>>>>>>>>>>>>>>> For
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer, reset the
> > > > > >>>>>>>>> sequences
> > > > > >>>>>>>>>> of
> > > > > >>>>>>>>>>>> all
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // inflight
> > > > > >>>>>>>>> batches
> > > > > >>>>>>>>>> to
> > > > > >>>>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>> from
> > > > > >>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> beginning
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> retry
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the transaction
> > > > > >>>>>>> does
> > > > > >>>>>>>>> not
> > > > > >>>>>>>>>>>> need to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // be
> > > > > >>>>>> aborted.
> > > > > >>>>>>>> For
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>> idempotent
> > > > > >>>>>>>>>>>>>>>>>>>>>>> producer,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> bump
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> avoid
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reusing (sequence,
> > > > > >>>>>>> epoch)
> > > > > >>>>>>>>>> pairs
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              if
> > > > > >>>>>>>>>> (isTransactional()) {
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>> this.producerIdAndEpoch);
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              } else {
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> requestEpochBumpForPartition(batch.topicPartition);
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              }
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > > > > >>>>>>>>> (!isTransactional())
> > > > > >>>>>>>>>> {
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // For the
> > > > > >>>>>>>>>> idempotent
> > > > > >>>>>>>>>>>>>>> producer,
> > > > > >>>>>>>>>>>>>>>>>>>> always
> > > > > >>>>>>>>>>>>>>>>>>>>>>> retry
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors. If the batch
> > > > > >>>>>> has
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>>>> current
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // producer
> > > > > >>>>>> ID
> > > > > >>>>>>>> and
> > > > > >>>>>>>>>>>> epoch,
> > > > > >>>>>>>>>>>>>>>>> request a
> > > > > >>>>>>>>>>>>>>>>>>>> bump
> > > > > >>>>>>>>>>>>>>>>>>>>>>> of
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Otherwise
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the
> > > > > >>>>>> produce.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>> requestEpochBumpForPartition(batch.topicPartition);
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I was considering
> > > > > >>>>>>> keeping
> > > > > >>>>>>>>> this
> > > > > >>>>>>>>>>>>>>> behavior —
> > > > > >>>>>>>>>>>>>>>>> but
> > > > > >>>>>>>>>>>>>>>>>>>> am
> > > > > >>>>>>>>>>>>>>>>>>>>>>> open
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simplifying
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We are leaving changes
> > > > > >>>>>>> to
> > > > > >>>>>>>>>> older
> > > > > >>>>>>>>>>>>>>> clients
> > > > > >>>>>>>>>>>>>>>> off
> > > > > >>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> table
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> here
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> since
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> caused
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many issues for
> > > > > >>>>>> clients
> > > > > >>>>>>> in
> > > > > >>>>>>>>> the
> > > > > >>>>>>>>>>>> past.
> > > > > >>>>>>>>>>>>>>>>>> Previously
> > > > > >>>>>>>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> was
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we didn't have the
> > > > > >>>>>>>>> mechanisms
> > > > > >>>>>>>>>> in
> > > > > >>>>>>>>>>>>>>> place to
> > > > > >>>>>>>>>>>>>>>>>>>> detect
> > > > > >>>>>>>>>>>>>>>>>>>>>>> when
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> was
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> legitimate
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case vs some bug or
> > > > > >>>>>> gap
> > > > > >>>>>>> in
> > > > > >>>>>>>>> the
> > > > > >>>>>>>>>>>>>>> protocol.
> > > > > >>>>>>>>>>>>>>>>>>>> Ensuring
> > > > > >>>>>>>>>>>>>>>>>>>>>>> each
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> its
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> own epoch should close
> > > > > >>>>>>>> this
> > > > > >>>>>>>>>> gap.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> And to address Jeff's
> > > > > >>>>>>>> second
> > > > > >>>>>>>>>>>> point:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *does the typical
> > > > > >>>>>>> produce
> > > > > >>>>>>>>>> request
> > > > > >>>>>>>>>>>> path
> > > > > >>>>>>>>>>>>>>>>> append
> > > > > >>>>>>>>>>>>>>>>>>>>>>> records
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> local
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> log
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along*
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *with the
> > > > > >>>>>>>>>> currentTxnFirstOffset
> > > > > >>>>>>>>>>>>>>>>> information?
> > > > > >>>>>>>>>>>>>>>>>> I
> > > > > >>>>>>>>>>>>>>>>>>>>> would
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> like
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand*
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *when the field is
> > > > > >>>>>>> written
> > > > > >>>>>>>>> to
> > > > > >>>>>>>>>>>> disk.*
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, the first produce
> > > > > >>>>>>>>> request
> > > > > >>>>>>>>>>>>>>> populates
> > > > > >>>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>>> field
> > > > > >>>>>>>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> writes
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> offset
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as part of the record
> > > > > >>>>>>>> batch
> > > > > >>>>>>>>>> and
> > > > > >>>>>>>>>>>> also
> > > > > >>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>> producer
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> state
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> snapshot.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we reload the records
> > > > > >>>>>> on
> > > > > >>>>>>>>>> restart
> > > > > >>>>>>>>>>>>>>> and/or
> > > > > >>>>>>>>>>>>>>>>>>>>>>> reassignment,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> repopulate
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> field with the
> > > > > >>>>>> snapshot
> > > > > >>>>>>>> from
> > > > > >>>>>>>>>> disk
> > > > > >>>>>>>>>>>>>>> along
> > > > > >>>>>>>>>>>>>>>>> with
> > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> rest
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if there
> > > > > >>>>>> are
> > > > > >>>>>>>>>> further
> > > > > >>>>>>>>>>>>>>> comments
> > > > > >>>>>>>>>>>>>>>>>>>> and/or
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> questions.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > > > > >>>>>> at
> > > > > >>>>>>>> 9:00
> > > > > >>>>>>>>>> PM
> > > > > >>>>>>>>>>>> Jeff
> > > > > >>>>>>>>>>>>>>> Kim
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>> <jeff.kim@confluent.io.invalid
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP! I
> > > > > >>>>>>>> have
> > > > > >>>>>>>>>> two
> > > > > >>>>>>>>>>>>>>>> questions:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1) For new clients,
> > > > > >>>>>> we
> > > > > >>>>>>>> can
> > > > > >>>>>>>>>> once
> > > > > >>>>>>>>>>>> again
> > > > > >>>>>>>>>>>>>>>>> return
> > > > > >>>>>>>>>>>>>>>>>>>> an
> > > > > >>>>>>>>>>>>>>>>>>>>>>> error
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for sequences
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that are non-zero
> > > > > >>>>>> when
> > > > > >>>>>>>>> there
> > > > > >>>>>>>>>> is
> > > > > >>>>>>>>>>>> no
> > > > > >>>>>>>>>>>>>>>>> producer
> > > > > >>>>>>>>>>>>>>>>>>>> state
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> present
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> on
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This will indicate we
> > > > > >>>>>>>>> missed
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>> 0
> > > > > >>>>>>>>>>>>>>>>> sequence
> > > > > >>>>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> yet
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> want
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the log.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would like to
> > > > > >>>>>>>> understand
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>> current
> > > > > >>>>>>>>>>>>>>>>>>>> behavior
> > > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> handle
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and if there are any
> > > > > >>>>>>>>> changes
> > > > > >>>>>>>>>> we
> > > > > >>>>>>>>>>>> are
> > > > > >>>>>>>>>>>>>>>>> making.
> > > > > >>>>>>>>>>>>>>>>>>>> Maybe
> > > > > >>>>>>>>>>>>>>>>>>>>>>> I'm
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> missing
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but we would want to
> > > > > >>>>>>>>> identify
> > > > > >>>>>>>>>>>>>>> whether we
> > > > > >>>>>>>>>>>>>>>>>>>> missed
> > > > > >>>>>>>>>>>>>>>>>>>>>>> the 0
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> sequence
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients, no?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2) Upon returning
> > > > > >>>>>> from
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>>>>>>>>>> coordinator, we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> set
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as ongoing on the
> > > > > >>>>>>> leader
> > > > > >>>>>>>> by
> > > > > >>>>>>>>>>>>>>> populating
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> currentTxnFirstOffset
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through the typical
> > > > > >>>>>>>> produce
> > > > > >>>>>>>>>>>> request
> > > > > >>>>>>>>>>>>>>>>>> handling.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does the typical
> > > > > >>>>>>> produce
> > > > > >>>>>>>>>> request
> > > > > >>>>>>>>>>>> path
> > > > > >>>>>>>>>>>>>>>>> append
> > > > > >>>>>>>>>>>>>>>>>>>>>>> records
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> local
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> log
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> > > > > >>>>>>>>>> currentTxnFirstOffset
> > > > > >>>>>>>>>>>>>>>>> information?
> > > > > >>>>>>>>>>>>>>>>>> I
> > > > > >>>>>>>>>>>>>>>>>>>>> would
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> like
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when the field is
> > > > > >>>>>>> written
> > > > > >>>>>>>>> to
> > > > > >>>>>>>>>>>> disk.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeff
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > > > > >>>>>> at
> > > > > >>>>>>>>> 4:44
> > > > > >>>>>>>>>> PM
> > > > > >>>>>>>>>>>> Artem
> > > > > >>>>>>>>>>>>>>>>>> Livshits
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
> > > > > >>>>>>> alivshits@confluent.io
> > > > > >>>>>>>>>> .invalid>
> > > > > >>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for the
> > > > > >>>>>> KIP.
> > > > > >>>>>>>> I
> > > > > >>>>>>>>>> have
> > > > > >>>>>>>>>>>> one
> > > > > >>>>>>>>>>>>>>>>>> question.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 5) For new clients,
> > > > > >>>>>> we
> > > > > >>>>>>>> can
> > > > > >>>>>>>>>> once
> > > > > >>>>>>>>>>>>>>> again
> > > > > >>>>>>>>>>>>>>>>>> return
> > > > > >>>>>>>>>>>>>>>>>>>> an
> > > > > >>>>>>>>>>>>>>>>>>>>>>> error
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe we had
> > > > > >>>>>>>> problems
> > > > > >>>>>>>>>> in the
> > > > > >>>>>>>>>>>>>>> past
> > > > > >>>>>>>>>>>>>>>>> with
> > > > > >>>>>>>>>>>>>>>>>>>>>>> returning
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because it was
> > > > > >>>>>>>> considered
> > > > > >>>>>>>>>> fatal
> > > > > >>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>> required
> > > > > >>>>>>>>>>>>>>>>>>>>>>> client
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> restart.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> It
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good to spell out
> > > > > >>>>>> the
> > > > > >>>>>>>> new
> > > > > >>>>>>>>>> client
> > > > > >>>>>>>>>>>>>>>> behavior
> > > > > >>>>>>>>>>>>>>>>>>>> when
> > > > > >>>>>>>>>>>>>>>>>>>>> it
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> receives
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > > > > >>>>>>> at
> > > > > >>>>>>>>>> 10:00 AM
> > > > > >>>>>>>>>>>>>>>> Justine
> > > > > >>>>>>>>>>>>>>>>>>>> Olshan
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>> <jo...@confluent.io.invalid>
> > > > > >>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
> > > > > >>>>>>>> look
> > > > > >>>>>>>>>>>> Matthias.
> > > > > >>>>>>>>>>>>>>>> I've
> > > > > >>>>>>>>>>>>>>>>>>>> tried
> > > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> answer
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> your
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> questions
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 10)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Right — so the
> > > > > >>>>>>> hanging
> > > > > >>>>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>>> only
> > > > > >>>>>>>>>>>>>>>>>>>> occurs
> > > > > >>>>>>>>>>>>>>>>>>>>>>> when
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> come in and the
> > > > > >>>>>>>> partition
> > > > > >>>>>>>>>> is
> > > > > >>>>>>>>>>>> never
> > > > > >>>>>>>>>>>>>>>> added
> > > > > >>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> again.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never add the
> > > > > >>>>>>> partition
> > > > > >>>>>>>>> to
> > > > > >>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>>>> transaction,
> > > > > >>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>> will
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> never
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> marker
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never advance the
> > > > > >>>>>>> LSO.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do end up
> > > > > >>>>>>> adding
> > > > > >>>>>>>>> the
> > > > > >>>>>>>>>>>>>>> partition
> > > > > >>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (I
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suppose
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happen before or
> > > > > >>>>>>> after
> > > > > >>>>>>>>> the
> > > > > >>>>>>>>>> late
> > > > > >>>>>>>>>>>>>>>> message
> > > > > >>>>>>>>>>>>>>>>>>>> comes
> > > > > >>>>>>>>>>>>>>>>>>>>> in)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> then
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> include
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late message in the
> > > > > >>>>>>>> next
> > > > > >>>>>>>>>>>>>>> (incorrect)
> > > > > >>>>>>>>>>>>>>>>>>>>> transaction.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> So perhaps it is
> > > > > >>>>>>>> clearer
> > > > > >>>>>>>>> to
> > > > > >>>>>>>>>>>> make
> > > > > >>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>> distinction
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> between
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> eventually get
> > > > > >>>>>> added
> > > > > >>>>>>> to
> > > > > >>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>>>>> (but
> > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> wrong
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> one)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> or
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that never get
> > > > > >>>>>> added
> > > > > >>>>>>>> and
> > > > > >>>>>>>>>> become
> > > > > >>>>>>>>>>>>>>>> hanging.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side
> > > > > >>>>>>> change
> > > > > >>>>>>>>> for
> > > > > >>>>>>>>>> 2 is
> > > > > >>>>>>>>>>>>>>>> removing
> > > > > >>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> addPartitions
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need
> > > > > >>>>>>> to
> > > > > >>>>>>>>> make
> > > > > >>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>> from
> > > > > >>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> producer
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In my opinion, the
> > > > > >>>>>>>> issue
> > > > > >>>>>>>>>> with
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> call
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is that we don't
> > > > > >>>>>> have
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>> epoch
> > > > > >>>>>>>>>>>>>>> bump,
> > > > > >>>>>>>>>>>>>>>> so
> > > > > >>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>> don't
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> know
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> belongs to the
> > > > > >>>>>>> previous
> > > > > >>>>>>>>>>>>>>> transaction or
> > > > > >>>>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>>> one.
> > > > > >>>>>>>>>>>>>>>>>>>>>>> We
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition has been
> > > > > >>>>>>>> added
> > > > > >>>>>>>>> to
> > > > > >>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>> transaction.
> > > > > >>>>>>>>>>>>>>>>>>>>> Of
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> course,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> means
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't completely
> > > > > >>>>>>> cover
> > > > > >>>>>>>>> the
> > > > > >>>>>>>>>> case
> > > > > >>>>>>>>>>>>>>> where
> > > > > >>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>> have a
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> really
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have added the
> > > > > >>>>>>>> partition
> > > > > >>>>>>>>> to
> > > > > >>>>>>>>>>>> the new
> > > > > >>>>>>>>>>>>>>>>>>>>> transaction,
> > > > > >>>>>>>>>>>>>>>>>>>>>>> but
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> that's
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something we will
> > > > > >>>>>>> need
> > > > > >>>>>>>>> the
> > > > > >>>>>>>>>> new
> > > > > >>>>>>>>>>>>>>> clients
> > > > > >>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>> cover.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is
> > > > > >>>>>>> ongoing
> > > > > >>>>>>>> =
> > > > > >>>>>>>>>>>> partition
> > > > > >>>>>>>>>>>>>>> was
> > > > > >>>>>>>>>>>>>>>>>>>> added to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> via
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn.
> > > > > >>>>>>> We
> > > > > >>>>>>>>>> check
> > > > > >>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>> with
> > > > > >>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this
> > > > > >>>>>>> wasn't
> > > > > >>>>>>>>>>>> sufficiently
> > > > > >>>>>>>>>>>>>>>>>>>> explained
> > > > > >>>>>>>>>>>>>>>>>>>>>>> here:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>
> > > > > >>>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>
> > > > > >>>>
> > > > > >>
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is
> > > > > >>>>>> that
> > > > > >>>>>>>> if
> > > > > >>>>>>>>>> any
> > > > > >>>>>>>>>>>>>>> messages
> > > > > >>>>>>>>>>>>>>>>>>>> somehow
> > > > > >>>>>>>>>>>>>>>>>>>>>>> come
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> before
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the
> > > > > >>>>>>> producer,
> > > > > >>>>>>>>> they
> > > > > >>>>>>>>>>>> will be
> > > > > >>>>>>>>>>>>>>>>>> fenced.
> > > > > >>>>>>>>>>>>>>>>>>>>>>> However,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it
> > > > > >>>>>> can
> > > > > >>>>>>> be
> > > > > >>>>>>>>>>>> discussed
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 50)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It should be
> > > > > >>>>>>>> synchronous
> > > > > >>>>>>>>>>>> because
> > > > > >>>>>>>>>>>>>>> if we
> > > > > >>>>>>>>>>>>>>>>>> have
> > > > > >>>>>>>>>>>>>>>>>>>> an
> > > > > >>>>>>>>>>>>>>>>>>>>>>> event
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (ie,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> an
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes us to need
> > > > > >>>>>> to
> > > > > >>>>>>>>> abort
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>> transaction,
> > > > > >>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>> need
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions to send
> > > > > >>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>> markers
> > > > > >>>>>>>>>>>>>>>>> to.
> > > > > >>>>>>>>>>>>>>>>>>>> We
> > > > > >>>>>>>>>>>>>>>>>>>>>>> know
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we added them to
> > > > > >>>>>> the
> > > > > >>>>>>>>>>>> coordinator
> > > > > >>>>>>>>>>>>>>> via
> > > > > >>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Previously we have
> > > > > >>>>>>> had
> > > > > >>>>>>>>>>>> asynchronous
> > > > > >>>>>>>>>>>>>>>>> calls
> > > > > >>>>>>>>>>>>>>>>>> in
> > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> past
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (ie,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit markers when
> > > > > >>>>>>> the
> > > > > >>>>>>>>>>>>>>> transaction is
> > > > > >>>>>>>>>>>>>>>>>>>>> completed)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> often
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes confusion as
> > > > > >>>>>>> we
> > > > > >>>>>>>>>> need to
> > > > > >>>>>>>>>>>> wait
> > > > > >>>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>> some
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> operations
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complete.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing commit
> > > > > >>>>>>> markers
> > > > > >>>>>>>>>> case,
> > > > > >>>>>>>>>>>>>>> clients
> > > > > >>>>>>>>>>>>>>>>> often
> > > > > >>>>>>>>>>>>>>>>>>>> see
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>> CONCURRENT_TRANSACTIONs
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error messages and
> > > > > >>>>>>> that
> > > > > >>>>>>>>>> can be
> > > > > >>>>>>>>>>>>>>>>> confusing.
> > > > > >>>>>>>>>>>>>>>>>>>> For
> > > > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> reason,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> may
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simpler to just
> > > > > >>>>>> have
> > > > > >>>>>>>>>>>> synchronous
> > > > > >>>>>>>>>>>>>>>> calls —
> > > > > >>>>>>>>>>>>>>>>>>>>>>> especially
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> block
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some operation's
> > > > > >>>>>>>>> completion
> > > > > >>>>>>>>>>>> anyway
> > > > > >>>>>>>>>>>>>>>>> before
> > > > > >>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>> can
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> start
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> next
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. And
> > > > > >>>>>>> yes, I
> > > > > >>>>>>>>>> meant
> > > > > >>>>>>>>>>>>>>>>>> coordinator. I
> > > > > >>>>>>>>>>>>>>>>>>>>> will
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> fix
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> that.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are
> > > > > >>>>>> checking
> > > > > >>>>>>> if
> > > > > >>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>>> ongoing,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the
> > > > > >>>>>> leader
> > > > > >>>>>>>>>> partition
> > > > > >>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for
> > > > > >>>>>>> this
> > > > > >>>>>>>>>>>> message to
> > > > > >>>>>>>>>>>>>>>> come
> > > > > >>>>>>>>>>>>>>>>>>>> back,
> > > > > >>>>>>>>>>>>>>>>>>>>> in
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> theory
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
> > > > > >>>>>>>> that
> > > > > >>>>>>>>>> would
> > > > > >>>>>>>>>>>> make
> > > > > >>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>> original
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> result
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why
> > > > > >>>>>> we
> > > > > >>>>>>>> can
> > > > > >>>>>>>>>> check
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>> leader
> > > > > >>>>>>>>>>>>>>>>>>>> state
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> before
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm happy to update
> > > > > >>>>>>> the
> > > > > >>>>>>>>>> KIP if
> > > > > >>>>>>>>>>>>>>> some of
> > > > > >>>>>>>>>>>>>>>>>> these
> > > > > >>>>>>>>>>>>>>>>>>>>>>> things
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> were
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Nov 21,
> > > > > >>>>>> 2022
> > > > > >>>>>>> at
> > > > > >>>>>>>>>> 7:11 PM
> > > > > >>>>>>>>>>>>>>>> Matthias
> > > > > >>>>>>>>>>>>>>>>>> J.
> > > > > >>>>>>>>>>>>>>>>>>>>> Sax <
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the
> > > > > >>>>>> KIP.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Couple of
> > > > > >>>>>>>> clarification
> > > > > >>>>>>>>>>>> questions
> > > > > >>>>>>>>>>>>>>> (I
> > > > > >>>>>>>>>>>>>>>> am
> > > > > >>>>>>>>>>>>>>>>>>>> not a
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> broker
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> expert
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> do
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some question are
> > > > > >>>>>>>>> obvious
> > > > > >>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>> others,
> > > > > >>>>>>>>>>>>>>>>> but
> > > > > >>>>>>>>>>>>>>>>>>>> not
> > > > > >>>>>>>>>>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> me
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> my
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lack
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker knowledge).
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (10)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What happens if
> > > > > >>>>>> the
> > > > > >>>>>>>>>> message
> > > > > >>>>>>>>>>>> come
> > > > > >>>>>>>>>>>>>>> in
> > > > > >>>>>>>>>>>>>>>>>> before
> > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> next
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request? It seems
> > > > > >>>>>>> the
> > > > > >>>>>>>>>> broker
> > > > > >>>>>>>>>>>>>>> hosting
> > > > > >>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>> data
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anything about it
> > > > > >>>>>>> and
> > > > > >>>>>>>>>> append
> > > > > >>>>>>>>>>>> it to
> > > > > >>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> partition,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> too?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> What
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> difference between
> > > > > >>>>>>>> both
> > > > > >>>>>>>>>> cases?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, it seems a
> > > > > >>>>>> TX
> > > > > >>>>>>>>> would
> > > > > >>>>>>>>>> only
> > > > > >>>>>>>>>>>>>>> hang,
> > > > > >>>>>>>>>>>>>>>> if
> > > > > >>>>>>>>>>>>>>>>>>>> there
> > > > > >>>>>>>>>>>>>>>>>>>>>>> is no
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> following
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> either committer
> > > > > >>>>>> or
> > > > > >>>>>>>>>> aborted?
> > > > > >>>>>>>>>>>> Thus,
> > > > > >>>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>> case
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> above,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> TX
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> might
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually not hang
> > > > > >>>>>>> (of
> > > > > >>>>>>>>>> course,
> > > > > >>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>> might
> > > > > >>>>>>>>>>>>>>>>>> get
> > > > > >>>>>>>>>>>>>>>>>>>> an
> > > > > >>>>>>>>>>>>>>>>>>>>>>> EOS
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> violation
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX was aborted and
> > > > > >>>>>>> the
> > > > > >>>>>>>>>> second
> > > > > >>>>>>>>>>>>>>>>> committed,
> > > > > >>>>>>>>>>>>>>>>>> or
> > > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> other
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> way
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around).
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (20)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Of course, 1 and
> > > > > >>>>>> 2
> > > > > >>>>>>>>>> require
> > > > > >>>>>>>>>>>>>>>> client-side
> > > > > >>>>>>>>>>>>>>>>>>>>>>> changes, so
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> those approaches
> > > > > >>>>>>> won’t
> > > > > >>>>>>>>>> apply.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For (1) I
> > > > > >>>>>> understand
> > > > > >>>>>>>>> why a
> > > > > >>>>>>>>>>>> client
> > > > > >>>>>>>>>>>>>>>>> change
> > > > > >>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> necessary,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we need a client
> > > > > >>>>>>>> change
> > > > > >>>>>>>>>> for
> > > > > >>>>>>>>>>>> (2).
> > > > > >>>>>>>>>>>>>>> Can
> > > > > >>>>>>>>>>>>>>>>> you
> > > > > >>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> --
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Later
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> explain
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that we should
> > > > > >>>>>> send
> > > > > >>>>>>> a
> > > > > >>>>>>>>>>>>>>>>>>>>>>> DescribeTransactionRequest,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> but I
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> am
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Can't we not just
> > > > > >>>>>> do
> > > > > >>>>>>>> an
> > > > > >>>>>>>>>>>> implicit
> > > > > >>>>>>>>>>>>>>>>>>>>>>> AddPartiitonToTx,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> too?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> If
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> old
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer correctly
> > > > > >>>>>>>>>> registered
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>> partition
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> already,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can just ignore it
> > > > > >>>>>>> as
> > > > > >>>>>>>>>> it's an
> > > > > >>>>>>>>>>>>>>>>> idempotent
> > > > > >>>>>>>>>>>>>>>>>>>>>>> operation?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (30)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To cover older
> > > > > >>>>>>>> clients,
> > > > > >>>>>>>>>> we
> > > > > >>>>>>>>>>>> will
> > > > > >>>>>>>>>>>>>>>>> ensure a
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ongoing
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> before
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we write to a
> > > > > >>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Not sure what you
> > > > > >>>>>>> mean
> > > > > >>>>>>>>> by
> > > > > >>>>>>>>>>>> this?
> > > > > >>>>>>>>>>>>>>> Can
> > > > > >>>>>>>>>>>>>>>> you
> > > > > >>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (40)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [the
> > > > > >>>>>>> TX-coordinator]
> > > > > >>>>>>>>> will
> > > > > >>>>>>>>>>>> write
> > > > > >>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>> prepare
> > > > > >>>>>>>>>>>>>>>>>>>>>>> commit
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch and send
> > > > > >>>>>>>>>>>>>>> WriteTxnMarkerRequests
> > > > > >>>>>>>>>>>>>>>>>> with
> > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> bumped
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why do we use the
> > > > > >>>>>>>> bumped
> > > > > >>>>>>>>>>>> epoch for
> > > > > >>>>>>>>>>>>>>>>> both?
> > > > > >>>>>>>>>>>>>>>>>> It
> > > > > >>>>>>>>>>>>>>>>>>>>>>> seems
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> more
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> intuitive
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the current epoch,
> > > > > >>>>>>> and
> > > > > >>>>>>>>>> only
> > > > > >>>>>>>>>>>> return
> > > > > >>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>> bumped
> > > > > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (50) "Implicit
> > > > > >>>>>>>>>>>>>>>>> AddPartitionToTransaction"
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why does the
> > > > > >>>>>>>> implicitly
> > > > > >>>>>>>>>> sent
> > > > > >>>>>>>>>>>>>>> request
> > > > > >>>>>>>>>>>>>>>>> need
> > > > > >>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> synchronous?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also says
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in case we need
> > > > > >>>>>> to
> > > > > >>>>>>>>> abort
> > > > > >>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>> need to
> > > > > >>>>>>>>>>>>>>>>>> know
> > > > > >>>>>>>>>>>>>>>>>>>>> which
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What do you mean
> > > > > >>>>>> by
> > > > > >>>>>>>>> this?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we don’t want to
> > > > > >>>>>>>> write
> > > > > >>>>>>>>>> to it
> > > > > >>>>>>>>>>>>>>> before
> > > > > >>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>> store
> > > > > >>>>>>>>>>>>>>>>>>>>> in
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> manager
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Do you mean
> > > > > >>>>>>>>> TX-coordinator
> > > > > >>>>>>>>>>>>>>> instead of
> > > > > >>>>>>>>>>>>>>>>>>>>> "manager"?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (60)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For older clients
> > > > > >>>>>>> and
> > > > > >>>>>>>>>> ensuring
> > > > > >>>>>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>> TX
> > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> ongoing,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> describe a
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> race condition. I
> > > > > >>>>>> am
> > > > > >>>>>>>> not
> > > > > >>>>>>>>>> sure
> > > > > >>>>>>>>>>>> if I
> > > > > >>>>>>>>>>>>>>>> can
> > > > > >>>>>>>>>>>>>>>>>>>> follow
> > > > > >>>>>>>>>>>>>>>>>>>>>>> here.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> Can
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/18/22 1:21
> > > > > >>>>>> PM,
> > > > > >>>>>>>>>> Justine
> > > > > >>>>>>>>>>>>>>> Olshan
> > > > > >>>>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey all!
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to
> > > > > >>>>>> start a
> > > > > >>>>>>>>>>>> discussion
> > > > > >>>>>>>>>>>>>>> on my
> > > > > >>>>>>>>>>>>>>>>>>>> proposal
> > > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> add
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> some
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checks on
> > > > > >>>>>>>> transactions
> > > > > >>>>>>>>> to
> > > > > >>>>>>>>>>>> avoid
> > > > > >>>>>>>>>>>>>>>>> hanging
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> transactions.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> been
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an issue for some
> > > > > >>>>>>>> time,
> > > > > >>>>>>>>>> so I
> > > > > >>>>>>>>>>>>>>> really
> > > > > >>>>>>>>>>>>>>>>> hope
> > > > > >>>>>>>>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>>>>>>> KIP
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> helpful
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users of EOS.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The KIP includes
> > > > > >>>>>>>>> changes
> > > > > >>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>> will
> > > > > >>>>>>>>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>>>>>>>>>> compatible
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> old
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> changes to
> > > > > >>>>>> improve
> > > > > >>>>>>>>>>>> performance
> > > > > >>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>> correctness
> > > > > >>>>>>>>>>>>>>>>>>>>>>> on
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Please take a
> > > > > >>>>>> look
> > > > > >>>>>>>> and
> > > > > >>>>>>>>>> leave
> > > > > >>>>>>>>>>>> any
> > > > > >>>>>>>>>>>>>>>>>> comments
> > > > > >>>>>>>>>>>>>>>>>>>> you
> > > > > >>>>>>>>>>>>>>>>>>>>>>> may
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>> have!
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>
> > > > > >>>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>
> > > > > >>>>
> > > > > >>
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> JIRA:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-14402
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>
> > > > > >>>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>
> > > > > >>>>>
> > > > > >>>>
> > > > > >>>
> > > > > >>
> > > > > >
> > > >
> > >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Guozhang Wang <gu...@gmail.com>.
Thanks folks, I agree with all of your points.

I think in the past we did have some cases where abortable txns are
handled as fatal ones which we are fixing, but maybe there are some
vice versa as well. I'd like to check that, under this KIP's
circumstances, if we could ever have "partial committed" txns where
some of the data are considered as committed while some others are
aborted later. And if there were, could we recover from it. If the
answer is "yes and yes", then I think we can bite the complexity
bullet inside the client and leave users in comfort and peace :)

Guozhang

On Fri, Jan 27, 2023 at 9:42 AM Artem Livshits
<al...@confluent.io.invalid> wrote:
>
> I agree.  I also think we should go a little further and be more
> prescriptive with our recommendations for error handling (i.e. not just
> think if there is a chance, but propose design patterns for typical
> cases).  For example, if a transaction is aborted, the application must
> either keep enough state to be able to redo the operation from the
> beginning, or reset all state and fully recover from Kafka (i.e. re-consume
> data from the last committed offsets).
>
> For failed commits, we need a way for the application to either learn the
> result (if commit went through, then continue, otherwise redo operation) or
> just reset all state and fully recover from Kafka (i.e. re-consume data
> from the last committed offsets).
>
> -Artem
>
> On Fri, Jan 27, 2023 at 9:31 AM Justine Olshan <jo...@confluent.io.invalid>
> wrote:
>
> > Thanks for clarifying Guozhang,
> >
> > I like the framing:
> > * “if we did abort the txn and continue, could the app ever has a chance to
> > recover and not have any side effects violating EOS guarantees”.*
> >
> > With this principle in mind, if we could convince ourselves that we won't
> > have side effects, then we could move forward with a non-fatal error. I
> > appreciate laying out the cases we've discussed. I think I also want to
> > take a look in very close detail and convince myself of the right option.
> > I'll get back to this thread with the final conclusion as well as update
> > the KIP.
> >
> > Thanks for also bringing up KIP-691. I should take a look there as well. :)
> >
> > Justine
> >
> > On Thu, Jan 26, 2023 at 5:28 AM Guozhang Wang <gu...@gmail.com>
> > wrote:
> >
> > > Hello Justine,
> > >
> > > Regarding which errors should be fatal v.s. abortable, I think the
> > > principle would be “if we did abort the txn and continue, could the
> > > app ever has a chance to recover and not have any side effects
> > > violating EOS guarantees”.Of course we can discuss whether that’s a
> > > good principle  e.g. one can argue that maybe it’s okay to violate EOS
> > > some times rather than killing the whole app, whereas my rationale is
> > > that since we want to make EOS the default config, it’s crucial that
> > > we try to guarantee it as much as possible and whenever we detected if
> > > there’s a chance that it maybe broken, we should let users know. And
> > > if we just log an error in app and try to abort and continue, it’s
> > > very likely that users would not be notified and probably only realize
> > > that after a while, which may cause an even harder scenario to
> > > debug.If people agree on that principle, we can go back and check this
> > > KIP’s scenario: there are several scenarios causing a partition leader
> > > detects an out of date epoch from a produce requests:
> > >
> > > * The old produce requests arrive late after a network partition, and
> > > the old producer is already gone.
> > > * The old produce requests arrive late after a network partition, but
> > > the old producer is still around.
> > > * A buggy producer client that did not follow the protocol.
> > >
> > > For the third case, we should try to let user know asap, and as clear
> > > as possible, and hence it’s best to just stop the client app; for the
> > > first case, it does not matter since the producer is already gone;
> > > only for the second, probably the least likely case, we need to
> > > ponder, and there my rationale again is that at that time, we may have
> > > already violated the EOS guarantees since there are some partial txn
> > > records that should be rejected while the txn itself has been
> > > committed. In this case I think it’s better to let clients/users know
> > > as soon as possible as well.
> > >
> > > Regarding the past scenarios where a fatal error killing the whole
> > > apps, I believe part of that reason is that we were doing an
> > > sub-optimal job on clients side handling various error cases and
> > > that’s what KIP-691 is trying to resolve, and hence personally I would
> > > suggest we do not weight in too much on that if we can trust that
> > > KIP-691 will be successfully eliminate those not-necessary-hard-fail
> > > scenarios.
> > >
> > >
> > > Guozhang
> > >
> > > On Wed, Jan 25, 2023 at 5:51 PM Matthias J. Sax <mj...@apache.org>
> > wrote:
> > > >
> > > > So the timestamp would be set when the write happens and thus no
> > > > out-of-order data (base in time) can be introduced with "append_time"
> > > > config even if a request sits in purgatory first while we check the TX
> > > > status.
> > > >
> > > > That does make sense. Thanks for confirming, that there is no
> > > > out-of-order issue for this case.
> > > >
> > > >
> > > > -Matthias
> > > >
> > > > On 1/25/23 5:04 PM, Justine Olshan wrote:
> > > > > Hey Matthias,
> > > > > Let me put it this way, if a producer is checking if a transaction is
> > > > > ongoing, then no writes to the partition from the producer will go
> > > through
> > > > > until the transaction is confirmed ongoing.
> > > > >  From then, I think I can apply the writes in the order they came in.
> > > Does
> > > > > that make sense?
> > > > >
> > > > > Let me know if I'm missing something.
> > > > > Justine
> > > > >
> > > > > On Wed, Jan 25, 2023 at 4:57 PM Matthias J. Sax <mj...@apache.org>
> > > wrote:
> > > > >
> > > > >>> would it build an offset map with just the latest timestamp for a
> > > key?
> > > > >>
> > > > >> Cannot remember the details without reading the KIP, but yes,
> > > something
> > > > >> like this (I believe it actually needs to track both, offset and
> > > > >> timestamp per key).
> > > > >>
> > > > >>> I wonder if ordering assumptions are baked in there, why not use
> > > > >> offset-based compaction.
> > > > >>
> > > > >> The use case is a compacted topic that does contain out-of-order
> > data.
> > > > >> If you set key k1=v1 @ 5 offset 100 and later key1 = v0 @ 3 at
> > offset
> > > > >> 200 we want to cleanup v0 with higher offset because it's
> > out-of-order
> > > > >> based on time, but keep v1 what is the actual latest version of k1.
> > > > >>
> > > > >>
> > > > >>> I was also not aware of this "guarantee" with regards to broker
> > side
> > > > >> time.
> > > > >>
> > > > >> As already said: I am not sure if it's a public contract, but based
> > on
> > > > >> my experience, people might reply on it as "implicit contract". --
> > > Maybe
> > > > >> somebody else knows if it's public or not, and if it would be ok to
> > > > >> "break" it.
> > > > >>
> > > > >>> Let me know if you have any concerns here.
> > > > >>
> > > > >> My understanding is: While we cannot make an offset-order guarantee
> > > for
> > > > >> interleaved writes of different producer, if the topic is configures
> > > > >> with "append_time", we "guarantee" (cf. my comment above") timestamp
> > > > >> order... If that's the case, it would be an issue if we break this
> > > > >> "guarantee".
> > > > >>
> > > > >> I am not sure when the broker sets the timestamp for "append_time"
> > > > >> config? If we do it before putting the request into purgatory, we
> > > have a
> > > > >> problem. However, if we set the timestamp when we actually process
> > the
> > > > >> request and do the actual append, it seems there is no issue, as the
> > > > >> request that was waiting in purgatory get the "newest" timestamp and
> > > > >> thus cannot introduce out-of-order data.
> > > > >>
> > > > >>
> > > > >> -Matthias
> > > > >>
> > > > >>
> > > > >> On 1/24/23 10:44 AM, Justine Olshan wrote:
> > > > >>> Hey Matthias,
> > > > >>>
> > > > >>> I have actually never heard of KIP-280 so thanks for bringing it
> > up.
> > > That
> > > > >>> seems interesting. I wonder how it would work though -- would it
> > > build an
> > > > >>> offset map with just the latest timestamp for a key? I wonder if
> > > ordering
> > > > >>> assumptions are baked in there, why not use offset-based
> > compaction.
> > > > >>>
> > > > >>> I was also not aware of this "guarantee" with regards to broker
> > side
> > > > >> time.
> > > > >>> I think that we can do in order handling for a given producer, but
> > > not
> > > > >>> across all producers. However, we can't guarantee that anyway.
> > > > >>>
> > > > >>> Let me know if you have any concerns here.
> > > > >>>
> > > > >>> Thanks,
> > > > >>> Justine
> > > > >>>
> > > > >>> On Mon, Jan 23, 2023 at 6:33 PM Matthias J. Sax <mj...@apache.org>
> > > > >> wrote:
> > > > >>>
> > > > >>>> Just a side note about Guozhang comments about timestamps.
> > > > >>>>
> > > > >>>> If the producer sets the timestamp, putting the record into
> > > purgatory
> > > > >>>> seems not to be an issue (as already said: for this case we don't
> > > > >>>> guarantee timestamp order between writes of different producers
> > > anyway).
> > > > >>>> However, if the broker sets the timestamp, the expectation is that
> > > there
> > > > >>>> is no out-of-order data in the partition ever; if we would
> > introduce
> > > > >>>> out-of-order data for this case (for interleaved writes of
> > different
> > > > >>>> producers), it seems we would violate the current contract? (To be
> > > fair:
> > > > >>>> I don't know if that's an official contract, but I assume people
> > > rely on
> > > > >>>> this behavior -- and it "advertised" in many public talks...)
> > > > >>>>
> > > > >>>> About compaction: there is actually KIP-280 that adds timestamp
> > > based
> > > > >>>> compaction what is a very useful feature for Kafka Streams with
> > > regard
> > > > >>>> to out-of-order data handling. So the impact if we introduce
> > > > >>>> out-of-order data could be larger scoped.
> > > > >>>>
> > > > >>>>
> > > > >>>> -Matthias
> > > > >>>>
> > > > >>>>
> > > > >>>> On 1/20/23 4:48 PM, Justine Olshan wrote:
> > > > >>>>> Hey Artem,
> > > > >>>>>
> > > > >>>>> I see there is a check for transactional producers. I'm wondering
> > > if we
> > > > >>>>> don't handle the epoch overflow case. I'm also not sure it will
> > be
> > > a
> > > > >> huge
> > > > >>>>> issue to extend to transactional producers, but maybe I'm missing
> > > > >>>> something.
> > > > >>>>>
> > > > >>>>> As for the recovery path -- I think Guozhang's point was if we
> > > have a
> > > > >> bad
> > > > >>>>> client that repeatedly tries to produce without adding to the
> > > > >> transaction
> > > > >>>>> we would do the following:
> > > > >>>>> a) if not fatal, we just fail the produce request over and over
> > > > >>>>> b) if fatal, we fence the producer
> > > > >>>>>
> > > > >>>>> Here with B, the issue with the client would be made clear more
> > > > >> quickly.
> > > > >>>> I
> > > > >>>>> suppose there are some intermediate cases where the issue only
> > > occurs
> > > > >>>>> sometimes, but I wonder if we should consider how to recover with
> > > > >> clients
> > > > >>>>> who don't behave as expected anyway.
> > > > >>>>>
> > > > >>>>> I think there is a place for the abortable error that we are
> > > adding --
> > > > >>>> just
> > > > >>>>> abort and try again. But I think there are also some cases where
> > > trying
> > > > >>>> to
> > > > >>>>> recover overcomplicates some logic. Especially if we are
> > > considering
> > > > >>>> older
> > > > >>>>> clients -- there I'm not sure if there's a ton we can do besides
> > > fail
> > > > >> the
> > > > >>>>> batch or fence the producer. With newer clients, we can consider
> > > more
> > > > >>>>> options for what can just be recovered after aborting. But epochs
> > > might
> > > > >>>> be
> > > > >>>>> a hard one unless we also want to reset producer ID.
> > > > >>>>>
> > > > >>>>> Thanks,
> > > > >>>>> Justine
> > > > >>>>>
> > > > >>>>>
> > > > >>>>>
> > > > >>>>> On Fri, Jan 20, 2023 at 3:59 PM Artem Livshits
> > > > >>>>> <al...@confluent.io.invalid> wrote:
> > > > >>>>>
> > > > >>>>>>>     besides the poorly written client case
> > > > >>>>>>
> > > > >>>>>> A poorly written client could create a lot of grief to people
> > who
> > > run
> > > > >>>> Kafka
> > > > >>>>>> brokers :-), so when deciding to make an error fatal I would see
> > > if
> > > > >>>> there
> > > > >>>>>> is a reasonable recovery path rather than how often it could
> > > happen.
> > > > >>>> If we
> > > > >>>>>> have solid implementation of transactions (which I hope we'll do
> > > as a
> > > > >>>>>> result of this KIP), it would help to recover from a large class
> > > of
> > > > >>>> errors
> > > > >>>>>> by just aborting a transaction, even if the cause of error is a
> > > race
> > > > >>>>>> condition or etc.
> > > > >>>>>>
> > > > >>>>>> -Artem
> > > > >>>>>>
> > > > >>>>>> On Fri, Jan 20, 2023 at 3:26 PM Justine Olshan
> > > > >>>>>> <jo...@confluent.io.invalid>
> > > > >>>>>> wrote:
> > > > >>>>>>
> > > > >>>>>>> Artem --
> > > > >>>>>>> I guess the discussion path we were going down is when we
> > expect
> > > to
> > > > >> see
> > > > >>>>>>> this error. I mentioned that it was hard to come up with cases
> > > for
> > > > >> when
> > > > >>>>>> the
> > > > >>>>>>> producer would still be around to receive the error besides the
> > > > >> poorly
> > > > >>>>>>> written client case.
> > > > >>>>>>> If we don't expect to have a producer to receive the response,
> > it
> > > > >> sort
> > > > >>>> of
> > > > >>>>>>> makes sense for it to be fatal.
> > > > >>>>>>>
> > > > >>>>>>> I had some discussion with Jason offline about the epoch being
> > > off
> > > > >>>> cases
> > > > >>>>>>> and I'm not sure we could find a ton (outside of produce
> > > requests)
> > > > >>>> where
> > > > >>>>>> we
> > > > >>>>>>> could/should recover. I'd be happy to hear some examples
> > though,
> > > > >> maybe
> > > > >>>>>> I'm
> > > > >>>>>>> missing something.
> > > > >>>>>>>
> > > > >>>>>>> Thanks,
> > > > >>>>>>> Justine
> > > > >>>>>>>
> > > > >>>>>>> On Fri, Jan 20, 2023 at 3:19 PM Artem Livshits
> > > > >>>>>>> <al...@confluent.io.invalid> wrote:
> > > > >>>>>>>
> > > > >>>>>>>> In general, I'd like to avoid fatal errors as much as
> > possible,
> > > in
> > > > >>>> some
> > > > >>>>>>>> sense fatal errors just push out recovery logic to the
> > > application
> > > > >>>>>> which
> > > > >>>>>>>> either complicates the application or leads to disruption
> > (we've
> > > > >> seen
> > > > >>>>>>> cases
> > > > >>>>>>>> when a transient broker error could lead to work stoppage when
> > > > >>>>>>> applications
> > > > >>>>>>>> need to be manually restarted).  I think we should strive to
> > > define
> > > > >>>>>>>> recovery logic for most errors (and/or encapsulate it in the
> > > Kafka
> > > > >>>>>> client
> > > > >>>>>>>> as much as possible).
> > > > >>>>>>>>
> > > > >>>>>>>> One benefit of transactions is that they simplify recovery
> > from
> > > > >>>> errors,
> > > > >>>>>>>> pretty much any error (that's not handled transparently by
> > > retries
> > > > >> in
> > > > >>>>>>> Kafka
> > > > >>>>>>>> client) can be handled by the application via aborting the
> > > > >> transaction
> > > > >>>>>>> and
> > > > >>>>>>>> repeating the transactional logic again.  One tricky error is
> > an
> > > > >> error
> > > > >>>>>>>> during commit, because we don't know the outcome.  For commit
> > > > >> errors,
> > > > >>>>>> the
> > > > >>>>>>>> recommendation should be to retry the commit until it returns
> > > the
> > > > >>>>>>> specific
> > > > >>>>>>>> result (committed or aborted).
> > > > >>>>>>>>
> > > > >>>>>>>> -Artem
> > > > >>>>>>>>
> > > > >>>>>>>> On Fri, Jan 20, 2023 at 2:52 PM Justine Olshan
> > > > >>>>>>>> <jo...@confluent.io.invalid>
> > > > >>>>>>>> wrote:
> > > > >>>>>>>>
> > > > >>>>>>>>> That's a fair point about other clients.
> > > > >>>>>>>>>
> > > > >>>>>>>>> I think the abortable error case is interesting because I'm
> > > curious
> > > > >>>>>> how
> > > > >>>>>>>>> other clients would handle this. I assume they would need to
> > > > >>>>>> implement
> > > > >>>>>>>>> handling for the error code unless they did something like
> > "any
> > > > >>>>>> unknown
> > > > >>>>>>>>> error codes/any codes that aren't x,y,z are retriable." I
> > would
> > > > >> hope
> > > > >>>>>>> that
> > > > >>>>>>>>> unknown error codes were fatal, and if the code was
> > > implemented it
> > > > >>>>>>> would
> > > > >>>>>>>>> abort the transaction. But I will think on this too.
> > > > >>>>>>>>>
> > > > >>>>>>>>> As for InvalidRecord -- you mentioned it was not fatal, but
> > I'm
> > > > >>>>>> taking
> > > > >>>>>>> a
> > > > >>>>>>>>> look through the code. We would see this on handling the
> > > produce
> > > > >>>>>>>> response.
> > > > >>>>>>>>> If I recall correctly, we check if errors are retriable. I
> > > think
> > > > >> this
> > > > >>>>>>>> error
> > > > >>>>>>>>> would not be retriable. But I guess the concern here is that
> > > it is
> > > > >>>>>> not
> > > > >>>>>>>>> enough for just that batch to fail. I guess I hadn't
> > considered
> > > > >> fully
> > > > >>>>>>>>> fencing the old producer but there are valid arguments here
> > > why we
> > > > >>>>>>> would
> > > > >>>>>>>>> want to.
> > > > >>>>>>>>>
> > > > >>>>>>>>> Thanks,
> > > > >>>>>>>>> Justine
> > > > >>>>>>>>>
> > > > >>>>>>>>> On Fri, Jan 20, 2023 at 2:35 PM Guozhang Wang <
> > > > >>>>>>>> guozhang.wang.us@gmail.com>
> > > > >>>>>>>>> wrote:
> > > > >>>>>>>>>
> > > > >>>>>>>>>> Thanks Justine for the replies! I agree with most of your
> > > > >> thoughts.
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> Just for 3/7), though I agree for our own AK producer, since
> > > we do
> > > > >>>>>>>>>> "nextRequest(boolean hasIncompleteBatches)", we guarantee
> > the
> > > > >>>>>> end-txn
> > > > >>>>>>>>>> would not be sent until we've effectively flushed, but I was
> > > > >>>>>>> referring
> > > > >>>>>>>>>> to any future bugs or other buggy clients that the same
> > > client may
> > > > >>>>>>> get
> > > > >>>>>>>>>> into this situation, in which case we should give the
> > client a
> > > > >>>>>> clear
> > > > >>>>>>>>>> msg that "you did something wrong, and hence now you should
> > > > >> fatally
> > > > >>>>>>>>>> close yourself". What I'm concerned about is that, by seeing
> > > an
> > > > >>>>>>>>>> "abortable error" or in some rare cases an "invalid record",
> > > the
> > > > >>>>>>>>>> client could not realize "something that's really bad
> > > happened".
> > > > >> So
> > > > >>>>>>>>>> it's not about adding a new error, it's mainly about those
> > > real
> > > > >>>>>> buggy
> > > > >>>>>>>>>> situations causing such "should never happen" cases, the
> > > errors
> > > > >>>>>>> return
> > > > >>>>>>>>>> would not be informative enough.
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> Thinking in other ways, if we believe that for most cases
> > such
> > > > >>>>>> error
> > > > >>>>>>>>>> codes would not reach the original clients since they would
> > be
> > > > >>>>>>>>>> disconnected or even gone by that time, and only in some
> > rare
> > > > >> cases
> > > > >>>>>>>>>> they would still be seen by the sending clients, then why
> > not
> > > make
> > > > >>>>>>>>>> them more fatal and more specific than generic.
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> Guozhang
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> On Fri, Jan 20, 2023 at 1:59 PM Justine Olshan
> > > > >>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> Hey Guozhang. Thanks for taking a look and for the detailed
> > > > >>>>>>> comments!
> > > > >>>>>>>>>> I'll
> > > > >>>>>>>>>>> do my best to address below.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> 1. I see what you are saying here, but I think I need to
> > look
> > > > >>>>>>> through
> > > > >>>>>>>>> the
> > > > >>>>>>>>>>> sequence of events you mention. Typically we've seen this
> > > issue
> > > > >>>>>> in
> > > > >>>>>>> a
> > > > >>>>>>>>> few
> > > > >>>>>>>>>>> cases.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>>     One is when we have a producer disconnect when trying
> > to
> > > > >>>>>> produce.
> > > > >>>>>>>>>>> Typically in these cases, we abort the transaction. We've
> > > seen
> > > > >>>>>> that
> > > > >>>>>>>>> after
> > > > >>>>>>>>>>> the markers are written, the disconnection can sometimes
> > > cause
> > > > >>>>>> the
> > > > >>>>>>>>>> request
> > > > >>>>>>>>>>> to get flushed to the broker. In this case, we don't need
> > > client
> > > > >>>>>>>>> handling
> > > > >>>>>>>>>>> because the producer we are responding to is gone. We just
> > > needed
> > > > >>>>>>> to
> > > > >>>>>>>>> make
> > > > >>>>>>>>>>> sure we didn't write to the log on the broker side. I'm
> > > trying to
> > > > >>>>>>>> think
> > > > >>>>>>>>>> of
> > > > >>>>>>>>>>> a case where we do have the client to return to. I'd think
> > > the
> > > > >>>>>> same
> > > > >>>>>>>>>> client
> > > > >>>>>>>>>>> couldn't progress to committing the transaction unless the
> > > > >>>>>> produce
> > > > >>>>>>>>>> request
> > > > >>>>>>>>>>> returned right? Of course, there is the incorrectly written
> > > > >>>>>> clients
> > > > >>>>>>>>> case.
> > > > >>>>>>>>>>> I'll think on this a bit more and let you know if I come up
> > > with
> > > > >>>>>>>>> another
> > > > >>>>>>>>>>> scenario when we would return to an active client when the
> > > > >>>>>>>> transaction
> > > > >>>>>>>>> is
> > > > >>>>>>>>>>> no longer ongoing.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> I was not aware that we checked the result of a send after
> > we
> > > > >>>>>>> commit
> > > > >>>>>>>>>>> though. I'll need to look into that a bit more.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> 2. There were some questions about this in the discussion.
> > > The
> > > > >>>>>> plan
> > > > >>>>>>>> is
> > > > >>>>>>>>> to
> > > > >>>>>>>>>>> handle overflow with the mechanism we currently have in the
> > > > >>>>>>> producer.
> > > > >>>>>>>>> If
> > > > >>>>>>>>>> we
> > > > >>>>>>>>>>> try to bump and the epoch will overflow, we actually
> > > allocate a
> > > > >>>>>> new
> > > > >>>>>>>>>>> producer ID. I need to confirm the fencing logic on the
> > last
> > > > >>>>>> epoch
> > > > >>>>>>>> (ie,
> > > > >>>>>>>>>> we
> > > > >>>>>>>>>>> probably shouldn't allow any records to be produced with
> > the
> > > > >>>>>> final
> > > > >>>>>>>>> epoch
> > > > >>>>>>>>>>> since we can never properly fence that one).
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> 3. I can agree with you that the current error handling is
> > > > >>>>>> messy. I
> > > > >>>>>>>>>> recall
> > > > >>>>>>>>>>> taking a look at your KIP a while back, but I think I
> > mostly
> > > saw
> > > > >>>>>>> the
> > > > >>>>>>>>>>> section about how the errors were wrapped. Maybe I need to
> > > take
> > > > >>>>>>>> another
> > > > >>>>>>>>>>> look. As for abortable error, the idea was that the
> > handling
> > > > >>>>>> would
> > > > >>>>>>> be
> > > > >>>>>>>>>>> simple -- if this error is seen, the transaction should be
> > > > >>>>>> aborted
> > > > >>>>>>> --
> > > > >>>>>>>>> no
> > > > >>>>>>>>>>> other logic about previous state or requests necessary. Is
> > > your
> > > > >>>>>>>> concern
> > > > >>>>>>>>>>> simply about adding new errors? We were hoping to have an
> > > error
> > > > >>>>>>> that
> > > > >>>>>>>>>> would
> > > > >>>>>>>>>>> have one meaning and many of the current errors have a
> > > history of
> > > > >>>>>>>>> meaning
> > > > >>>>>>>>>>> different things on different client versions. That was the
> > > main
> > > > >>>>>>>>>> motivation
> > > > >>>>>>>>>>> for adding a new error.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> 4. This is a good point about record timestamp reordering.
> > > > >>>>>>> Timestamps
> > > > >>>>>>>>>> don't
> > > > >>>>>>>>>>> affect compaction, but they do affect retention deletion.
> > For
> > > > >>>>>> that,
> > > > >>>>>>>>> kafka
> > > > >>>>>>>>>>> considers the largest timestamp in the segment, so I think
> > a
> > > > >>>>>> small
> > > > >>>>>>>>> amount
> > > > >>>>>>>>>>> of reordering (hopefully on the order of milliseconds or
> > even
> > > > >>>>>>>> seconds)
> > > > >>>>>>>>>> will
> > > > >>>>>>>>>>> be ok. We take timestamps from clients so there is already
> > a
> > > > >>>>>>>>> possibility
> > > > >>>>>>>>>>> for some drift and non-monotonically increasing timestamps.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> 5. Thanks for catching. The error is there, but it's
> > actually
> > > > >>>>>> that
> > > > >>>>>>>>> those
> > > > >>>>>>>>>>> fields should be 4+! Due to how the message generator
> > works,
> > > I
> > > > >>>>>>>> actually
> > > > >>>>>>>>>>> have to redefine those fields inside the
> > > > >>>>>>>>> `"AddPartitionsToTxnTransaction`
> > > > >>>>>>>>>>> block for it to build correctly. I'll fix it to be correct.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> 6. Correct -- we will only add the request to purgatory if
> > > the
> > > > >>>>>>> cache
> > > > >>>>>>>>> has
> > > > >>>>>>>>>> no
> > > > >>>>>>>>>>> ongoing transaction. I can change the wording to make that
> > > > >>>>>> clearer
> > > > >>>>>>>> that
> > > > >>>>>>>>>> we
> > > > >>>>>>>>>>> only place the request in purgatory if we need to contact
> > the
> > > > >>>>>>>>> transaction
> > > > >>>>>>>>>>> coordinator.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> 7. We did take a look at some of the errors and it was hard
> > > to
> > > > >>>>>> come
> > > > >>>>>>>> up
> > > > >>>>>>>>>> with
> > > > >>>>>>>>>>> a good one. I agree that InvalidTxnStateException is ideal
> > > except
> > > > >>>>>>> for
> > > > >>>>>>>>> the
> > > > >>>>>>>>>>> fact that it hasn't been returned on Produce requests
> > > before. The
> > > > >>>>>>>> error
> > > > >>>>>>>>>>> handling for clients is a bit vague (which is why I opened
> > > > >>>>>>>> KAFKA-14439
> > > > >>>>>>>>>>> <https://issues.apache.org/jira/browse/KAFKA-14439>), but
> > > the
> > > > >>>>>>>> decision
> > > > >>>>>>>>>> we
> > > > >>>>>>>>>>> made here was to only return errors that have been
> > previously
> > > > >>>>>>>> returned
> > > > >>>>>>>>> to
> > > > >>>>>>>>>>> producers. As for not being fatal, I think part of the
> > > theory was
> > > > >>>>>>>> that
> > > > >>>>>>>>> in
> > > > >>>>>>>>>>> many cases, the producer would be disconnected. (See point
> > > 1) and
> > > > >>>>>>>> this
> > > > >>>>>>>>>>> would just be an error to return from the server. I did
> > plan
> > > to
> > > > >>>>>>> think
> > > > >>>>>>>>>> about
> > > > >>>>>>>>>>> other cases, so let me know if you think of any as well!
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> Lots to say! Let me know if you have further thoughts!
> > > > >>>>>>>>>>> Justine
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang <
> > > > >>>>>>>>>> guozhang.wang.us@gmail.com>
> > > > >>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>>> Hello Justine,
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> Thanks for the great write-up! I made a quick pass through
> > > it
> > > > >>>>>> and
> > > > >>>>>>>>> here
> > > > >>>>>>>>>>>> are some thoughts (I have not been able to read through
> > this
> > > > >>>>>>> thread
> > > > >>>>>>>>> so
> > > > >>>>>>>>>>>> pardon me if they have overlapped or subsumed by previous
> > > > >>>>>>>> comments):
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> First are some meta ones:
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> 1. I think we need to also improve the client's experience
> > > once
> > > > >>>>>>> we
> > > > >>>>>>>>>>>> have this defence in place. More concretely, say a user's
> > > > >>>>>>> producer
> > > > >>>>>>>>>>>> code is like following:
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> future = producer.send();
> > > > >>>>>>>>>>>> // producer.flush();
> > > > >>>>>>>>>>>> producer.commitTransaction();
> > > > >>>>>>>>>>>> future.get();
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> Which resulted in the order of a) produce-request sent by
> > > > >>>>>>> producer,
> > > > >>>>>>>>> b)
> > > > >>>>>>>>>>>> end-txn-request sent by producer, c) end-txn-response sent
> > > > >>>>>> back,
> > > > >>>>>>> d)
> > > > >>>>>>>>>>>> txn-marker-request sent from coordinator to partition
> > > leader,
> > > > >>>>>> e)
> > > > >>>>>>>>>>>> produce-request finally received by the partition leader,
> > > > >>>>>> before
> > > > >>>>>>>> this
> > > > >>>>>>>>>>>> KIP e) step would be accepted causing a dangling txn; now
> > it
> > > > >>>>>>> would
> > > > >>>>>>>> be
> > > > >>>>>>>>>>>> rejected in step e) which is good. But from the client's
> > > point
> > > > >>>>>> of
> > > > >>>>>>>>> view
> > > > >>>>>>>>>>>> now it becomes confusing since the `commitTransaction()`
> > > > >>>>>> returns
> > > > >>>>>>>>>>>> successfully, but the "future" throws an invalid-epoch
> > > error,
> > > > >>>>>> and
> > > > >>>>>>>>> they
> > > > >>>>>>>>>>>> are not sure if the transaction did succeed or not. In
> > > fact, it
> > > > >>>>>>>>>>>> "partially succeeded" with some msgs being rejected but
> > > others
> > > > >>>>>>>>>>>> committed successfully.
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> Of course the easy way to avoid this is, always call
> > > > >>>>>>>>>>>> "producer.flush()" before commitTxn and that's what we do
> > > > >>>>>>>> ourselves,
> > > > >>>>>>>>>>>> and what we recommend users do. But I suspect not everyone
> > > does
> > > > >>>>>>> it.
> > > > >>>>>>>>> In
> > > > >>>>>>>>>>>> fact I just checked the javadoc in KafkaProducer and our
> > > code
> > > > >>>>>>>> snippet
> > > > >>>>>>>>>>>> does not include a `flush()` call. So I'm thinking maybe
> > we
> > > can
> > > > >>>>>>> in
> > > > >>>>>>>>>>>> side the `commitTxn` code to enforce flushing before
> > sending
> > > > >>>>>> the
> > > > >>>>>>>>>>>> end-txn request.
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> 2. I'd like to clarify a bit details on "just add
> > > partitions to
> > > > >>>>>>> the
> > > > >>>>>>>>>>>> transaction on the first produce request during a
> > > transaction".
> > > > >>>>>>> My
> > > > >>>>>>>>>>>> understanding is that the partition leader's cache has the
> > > > >>>>>>> producer
> > > > >>>>>>>>> id
> > > > >>>>>>>>>>>> / sequence / epoch for the latest txn, either on-going or
> > is
> > > > >>>>>>>>> completed
> > > > >>>>>>>>>>>> (upon receiving the marker request from coordinator).
> > When a
> > > > >>>>>>>> produce
> > > > >>>>>>>>>>>> request is received, if
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> * producer's epoch < cached epoch, or producer's epoch ==
> > > > >>>>>> cached
> > > > >>>>>>>>> epoch
> > > > >>>>>>>>>>>> but the latest txn is completed, leader directly reject
> > with
> > > > >>>>>>>>>>>> invalid-epoch.
> > > > >>>>>>>>>>>> * producer's epoch > cached epoch, park the the request
> > and
> > > > >>>>>> send
> > > > >>>>>>>>>>>> add-partitions request to coordinator.
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> In order to do it, does the coordinator need to bump the
> > > > >>>>>> sequence
> > > > >>>>>>>> and
> > > > >>>>>>>>>>>> reset epoch to 0 when the next epoch is going to overflow?
> > > If
> > > > >>>>>> no
> > > > >>>>>>>> need
> > > > >>>>>>>>>>>> to do so, then how we handle the (admittedly rare, but
> > still
> > > > >>>>>> may
> > > > >>>>>>>>>>>> happen) epoch overflow situation?
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> 3. I'm a bit concerned about adding a generic
> > > "ABORTABLE_ERROR"
> > > > >>>>>>>> given
> > > > >>>>>>>>>>>> we already have a pretty messy error classification and
> > > error
> > > > >>>>>>>>> handling
> > > > >>>>>>>>>>>> on the producer clients side --- I have a summary about
> > the
> > > > >>>>>>> issues
> > > > >>>>>>>>> and
> > > > >>>>>>>>>>>> a proposal to address this in
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>
> > > > >>>>>>
> > > > >>>>
> > > > >>
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
> > > > >>>>>>>>>>>> -- I understand we do not want to use
> > "UNKNOWN_PRODUCER_ID"
> > > > >>>>>>> anymore
> > > > >>>>>>>>>>>> and in fact we intend to deprecate it in KIP-360 and
> > > eventually
> > > > >>>>>>>>> remove
> > > > >>>>>>>>>>>> it; but I'm wondering can we still use specific error
> > codes.
> > > > >>>>>> E.g.
> > > > >>>>>>>>> what
> > > > >>>>>>>>>>>> about "InvalidProducerEpochException" since for new
> > clients,
> > > > >>>>>> the
> > > > >>>>>>>>>>>> actual reason this would actually be rejected is indeed
> > > because
> > > > >>>>>>> the
> > > > >>>>>>>>>>>> epoch on the coordinator caused the add-partitions-request
> > > from
> > > > >>>>>>> the
> > > > >>>>>>>>>>>> brokers to be rejected anyways?
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> 4. It seems we put the producer request into purgatory
> > > before
> > > > >>>>>> we
> > > > >>>>>>>> ever
> > > > >>>>>>>>>>>> append the records, while other producer's records may
> > > still be
> > > > >>>>>>>>>>>> appended during the time; and that potentially may result
> > in
> > > > >>>>>> some
> > > > >>>>>>>>>>>> re-ordering compared with reception order. I'm not super
> > > > >>>>>>> concerned
> > > > >>>>>>>>>>>> about it since Kafka does not guarantee reception ordering
> > > > >>>>>> across
> > > > >>>>>>>>>>>> producers anyways, but it may make the timestamps of
> > records
> > > > >>>>>>>> inside a
> > > > >>>>>>>>>>>> partition to be more out-of-ordered. Are we aware of any
> > > > >>>>>>> scenarios
> > > > >>>>>>>>>>>> such as future enhancements on log compactions that may be
> > > > >>>>>>> affected
> > > > >>>>>>>>> by
> > > > >>>>>>>>>>>> this effect?
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> Below are just minor comments:
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> 5. In "AddPartitionsToTxnTransaction" field of
> > > > >>>>>>>>>>>> "AddPartitionsToTxnRequest" RPC, the versions of those
> > inner
> > > > >>>>>>> fields
> > > > >>>>>>>>>>>> are "0-3" while I thought they should be "0+" still?
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> 6. Regarding "we can place the request in a purgatory of
> > > sorts
> > > > >>>>>>> and
> > > > >>>>>>>>>>>> check if there is any state for the transaction on the
> > > > >>>>>> broker": i
> > > > >>>>>>>>>>>> think at this time when we just do the checks against the
> > > > >>>>>> cached
> > > > >>>>>>>>>>>> state, we do not need to put the request to purgatory yet?
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> 7. This is related to 3) above. I feel using
> > > > >>>>>>>> "InvalidRecordException"
> > > > >>>>>>>>>>>> for older clients may also be a bit confusing, and also it
> > > is
> > > > >>>>>> not
> > > > >>>>>>>>>>>> fatal -- for old clients, it better to be fatal since this
> > > > >>>>>>>> indicates
> > > > >>>>>>>>>>>> the clients is doing something wrong and hence it should
> > be
> > > > >>>>>>> closed.
> > > > >>>>>>>>>>>> And in general I'd prefer to use slightly more specific
> > > meaning
> > > > >>>>>>>> error
> > > > >>>>>>>>>>>> codes for clients. That being said, I also feel
> > > > >>>>>>>>>>>> "InvalidProducerEpochException" is not suitable for old
> > > > >>>>>> versioned
> > > > >>>>>>>>>>>> clients, and we'd have to pick one that old clients
> > > recognize.
> > > > >>>>>>> I'd
> > > > >>>>>>>>>>>> prefer "InvalidTxnStateException" but that one is supposed
> > > to
> > > > >>>>>> be
> > > > >>>>>>>>>>>> returned from txn coordinators only today. I'd suggest we
> > > do a
> > > > >>>>>>>> quick
> > > > >>>>>>>>>>>> check in the current client's code path and see if that
> > one
> > > > >>>>>> would
> > > > >>>>>>>> be
> > > > >>>>>>>>>>>> handled if it's from a produce-response, and if yes, use
> > > this
> > > > >>>>>>> one;
> > > > >>>>>>>>>>>> otherwise, use "ProducerFencedException" which is much
> > less
> > > > >>>>>>>>> meaningful
> > > > >>>>>>>>>>>> but it's still a fatal error.
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> Thanks,
> > > > >>>>>>>>>>>> Guozhang
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
> > > > >>>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> Yeah -- looks like we already have code to handle bumping
> > > the
> > > > >>>>>>>> epoch
> > > > >>>>>>>>>> and
> > > > >>>>>>>>>>>>> when the epoch is Short.MAX_VALUE, we get a new producer
> > > ID.
> > > > >>>>>>>> Since
> > > > >>>>>>>>>> this
> > > > >>>>>>>>>>>> is
> > > > >>>>>>>>>>>>> already the behavior, do we want to change it further?
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> Justine
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <
> > > > >>>>>>>>> jolshan@confluent.io
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> Hey all, just wanted to quickly update and say I've
> > > > >>>>>> modified
> > > > >>>>>>>> the
> > > > >>>>>>>>>> KIP to
> > > > >>>>>>>>>>>>>> explicitly mention that AddOffsetCommitsToTxnRequest
> > will
> > > > >>>>>> be
> > > > >>>>>>>>>> replaced
> > > > >>>>>>>>>>>> by
> > > > >>>>>>>>>>>>>> a coordinator-side (inter-broker) AddPartitionsToTxn
> > > > >>>>>> implicit
> > > > >>>>>>>>>> request.
> > > > >>>>>>>>>>>> This
> > > > >>>>>>>>>>>>>> mirrors the user partitions and will implicitly add
> > offset
> > > > >>>>>>>>>> partitions
> > > > >>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>> transactions when we commit offsets on them. We will
> > > > >>>>>>> deprecate
> > > > >>>>>>>>>>>> AddOffsetCommitsToTxnRequest
> > > > >>>>>>>>>>>>>> for new clients.
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> Also to address Artem's comments --
> > > > >>>>>>>>>>>>>> I'm a bit unsure if the changes here will change the
> > > > >>>>>> previous
> > > > >>>>>>>>>> behavior
> > > > >>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>> fencing producers. In the case you mention in the first
> > > > >>>>>>>>> paragraph,
> > > > >>>>>>>>>> are
> > > > >>>>>>>>>>>> you
> > > > >>>>>>>>>>>>>> saying we bump the epoch before we try to abort the
> > > > >>>>>>>> transaction?
> > > > >>>>>>>>> I
> > > > >>>>>>>>>>>> think I
> > > > >>>>>>>>>>>>>> need to understand the scenarios you mention a bit
> > better.
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> As for the second part -- I think it makes sense to have
> > > > >>>>>> some
> > > > >>>>>>>>> sort
> > > > >>>>>>>>>> of
> > > > >>>>>>>>>>>>>> "sentinel" epoch to signal epoch is about to overflow (I
> > > > >>>>>>> think
> > > > >>>>>>>> we
> > > > >>>>>>>>>> sort
> > > > >>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>> have this value in place in some ways) so we can codify
> > it
> > > > >>>>>> in
> > > > >>>>>>>> the
> > > > >>>>>>>>>> KIP.
> > > > >>>>>>>>>>>> I'll
> > > > >>>>>>>>>>>>>> look into that and try to update soon.
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> Thanks,
> > > > >>>>>>>>>>>>>> Justine.
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
> > > > >>>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> It's good to know that KIP-588 addressed some of the
> > > > >>>>>> issues.
> > > > >>>>>>>>>> Looking
> > > > >>>>>>>>>>>> at
> > > > >>>>>>>>>>>>>>> the code, it still looks like there are some cases that
> > > > >>>>>>> would
> > > > >>>>>>>>>> result
> > > > >>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>> fatal error, e.g. PRODUCER_FENCED is issued by the
> > > > >>>>>>> transaction
> > > > >>>>>>>>>>>> coordinator
> > > > >>>>>>>>>>>>>>> if epoch doesn't match, and the client treats it as a
> > > > >>>>>> fatal
> > > > >>>>>>>>> error
> > > > >>>>>>>>>>>> (code in
> > > > >>>>>>>>>>>>>>> TransactionManager request handling).  If we consider,
> > > for
> > > > >>>>>>>>>> example,
> > > > >>>>>>>>>>>>>>> committing a transaction that returns a timeout, but
> > > > >>>>>>> actually
> > > > >>>>>>>>>>>> succeeds,
> > > > >>>>>>>>>>>>>>> trying to abort it or re-commit may result in
> > > > >>>>>>> PRODUCER_FENCED
> > > > >>>>>>>>>> error
> > > > >>>>>>>>>>>>>>> (because of epoch bump).
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> For failed commits, specifically, we need to know the
> > > > >>>>>> actual
> > > > >>>>>>>>>> outcome,
> > > > >>>>>>>>>>>>>>> because if we return an error the application may think
> > > > >>>>>> that
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>>>>> transaction is aborted and redo the work, leading to
> > > > >>>>>>>> duplicates.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> Re: overflowing epoch.  We could either do it on the TC
> > > > >>>>>> and
> > > > >>>>>>>>> return
> > > > >>>>>>>>>>>> both
> > > > >>>>>>>>>>>>>>> producer id and epoch (e.g. change the protocol), or
> > > > >>>>>> signal
> > > > >>>>>>>> the
> > > > >>>>>>>>>> client
> > > > >>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>> it needs to get a new producer id.  Checking for max
> > > epoch
> > > > >>>>>>>> could
> > > > >>>>>>>>>> be a
> > > > >>>>>>>>>>>>>>> reasonable signal, the value to check should probably
> > be
> > > > >>>>>>>> present
> > > > >>>>>>>>>> in
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>> KIP
> > > > >>>>>>>>>>>>>>> as this is effectively a part of the contract.  Also,
> > the
> > > > >>>>>> TC
> > > > >>>>>>>>>> should
> > > > >>>>>>>>>>>>>>> probably return an error if the client didn't change
> > > > >>>>>>> producer
> > > > >>>>>>>> id
> > > > >>>>>>>>>> after
> > > > >>>>>>>>>>>>>>> hitting max epoch.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> -Artem
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
> > > > >>>>>>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> Thanks for the discussion Artem.
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> With respect to the handling of fenced producers, we
> > > > >>>>>> have
> > > > >>>>>>>> some
> > > > >>>>>>>>>>>> behavior
> > > > >>>>>>>>>>>>>>>> already in place. As of KIP-588:
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>
> > > > >>>>>>
> > > > >>>>
> > > > >>
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> > > > >>>>>>>>>>>>>>>> ,
> > > > >>>>>>>>>>>>>>>> we handle timeouts more gracefully. The producer can
> > > > >>>>>>>> recover.
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> Produce requests can also recover from epoch fencing
> > by
> > > > >>>>>>>>>> aborting the
> > > > >>>>>>>>>>>>>>>> transaction and starting over.
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> What other cases were you considering that would cause
> > > > >>>>>> us
> > > > >>>>>>> to
> > > > >>>>>>>>>> have a
> > > > >>>>>>>>>>>>>>> fenced
> > > > >>>>>>>>>>>>>>>> epoch but we'd want to recover?
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> The first point about handling epoch overflows is
> > fair.
> > > > >>>>>> I
> > > > >>>>>>>>> think
> > > > >>>>>>>>>>>> there is
> > > > >>>>>>>>>>>>>>>> some logic we'd need to consider. (ie, if we are one
> > > > >>>>>> away
> > > > >>>>>>>> from
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>> max
> > > > >>>>>>>>>>>>>>>> epoch, we need to reset the producer ID.) I'm still
> > > > >>>>>>>> wondering
> > > > >>>>>>>>> if
> > > > >>>>>>>>>>>> there
> > > > >>>>>>>>>>>>>>> is a
> > > > >>>>>>>>>>>>>>>> way to direct this from the response, or if everything
> > > > >>>>>>>> should
> > > > >>>>>>>>> be
> > > > >>>>>>>>>>>> done on
> > > > >>>>>>>>>>>>>>>> the client side. Let me know if you have any thoughts
> > > > >>>>>>> here.
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> Thanks,
> > > > >>>>>>>>>>>>>>>> Justine
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
> > > > >>>>>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> There are some workflows in the client that are
> > > > >>>>>> implied
> > > > >>>>>>> by
> > > > >>>>>>>>>>>> protocol
> > > > >>>>>>>>>>>>>>>>> changes, e.g.:
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> - for new clients, epoch changes with every
> > > > >>>>>> transaction
> > > > >>>>>>>> and
> > > > >>>>>>>>>> can
> > > > >>>>>>>>>>>>>>> overflow,
> > > > >>>>>>>>>>>>>>>>> in old clients this condition was handled
> > > > >>>>>> transparently,
> > > > >>>>>>>>>> because
> > > > >>>>>>>>>>>> epoch
> > > > >>>>>>>>>>>>>>>> was
> > > > >>>>>>>>>>>>>>>>> bumped in InitProducerId and it would return a new
> > > > >>>>>>>> producer
> > > > >>>>>>>>>> id if
> > > > >>>>>>>>>>>>>>> epoch
> > > > >>>>>>>>>>>>>>>>> overflows, the new clients would need to implement
> > > > >>>>>> some
> > > > >>>>>>>>>> workflow
> > > > >>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>> refresh
> > > > >>>>>>>>>>>>>>>>> producer id
> > > > >>>>>>>>>>>>>>>>> - how to handle fenced producers, for new clients
> > > > >>>>>> epoch
> > > > >>>>>>>>>> changes
> > > > >>>>>>>>>>>> with
> > > > >>>>>>>>>>>>>>>> every
> > > > >>>>>>>>>>>>>>>>> transaction, so in presence of failures during
> > > > >>>>>> commits /
> > > > >>>>>>>>>> aborts,
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>> producer could get easily fenced, old clients would
> > > > >>>>>>> pretty
> > > > >>>>>>>>>> much
> > > > >>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>> get
> > > > >>>>>>>>>>>>>>>>> fenced when a new incarnation of the producer was
> > > > >>>>>>>>> initialized
> > > > >>>>>>>>>> with
> > > > >>>>>>>>>>>>>>>>> InitProducerId so it's ok to treat as a fatal error,
> > > > >>>>>> the
> > > > >>>>>>>> new
> > > > >>>>>>>>>>>> clients
> > > > >>>>>>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>> need to implement some workflow to handle that error,
> > > > >>>>>>>>>> otherwise
> > > > >>>>>>>>>>>> they
> > > > >>>>>>>>>>>>>>>> could
> > > > >>>>>>>>>>>>>>>>> get fenced by themselves
> > > > >>>>>>>>>>>>>>>>> - in particular (as a subset of the previous issue),
> > > > >>>>>>> what
> > > > >>>>>>>>>> would
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>> client
> > > > >>>>>>>>>>>>>>>>> do if it got a timeout during commit?  commit
> > could've
> > > > >>>>>>>>>> succeeded
> > > > >>>>>>>>>>>> or
> > > > >>>>>>>>>>>>>>>> failed
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> Not sure if this has to be defined in the KIP as
> > > > >>>>>>>>> implementing
> > > > >>>>>>>>>>>> those
> > > > >>>>>>>>>>>>>>>>> probably wouldn't require protocol changes, but we
> > > > >>>>>> have
> > > > >>>>>>>>>> multiple
> > > > >>>>>>>>>>>>>>>>> implementations of Kafka clients, so probably would
> > be
> > > > >>>>>>>> good
> > > > >>>>>>>>> to
> > > > >>>>>>>>>>>> have
> > > > >>>>>>>>>>>>>>> some
> > > > >>>>>>>>>>>>>>>>> client implementation guidance.  Could also be done
> > > > >>>>>> as a
> > > > >>>>>>>>>> separate
> > > > >>>>>>>>>>>> doc.
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> -Artem
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
> > > > >>>>>>>>>>>>>>>> <jolshan@confluent.io.invalid
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> Hey all, I've updated the KIP to incorporate Jason's
> > > > >>>>>>>>>>>> suggestions.
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>
> > > > >>>>>>
> > > > >>>>
> > > > >>
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> 1. Use AddPartitionsToTxn + verify flag to check on
> > > > >>>>>>> old
> > > > >>>>>>>>>> clients
> > > > >>>>>>>>>>>>>>>>>> 2. Updated AddPartitionsToTxn API to support
> > > > >>>>>>> transaction
> > > > >>>>>>>>>>>> batching
> > > > >>>>>>>>>>>>>>>>>> 3. Mention IBP bump
> > > > >>>>>>>>>>>>>>>>>> 4. Mention auth change on new AddPartitionsToTxn
> > > > >>>>>>>> version.
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> I'm planning on opening a vote soon.
> > > > >>>>>>>>>>>>>>>>>> Thanks,
> > > > >>>>>>>>>>>>>>>>>> Justine
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <
> > > > >>>>>>>>>>>> jolshan@confluent.io
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> Thanks Jason. Those changes make sense to me. I
> > > > >>>>>> will
> > > > >>>>>>>>>> update
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>> KIP.
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
> > > > >>>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
> > > > >>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> Hey Justine,
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> I was wondering about compatibility here. When
> > > > >>>>>> we
> > > > >>>>>>>>> send
> > > > >>>>>>>>>>>> requests
> > > > >>>>>>>>>>>>>>>>>>>> between brokers, we want to ensure that the
> > > > >>>>>>> receiving
> > > > >>>>>>>>>> broker
> > > > >>>>>>>>>>>>>>>>> understands
> > > > >>>>>>>>>>>>>>>>>>>> the request (specifically the new fields).
> > > > >>>>>>> Typically
> > > > >>>>>>>>>> this is
> > > > >>>>>>>>>>>> done
> > > > >>>>>>>>>>>>>>>> via
> > > > >>>>>>>>>>>>>>>>>>>> IBP/metadata version.
> > > > >>>>>>>>>>>>>>>>>>>> I'm trying to think if there is a way around it
> > > > >>>>>> but
> > > > >>>>>>>> I'm
> > > > >>>>>>>>>> not
> > > > >>>>>>>>>>>> sure
> > > > >>>>>>>>>>>>>>>> there
> > > > >>>>>>>>>>>>>>>>>> is.
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> Yes. I think we would gate usage of this behind
> > > > >>>>>> an
> > > > >>>>>>>> IBP
> > > > >>>>>>>>>> bump.
> > > > >>>>>>>>>>>> Does
> > > > >>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>> seem
> > > > >>>>>>>>>>>>>>>>>>>> reasonable?
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> As for the improvements -- can you clarify how
> > > > >>>>>>> the
> > > > >>>>>>>>>> multiple
> > > > >>>>>>>>>>>>>>>>>>>> transactional
> > > > >>>>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking of a case
> > > > >>>>>>>> where
> > > > >>>>>>>>> we
> > > > >>>>>>>>>>>>>>> wait/batch
> > > > >>>>>>>>>>>>>>>>>>>> multiple produce requests together? My
> > > > >>>>>>> understanding
> > > > >>>>>>>>> for
> > > > >>>>>>>>>> now
> > > > >>>>>>>>>>>> was
> > > > >>>>>>>>>>>>>>> 1
> > > > >>>>>>>>>>>>>>>>>>>> transactional ID and one validation per 1 produce
> > > > >>>>>>>>>> request.
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> Each call to `AddPartitionsToTxn` is essentially
> > > > >>>>>> a
> > > > >>>>>>>>> write
> > > > >>>>>>>>>> to
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>> log and must block on replication. The more we
> > > > >>>>>> can
> > > > >>>>>>>> fit
> > > > >>>>>>>>>> into a
> > > > >>>>>>>>>>>>>>> single
> > > > >>>>>>>>>>>>>>>>>>>> request, the more writes we can do in parallel.
> > > > >>>>>> The
> > > > >>>>>>>>>>>> alternative
> > > > >>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>> make
> > > > >>>>>>>>>>>>>>>>>>>> use of more connections, but usually we prefer
> > > > >>>>>>>> batching
> > > > >>>>>>>>>>>> since the
> > > > >>>>>>>>>>>>>>>>>> network
> > > > >>>>>>>>>>>>>>>>>>>> stack is not really optimized for high
> > > > >>>>>>>>> connection/request
> > > > >>>>>>>>>>>> loads.
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> Finally with respect to the authorizations, I
> > > > >>>>>>> think
> > > > >>>>>>>>> it
> > > > >>>>>>>>>>>> makes
> > > > >>>>>>>>>>>>>>> sense
> > > > >>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>> skip
> > > > >>>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit confused by
> > > > >>>>>> the
> > > > >>>>>>>>>> "leader
> > > > >>>>>>>>>>>> ID"
> > > > >>>>>>>>>>>>>>>> field.
> > > > >>>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the request as
> > > > >>>>>> from a
> > > > >>>>>>>>>> broker
> > > > >>>>>>>>>>>> (does
> > > > >>>>>>>>>>>>>>> it
> > > > >>>>>>>>>>>>>>>>>> matter
> > > > >>>>>>>>>>>>>>>>>>>> which one?).
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> We could also make it version-based. For the next
> > > > >>>>>>>>>> version, we
> > > > >>>>>>>>>>>>>>> could
> > > > >>>>>>>>>>>>>>>>>>>> require
> > > > >>>>>>>>>>>>>>>>>>>> CLUSTER auth. So clients would not be able to use
> > > > >>>>>>> the
> > > > >>>>>>>>> API
> > > > >>>>>>>>>>>>>>> anymore,
> > > > >>>>>>>>>>>>>>>>> which
> > > > >>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>> probably what we want.
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> -Jason
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> > > > >>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> As a follow up, I was just thinking about the
> > > > >>>>>>>>> batching
> > > > >>>>>>>>>> a
> > > > >>>>>>>>>>>> bit
> > > > >>>>>>>>>>>>>>> more.
> > > > >>>>>>>>>>>>>>>>>>>>> I suppose if we have one request in flight and
> > > > >>>>>> we
> > > > >>>>>>>>>> queue up
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>> other
> > > > >>>>>>>>>>>>>>>>>>>>> produce requests in some sort of purgatory, we
> > > > >>>>>>>> could
> > > > >>>>>>>>>> send
> > > > >>>>>>>>>>>>>>>>> information
> > > > >>>>>>>>>>>>>>>>>>>> out
> > > > >>>>>>>>>>>>>>>>>>>>> for all of them rather than one by one. So that
> > > > >>>>>>>> would
> > > > >>>>>>>>>> be a
> > > > >>>>>>>>>>>>>>> benefit
> > > > >>>>>>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>>>>> batching partitions to add per transaction.
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> I'll need to think a bit more on the design of
> > > > >>>>>>> this
> > > > >>>>>>>>>> part
> > > > >>>>>>>>>>>> of the
> > > > >>>>>>>>>>>>>>>> KIP,
> > > > >>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>> will update the KIP in the next few days.
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > >>>>>>>>>>>>>>>>>>>>> Justine
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan
> > > > >>>>>> <
> > > > >>>>>>>>>>>>>>>>> jolshan@confluent.io>
> > > > >>>>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> Hey Jason -- thanks for the input -- I was
> > > > >>>>>> just
> > > > >>>>>>>>>> digging
> > > > >>>>>>>>>>>> a bit
> > > > >>>>>>>>>>>>>>>>> deeper
> > > > >>>>>>>>>>>>>>>>>>>> into
> > > > >>>>>>>>>>>>>>>>>>>>>> the design + implementation of the validation
> > > > >>>>>>>> calls
> > > > >>>>>>>>>> here
> > > > >>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>> what
> > > > >>>>>>>>>>>>>>>>>> you
> > > > >>>>>>>>>>>>>>>>>>>> say
> > > > >>>>>>>>>>>>>>>>>>>>>> makes sense.
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> I was wondering about compatibility here.
> > > > >>>>>> When
> > > > >>>>>>> we
> > > > >>>>>>>>>> send
> > > > >>>>>>>>>>>>>>> requests
> > > > >>>>>>>>>>>>>>>>>>>>>> between brokers, we want to ensure that the
> > > > >>>>>>>>> receiving
> > > > >>>>>>>>>>>> broker
> > > > >>>>>>>>>>>>>>>>>>>> understands
> > > > >>>>>>>>>>>>>>>>>>>>>> the request (specifically the new fields).
> > > > >>>>>>>>> Typically
> > > > >>>>>>>>>>>> this is
> > > > >>>>>>>>>>>>>>>> done
> > > > >>>>>>>>>>>>>>>>>> via
> > > > >>>>>>>>>>>>>>>>>>>>>> IBP/metadata version.
> > > > >>>>>>>>>>>>>>>>>>>>>> I'm trying to think if there is a way around
> > > > >>>>>> it
> > > > >>>>>>>> but
> > > > >>>>>>>>>> I'm
> > > > >>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>> sure
> > > > >>>>>>>>>>>>>>>>>> there
> > > > >>>>>>>>>>>>>>>>>>>>> is.
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> As for the improvements -- can you clarify
> > > > >>>>>> how
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>> multiple
> > > > >>>>>>>>>>>>>>>>>>>> transactional
> > > > >>>>>>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking of a
> > > > >>>>>>> case
> > > > >>>>>>>>>> where we
> > > > >>>>>>>>>>>>>>>>> wait/batch
> > > > >>>>>>>>>>>>>>>>>>>>>> multiple produce requests together? My
> > > > >>>>>>>>> understanding
> > > > >>>>>>>>>> for
> > > > >>>>>>>>>>>> now
> > > > >>>>>>>>>>>>>>>> was 1
> > > > >>>>>>>>>>>>>>>>>>>>>> transactional ID and one validation per 1
> > > > >>>>>>> produce
> > > > >>>>>>>>>>>> request.
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> Finally with respect to the authorizations, I
> > > > >>>>>>>> think
> > > > >>>>>>>>>> it
> > > > >>>>>>>>>>>> makes
> > > > >>>>>>>>>>>>>>>> sense
> > > > >>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>> skip
> > > > >>>>>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit confused
> > > > >>>>>> by
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>> "leader
> > > > >>>>>>>>>>>>>>> ID"
> > > > >>>>>>>>>>>>>>>>>> field.
> > > > >>>>>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the request as
> > > > >>>>>>>> from a
> > > > >>>>>>>>>>>> broker
> > > > >>>>>>>>>>>>>>> (does
> > > > >>>>>>>>>>>>>>>>> it
> > > > >>>>>>>>>>>>>>>>>>>>> matter
> > > > >>>>>>>>>>>>>>>>>>>>>> which one?).
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> I think I want to adopt these suggestions,
> > > > >>>>>> just
> > > > >>>>>>>> had
> > > > >>>>>>>>>> a few
> > > > >>>>>>>>>>>>>>>>> questions
> > > > >>>>>>>>>>>>>>>>>> on
> > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>> details.
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > >>>>>>>>>>>>>>>>>>>>>> Justine
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> On Thu, Jan 5, 2023 at 5:05 PM Jason
> > > > >>>>>> Gustafson
> > > > >>>>>>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
> > > > >>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> Thanks for the proposal.
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> I was thinking about the implementation a
> > > > >>>>>>> little
> > > > >>>>>>>>>> bit.
> > > > >>>>>>>>>>>> In the
> > > > >>>>>>>>>>>>>>>>>> current
> > > > >>>>>>>>>>>>>>>>>>>>>>> proposal, the behavior depends on whether we
> > > > >>>>>>>> have
> > > > >>>>>>>>> an
> > > > >>>>>>>>>>>> old or
> > > > >>>>>>>>>>>>>>> new
> > > > >>>>>>>>>>>>>>>>>>>> client.
> > > > >>>>>>>>>>>>>>>>>>>>>>> For
> > > > >>>>>>>>>>>>>>>>>>>>>>> old clients, we send `DescribeTransactions`
> > > > >>>>>>> and
> > > > >>>>>>>>>> verify
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>> result
> > > > >>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>>>>> new clients, we send `AddPartitionsToTxn`.
> > > > >>>>>> We
> > > > >>>>>>>>> might
> > > > >>>>>>>>>> be
> > > > >>>>>>>>>>>> able
> > > > >>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>> simplify
> > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> implementation if we can use the same
> > > > >>>>>> request
> > > > >>>>>>>>> type.
> > > > >>>>>>>>>> For
> > > > >>>>>>>>>>>>>>>> example,
> > > > >>>>>>>>>>>>>>>>>>>> what if
> > > > >>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>> bump the protocol version for
> > > > >>>>>>>> `AddPartitionsToTxn`
> > > > >>>>>>>>>> and
> > > > >>>>>>>>>>>> add a
> > > > >>>>>>>>>>>>>>>>>>>>>>> `validateOnly`
> > > > >>>>>>>>>>>>>>>>>>>>>>> flag? For older versions, we can set
> > > > >>>>>>>>>>>> `validateOnly=true` so
> > > > >>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> request only returns successfully if the
> > > > >>>>>>>> partition
> > > > >>>>>>>>>> had
> > > > >>>>>>>>>>>>>>> already
> > > > >>>>>>>>>>>>>>>>> been
> > > > >>>>>>>>>>>>>>>>>>>>> added.
> > > > >>>>>>>>>>>>>>>>>>>>>>> For new versions, we can set
> > > > >>>>>>>> `validateOnly=false`
> > > > >>>>>>>>>> and
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>> partition
> > > > >>>>>>>>>>>>>>>>>>>> will
> > > > >>>>>>>>>>>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>>>>> added to the transaction. The other slightly
> > > > >>>>>>>>>> annoying
> > > > >>>>>>>>>>>> thing
> > > > >>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>>>>>>> get around is the need to collect the
> > > > >>>>>>>> transaction
> > > > >>>>>>>>>> state
> > > > >>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>> all
> > > > >>>>>>>>>>>>>>>>>>>>> partitions
> > > > >>>>>>>>>>>>>>>>>>>>>>> even when we only care about a subset.
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> Some additional improvements to consider:
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> - We can give `AddPartitionsToTxn` better
> > > > >>>>>>> batch
> > > > >>>>>>>>>> support
> > > > >>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>> inter-broker
> > > > >>>>>>>>>>>>>>>>>>>>>>> usage. Currently we only allow one
> > > > >>>>>>>>>> `TransactionalId` to
> > > > >>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>> specified,
> > > > >>>>>>>>>>>>>>>>>>>>> but
> > > > >>>>>>>>>>>>>>>>>>>>>>> the broker may get some benefit being able
> > > > >>>>>> to
> > > > >>>>>>>>> batch
> > > > >>>>>>>>>>>> across
> > > > >>>>>>>>>>>>>>>>> multiple
> > > > >>>>>>>>>>>>>>>>>>>>>>> transactions.
> > > > >>>>>>>>>>>>>>>>>>>>>>> - Another small improvement is skipping
> > > > >>>>>> topic
> > > > >>>>>>>>>>>> authorization
> > > > >>>>>>>>>>>>>>>>> checks
> > > > >>>>>>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>>>>> `AddPartitionsToTxn` when the request is
> > > > >>>>>> from
> > > > >>>>>>> a
> > > > >>>>>>>>>> broker.
> > > > >>>>>>>>>>>>>>> Perhaps
> > > > >>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>> can
> > > > >>>>>>>>>>>>>>>>>>>>> add
> > > > >>>>>>>>>>>>>>>>>>>>>>> a field for the `LeaderId` or something like
> > > > >>>>>>>> that
> > > > >>>>>>>>>> and
> > > > >>>>>>>>>>>>>>> require
> > > > >>>>>>>>>>>>>>>>>> CLUSTER
> > > > >>>>>>>>>>>>>>>>>>>>>>> permission when set.
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> Best,
> > > > >>>>>>>>>>>>>>>>>>>>>>> Jason
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
> > > > >>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation. It makes sense
> > > > >>>>>>> to
> > > > >>>>>>>> me
> > > > >>>>>>>>>> now.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 1:42 PM Justine
> > > > >>>>>>> Olshan
> > > > >>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> My understanding of the mechanism is
> > > > >>>>>> that
> > > > >>>>>>>> when
> > > > >>>>>>>>>> we
> > > > >>>>>>>>>>>> get to
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>> last
> > > > >>>>>>>>>>>>>>>>>>>>>>> epoch,
> > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> increment to the fencing/last epoch and
> > > > >>>>>> if
> > > > >>>>>>>> any
> > > > >>>>>>>>>>>> further
> > > > >>>>>>>>>>>>>>>>> requests
> > > > >>>>>>>>>>>>>>>>>>>> come
> > > > >>>>>>>>>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> this producer ID they are fenced. Then
> > > > >>>>>> the
> > > > >>>>>>>>>> producer
> > > > >>>>>>>>>>>>>>> gets a
> > > > >>>>>>>>>>>>>>>>> new
> > > > >>>>>>>>>>>>>>>>>> ID
> > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> restarts with epoch/sequence 0. The
> > > > >>>>>> fenced
> > > > >>>>>>>>> epoch
> > > > >>>>>>>>>>>> sticks
> > > > >>>>>>>>>>>>>>>>> around
> > > > >>>>>>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> duration of producer.id.expiration.ms
> > > > >>>>>> and
> > > > >>>>>>>>>> blocks
> > > > >>>>>>>>>>>> any
> > > > >>>>>>>>>>>>>>> late
> > > > >>>>>>>>>>>>>>>>>>>> messages
> > > > >>>>>>>>>>>>>>>>>>>>>>>> there.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> The new ID will get to take advantage of
> > > > >>>>>>> the
> > > > >>>>>>>>>>>> improved
> > > > >>>>>>>>>>>>>>>>> semantics
> > > > >>>>>>>>>>>>>>>>>>>>> around
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> non-zero start sequences. So I think we
> > > > >>>>>>> are
> > > > >>>>>>>>>> covered.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> The only potential issue is overloading
> > > > >>>>>>> the
> > > > >>>>>>>>>> cache,
> > > > >>>>>>>>>>>> but
> > > > >>>>>>>>>>>>>>>>>> hopefully
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> improvements (lowered
> > > > >>>>>>>>> producer.id.expiration.ms
> > > > >>>>>>>>>> )
> > > > >>>>>>>>>>>> will
> > > > >>>>>>>>>>>>>>> help
> > > > >>>>>>>>>>>>>>>>>> with
> > > > >>>>>>>>>>>>>>>>>>>>> that.
> > > > >>>>>>>>>>>>>>>>>>>>>>>> Let
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> me know if you still have concerns.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
> > > > >>>>>>>>>>>>>>>>>>>> <ju...@confluent.io.invalid>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> 70. The proposed fencing logic doesn't
> > > > >>>>>>>> apply
> > > > >>>>>>>>>> when
> > > > >>>>>>>>>>>> pid
> > > > >>>>>>>>>>>>>>>>>> changes,
> > > > >>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> right? If so, I am not sure how
> > > > >>>>>> complete
> > > > >>>>>>>> we
> > > > >>>>>>>>>> are
> > > > >>>>>>>>>>>>>>>> addressing
> > > > >>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>>>> issue
> > > > >>>>>>>>>>>>>>>>>>>>>>>> if
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the pid changes more frequently.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 16, 2022 at 9:16 AM
> > > > >>>>>> Justine
> > > > >>>>>>>>> Olshan
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for replying!
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> 70.We already do the overflow
> > > > >>>>>>> mechanism,
> > > > >>>>>>>>> so
> > > > >>>>>>>>>> my
> > > > >>>>>>>>>>>>>>> change
> > > > >>>>>>>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>>>> just
> > > > >>>>>>>>>>>>>>>>>>>>>>> make
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> happen more often.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I was also not suggesting a new
> > > > >>>>>> field
> > > > >>>>>>> in
> > > > >>>>>>>>> the
> > > > >>>>>>>>>>>> log,
> > > > >>>>>>>>>>>>>>> but
> > > > >>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>> response,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> which would be gated by the client
> > > > >>>>>>>>> version.
> > > > >>>>>>>>>>>> Sorry if
> > > > >>>>>>>>>>>>>>>>>>>> something
> > > > >>>>>>>>>>>>>>>>>>>>>>> there
> > > > >>>>>>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> unclear. I think we are starting to
> > > > >>>>>>>>> diverge.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> The goal of this KIP is to not
> > > > >>>>>> change
> > > > >>>>>>> to
> > > > >>>>>>>>> the
> > > > >>>>>>>>>>>> marker
> > > > >>>>>>>>>>>>>>>>> format
> > > > >>>>>>>>>>>>>>>>>> at
> > > > >>>>>>>>>>>>>>>>>>>>> all.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> 71. Yes, I guess I was going under
> > > > >>>>>> the
> > > > >>>>>>>>>>>> assumption
> > > > >>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> log
> > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> just
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> look at its last epoch and treat it
> > > > >>>>>> as
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>> current
> > > > >>>>>>>>>>>>>>>>> epoch. I
> > > > >>>>>>>>>>>>>>>>>>>>>>> suppose
> > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> have some special logic that if the
> > > > >>>>>>> last
> > > > >>>>>>>>>> epoch
> > > > >>>>>>>>>>>> was
> > > > >>>>>>>>>>>>>>> on a
> > > > >>>>>>>>>>>>>>>>>>>> marker
> > > > >>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> actually
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> expect the next epoch or something
> > > > >>>>>>> like
> > > > >>>>>>>>>> that. We
> > > > >>>>>>>>>>>>>>> just
> > > > >>>>>>>>>>>>>>>>> need
> > > > >>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> distinguish
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> based on whether we had a
> > > > >>>>>> commit/abort
> > > > >>>>>>>>>> marker.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> 72.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> if the producer epoch hasn't been
> > > > >>>>>>>> bumped
> > > > >>>>>>>>>> on
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
> > > > >>>>>>>> message
> > > > >>>>>>>>>> will
> > > > >>>>>>>>>>>> fail
> > > > >>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> sequence
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> validation
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the producer
> > > > >>>>>>>> epoch
> > > > >>>>>>>>>> has
> > > > >>>>>>>>>>>> been
> > > > >>>>>>>>>>>>>>>>> bumped,
> > > > >>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>> ignore
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck message
> > > > >>>>>>>> could
> > > > >>>>>>>>>> be
> > > > >>>>>>>>>>>>>>> appended
> > > > >>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>> log.
> > > > >>>>>>>>>>>>>>>>>>>>>>>> So,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
> > > > >>>>>> guard?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I follow that "the
> > > > >>>>>>> message
> > > > >>>>>>>>> will
> > > > >>>>>>>>>>>> fail
> > > > >>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> sequence
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> validation".
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> In some of these cases, we had an
> > > > >>>>>>> abort
> > > > >>>>>>>>>> marker
> > > > >>>>>>>>>>>> (due
> > > > >>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>> an
> > > > >>>>>>>>>>>>>>>>>>>> error)
> > > > >>>>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> then
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the late message comes in with the
> > > > >>>>>>>> correct
> > > > >>>>>>>>>>>> sequence
> > > > >>>>>>>>>>>>>>>>> number.
> > > > >>>>>>>>>>>>>>>>>>>> This
> > > > >>>>>>>>>>>>>>>>>>>>>>> is a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> case
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> covered by the KIP.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> The latter case is actually not
> > > > >>>>>>>> something
> > > > >>>>>>>>>> we've
> > > > >>>>>>>>>>>>>>>>> considered
> > > > >>>>>>>>>>>>>>>>>>>>> here. I
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> think
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> generally when we bump the epoch, we
> > > > >>>>>>> are
> > > > >>>>>>>>>>>> accepting
> > > > >>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> sequence
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> does
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> not need to be checked anymore. My
> > > > >>>>>>>>>>>> understanding is
> > > > >>>>>>>>>>>>>>>> also
> > > > >>>>>>>>>>>>>>>>>>>> that we
> > > > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> typically bump epoch mid transaction
> > > > >>>>>>>>> (based
> > > > >>>>>>>>>> on a
> > > > >>>>>>>>>>>>>>> quick
> > > > >>>>>>>>>>>>>>>>> look
> > > > >>>>>>>>>>>>>>>>>>>> at
> > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> code)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but let me know if that is the case.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 15, 2022 at 12:23 PM Jun
> > > > >>>>>>> Rao
> > > > >>>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the reply.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Assigning a new pid on int
> > > > >>>>>>>> overflow
> > > > >>>>>>>>>> seems
> > > > >>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>> bit
> > > > >>>>>>>>>>>>>>>>>> hacky.
> > > > >>>>>>>>>>>>>>>>>>>> If
> > > > >>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> need a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> level id, it will be better to
> > > > >>>>>> model
> > > > >>>>>>>>> this
> > > > >>>>>>>>>>>>>>> explicitly.
> > > > >>>>>>>>>>>>>>>>>>>> Adding a
> > > > >>>>>>>>>>>>>>>>>>>>>>> new
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> field
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> would require a bit more work
> > > > >>>>>> since
> > > > >>>>>>> it
> > > > >>>>>>>>>>>> requires a
> > > > >>>>>>>>>>>>>>> new
> > > > >>>>>>>>>>>>>>>>> txn
> > > > >>>>>>>>>>>>>>>>>>>>> marker
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> format
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the log. So, we probably need to
> > > > >>>>>>> guard
> > > > >>>>>>>>> it
> > > > >>>>>>>>>>>> with an
> > > > >>>>>>>>>>>>>>> IBP
> > > > >>>>>>>>>>>>>>>>> or
> > > > >>>>>>>>>>>>>>>>>>>>>>> metadata
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> version
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and document the impact on
> > > > >>>>>> downgrade
> > > > >>>>>>>>> once
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>> new
> > > > >>>>>>>>>>>>>>>>> format
> > > > >>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>> written
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 71. Hmm, once the marker is
> > > > >>>>>> written,
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>> partition
> > > > >>>>>>>>>>>>>>>> will
> > > > >>>>>>>>>>>>>>>>>>>> expect
> > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> next
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> append to be on the next epoch.
> > > > >>>>>> Does
> > > > >>>>>>>>> that
> > > > >>>>>>>>>>>> cover
> > > > >>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>> case
> > > > >>>>>>>>>>>>>>>>>>>> you
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> mentioned?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 72. Also, just to be clear on the
> > > > >>>>>>>>> stucked
> > > > >>>>>>>>>>>> message
> > > > >>>>>>>>>>>>>>>> issue
> > > > >>>>>>>>>>>>>>>>>>>>>>> described
> > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> motivation. With EoS, we also
> > > > >>>>>>> validate
> > > > >>>>>>>>> the
> > > > >>>>>>>>>>>>>>> sequence
> > > > >>>>>>>>>>>>>>>> id
> > > > >>>>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> idempotency.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> So,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> with the current logic, if the
> > > > >>>>>>>> producer
> > > > >>>>>>>>>> epoch
> > > > >>>>>>>>>>>>>>> hasn't
> > > > >>>>>>>>>>>>>>>>> been
> > > > >>>>>>>>>>>>>>>>>>>>>>> bumped on
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
> > > > >>>>>>>>> message
> > > > >>>>>>>>>> will
> > > > >>>>>>>>>>>>>>> fail
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>> sequence
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> validation
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the
> > > > >>>>>> producer
> > > > >>>>>>>>>> epoch has
> > > > >>>>>>>>>>>>>>> been
> > > > >>>>>>>>>>>>>>>>>>>> bumped, we
> > > > >>>>>>>>>>>>>>>>>>>>>>>> ignore
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck
> > > > >>>>>> message
> > > > >>>>>>>>>> could be
> > > > >>>>>>>>>>>>>>>> appended
> > > > >>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> log.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> So,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
> > > > >>>>>>> guard?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 10:44 AM
> > > > >>>>>>>> Justine
> > > > >>>>>>>>>>>> Olshan
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > > >>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias — thanks again for
> > > > >>>>>> taking
> > > > >>>>>>>>> time
> > > > >>>>>>>>>> to
> > > > >>>>>>>>>>>> look
> > > > >>>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>> this.
> > > > >>>>>>>>>>>>>>>>>>>> You
> > > > >>>>>>>>>>>>>>>>>>>>>>> said:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My proposal was only focusing
> > > > >>>>>> to
> > > > >>>>>>>>> avoid
> > > > >>>>>>>>>>>>>>> dangling
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
> > > > >>>>>> added
> > > > >>>>>>>>>> without
> > > > >>>>>>>>>>>>>>>> registered
> > > > >>>>>>>>>>>>>>>>>>>>>>> partition.
> > > > >>>>>>>>>>>>>>>>>>>>>>>> --
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more details
> > > > >>>>>> to
> > > > >>>>>>>> the
> > > > >>>>>>>>>> KIP
> > > > >>>>>>>>>>>> about
> > > > >>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>> scenario
> > > > >>>>>>>>>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> better
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I understand what
> > > > >>>>>> you
> > > > >>>>>>>>> mean
> > > > >>>>>>>>>>>> here.
> > > > >>>>>>>>>>>>>>> The
> > > > >>>>>>>>>>>>>>>>>>>> motivation
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> section
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> describes two scenarios about
> > > > >>>>>> how
> > > > >>>>>>>> the
> > > > >>>>>>>>>> record
> > > > >>>>>>>>>>>>>>> can be
> > > > >>>>>>>>>>>>>>>>>> added
> > > > >>>>>>>>>>>>>>>>>>>>>>>> without a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> registered partition:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Another way hanging
> > > > >>>>>> transactions
> > > > >>>>>>>> can
> > > > >>>>>>>>>>>> occur is
> > > > >>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>>>> client
> > > > >>>>>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> buggy
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> may somehow try to write to a
> > > > >>>>>>>>> partition
> > > > >>>>>>>>>>>> before
> > > > >>>>>>>>>>>>>>> it
> > > > >>>>>>>>>>>>>>>>> adds
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> partition
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> For the first example of this
> > > > >>>>>>> would
> > > > >>>>>>>> it
> > > > >>>>>>>>>> be
> > > > >>>>>>>>>>>>>>> helpful
> > > > >>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>> say
> > > > >>>>>>>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> comes in after the abort, but
> > > > >>>>>>> before
> > > > >>>>>>>>> the
> > > > >>>>>>>>>>>>>>> partition
> > > > >>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>> added
> > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> next
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction so it becomes
> > > > >>>>>>> "hanging."
> > > > >>>>>>>>>>>> Perhaps the
> > > > >>>>>>>>>>>>>>>> next
> > > > >>>>>>>>>>>>>>>>>>>>> sentence
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> describing
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the message becoming part of the
> > > > >>>>>>>> next
> > > > >>>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>> (a
> > > > >>>>>>>>>>>>>>>>>>>>> different
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> case)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> was
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not properly differentiated.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun — thanks for reading the
> > > > >>>>>> KIP.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. The int typing was a
> > > > >>>>>> concern.
> > > > >>>>>>>>>> Currently
> > > > >>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>> have a
> > > > >>>>>>>>>>>>>>>>>>>>>>> mechanism
> > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> place
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> fence the final epoch when the
> > > > >>>>>>> epoch
> > > > >>>>>>>>> is
> > > > >>>>>>>>>>>> about to
> > > > >>>>>>>>>>>>>>>>>> overflow
> > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> assign
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer ID with epoch 0. Of
> > > > >>>>>>> course,
> > > > >>>>>>>>>> this
> > > > >>>>>>>>>>>> is a
> > > > >>>>>>>>>>>>>>> bit
> > > > >>>>>>>>>>>>>>>>>> tricky
> > > > >>>>>>>>>>>>>>>>>>>>>>> when it
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> comes
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response back to the client.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Making this a long could be
> > > > >>>>>>> another
> > > > >>>>>>>>>> option,
> > > > >>>>>>>>>>>> but
> > > > >>>>>>>>>>>>>>> I
> > > > >>>>>>>>>>>>>>>>>> wonder
> > > > >>>>>>>>>>>>>>>>>>>> are
> > > > >>>>>>>>>>>>>>>>>>>>>>>> there
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> implications on changing this
> > > > >>>>>>> field
> > > > >>>>>>>> if
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>>>> epoch is
> > > > >>>>>>>>>>>>>>>>>>>>> persisted
> > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> disk?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to check the usages.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 71.This was something Matthias
> > > > >>>>>>> asked
> > > > >>>>>>>>>> about
> > > > >>>>>>>>>>>> as
> > > > >>>>>>>>>>>>>>>> well. I
> > > > >>>>>>>>>>>>>>>>>> was
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> considering a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible edge case where a
> > > > >>>>>> produce
> > > > >>>>>>>>>> request
> > > > >>>>>>>>>>>> from
> > > > >>>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>> new
> > > > >>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> somehow
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> gets sent right after the marker
> > > > >>>>>>> is
> > > > >>>>>>>>>>>> written, but
> > > > >>>>>>>>>>>>>>>>> before
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> alerted of the newly bumped
> > > > >>>>>> epoch.
> > > > >>>>>>>> In
> > > > >>>>>>>>>> this
> > > > >>>>>>>>>>>>>>> case, we
> > > > >>>>>>>>>>>>>>>>> may
> > > > >>>>>>>>>>>>>>>>>>>>>>> include
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> record
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we don't want to. I suppose
> > > > >>>>>>> we
> > > > >>>>>>>>>> could
> > > > >>>>>>>>>>>> try
> > > > >>>>>>>>>>>>>>> to do
> > > > >>>>>>>>>>>>>>>>>>>>> something
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> client
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> side
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to bump the epoch after sending
> > > > >>>>>> an
> > > > >>>>>>>>>> endTxn as
> > > > >>>>>>>>>>>>>>> well
> > > > >>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>>>>> scenario
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> —
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> I
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wonder how it would work when
> > > > >>>>>> the
> > > > >>>>>>>>>> server is
> > > > >>>>>>>>>>>>>>>> aborting
> > > > >>>>>>>>>>>>>>>>>>>> based
> > > > >>>>>>>>>>>>>>>>>>>>> on
> > > > >>>>>>>>>>>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> error. I could also be missing
> > > > >>>>>>>>>> something and
> > > > >>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>> scenario
> > > > >>>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> actually
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again to everyone reading
> > > > >>>>>>> and
> > > > >>>>>>>>>>>> commenting.
> > > > >>>>>>>>>>>>>>>> Let
> > > > >>>>>>>>>>>>>>>>> me
> > > > >>>>>>>>>>>>>>>>>>>> know
> > > > >>>>>>>>>>>>>>>>>>>>>>>> about
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> further questions or comments.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 9:41 AM
> > > > >>>>>>> Jun
> > > > >>>>>>>>> Rao
> > > > >>>>>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP. A couple
> > > > >>>>>> of
> > > > >>>>>>>>>> comments.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Currently, the producer
> > > > >>>>>>> epoch
> > > > >>>>>>>> is
> > > > >>>>>>>>>> an
> > > > >>>>>>>>>>>> int.
> > > > >>>>>>>>>>>>>>> I am
> > > > >>>>>>>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>>>>>> sure
> > > > >>>>>>>>>>>>>>>>>>>>>>> if
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> it's
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> enough
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to accommodate all
> > > > >>>>>> transactions
> > > > >>>>>>> in
> > > > >>>>>>>>> the
> > > > >>>>>>>>>>>>>>> lifetime
> > > > >>>>>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>>>>>> producer.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Should
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> change that to a long or add a
> > > > >>>>>>> new
> > > > >>>>>>>>>> long
> > > > >>>>>>>>>>>> field
> > > > >>>>>>>>>>>>>>>> like
> > > > >>>>>>>>>>>>>>>>>>>> txnId?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 71. "it will write the prepare
> > > > >>>>>>>>> commit
> > > > >>>>>>>>>>>> message
> > > > >>>>>>>>>>>>>>>> with
> > > > >>>>>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>>>> bumped
> > > > >>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> send
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WriteTxnMarkerRequests with
> > > > >>>>>> the
> > > > >>>>>>>>> bumped
> > > > >>>>>>>>>>>> epoch."
> > > > >>>>>>>>>>>>>>>> Hmm,
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > >>>>>>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> associated
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the current txn right?
> > > > >>>>>> So,
> > > > >>>>>>> it
> > > > >>>>>>>>>> seems
> > > > >>>>>>>>>>>>>>> weird to
> > > > >>>>>>>>>>>>>>>>>>>> write a
> > > > >>>>>>>>>>>>>>>>>>>>>>>> commit
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with a bumped epoch. Should we
> > > > >>>>>>>> only
> > > > >>>>>>>>>> bump
> > > > >>>>>>>>>>>> up
> > > > >>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>> epoch
> > > > >>>>>>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> EndTxnResponse
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rename the field to sth like
> > > > >>>>>>>>>>>>>>> nextProducerEpoch?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 12, 2022 at 8:54
> > > > >>>>>> PM
> > > > >>>>>>>>>> Matthias
> > > > >>>>>>>>>>>> J.
> > > > >>>>>>>>>>>>>>> Sax <
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the background.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30: SGTM. My proposal was
> > > > >>>>>>>> only
> > > > >>>>>>>>>>>> focusing
> > > > >>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>> avoid
> > > > >>>>>>>>>>>>>>>>>>>>>>> dangling
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
> > > > >>>>>>>> added
> > > > >>>>>>>>>>>> without
> > > > >>>>>>>>>>>>>>>>>> registered
> > > > >>>>>>>>>>>>>>>>>>>>>>>> partition.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> --
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more
> > > > >>>>>> details
> > > > >>>>>>>> to
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>> KIP
> > > > >>>>>>>>>>>>>>> about
> > > > >>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>>>> scenario
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40: I think you hit a fair
> > > > >>>>>>> point
> > > > >>>>>>>>>> about
> > > > >>>>>>>>>>>> race
> > > > >>>>>>>>>>>>>>>>>>>> conditions
> > > > >>>>>>>>>>>>>>>>>>>>> or
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> client
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> bugs
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (incorrectly not bumping the
> > > > >>>>>>>>>> epoch). The
> > > > >>>>>>>>>>>>>>>>>>>>>>> complexity/confusion
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> using
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the bumped epoch I see, is
> > > > >>>>>>>> mainly
> > > > >>>>>>>>>> for
> > > > >>>>>>>>>>>>>>> internal
> > > > >>>>>>>>>>>>>>>>>>>>> debugging,
> > > > >>>>>>>>>>>>>>>>>>>>>>> ie,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> inspecting
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log segment dumps -- it
> > > > >>>>>> seems
> > > > >>>>>>>>>> harder to
> > > > >>>>>>>>>>>>>>> reason
> > > > >>>>>>>>>>>>>>>>>> about
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>> system
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> us
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> humans. But if we get better
> > > > >>>>>>>>>>>> guarantees, it
> > > > >>>>>>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>>>>> worth to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> use
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped epoch.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60: as I mentioned already,
> > > > >>>>>> I
> > > > >>>>>>>>> don't
> > > > >>>>>>>>>>>> know the
> > > > >>>>>>>>>>>>>>>>> broker
> > > > >>>>>>>>>>>>>>>>>>>>>>> internals
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> provide
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more input. So if nobody
> > > > >>>>>> else
> > > > >>>>>>>>> chimes
> > > > >>>>>>>>>>>> in, we
> > > > >>>>>>>>>>>>>>>>> should
> > > > >>>>>>>>>>>>>>>>>>>> just
> > > > >>>>>>>>>>>>>>>>>>>>>>> move
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> forward
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with your proposal.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 12/6/22 4:22 PM, Justine
> > > > >>>>>>>> Olshan
> > > > >>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi all,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> After Artem's questions
> > > > >>>>>>> about
> > > > >>>>>>>>>> error
> > > > >>>>>>>>>>>>>>> behavior,
> > > > >>>>>>>>>>>>>>>>>> I've
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> re-evaluated
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unknown producer ID
> > > > >>>>>>> exception
> > > > >>>>>>>>> and
> > > > >>>>>>>>>> had
> > > > >>>>>>>>>>>> some
> > > > >>>>>>>>>>>>>>>>>>>> discussions
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> offline.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think generally it makes
> > > > >>>>>>>> sense
> > > > >>>>>>>>>> to
> > > > >>>>>>>>>>>>>>> simplify
> > > > >>>>>>>>>>>>>>>>>> error
> > > > >>>>>>>>>>>>>>>>>>>>>>> handling
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> cases
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this and the
> > > > >>>>>>>> UNKNOWN_PRODUCER_ID
> > > > >>>>>>>>>> error
> > > > >>>>>>>>>>>>>>> has a
> > > > >>>>>>>>>>>>>>>>>> pretty
> > > > >>>>>>>>>>>>>>>>>>>>> long
> > > > >>>>>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complicated
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> history. Because of this,
> > > > >>>>>> I
> > > > >>>>>>>>>> propose
> > > > >>>>>>>>>>>>>>> adding a
> > > > >>>>>>>>>>>>>>>>> new
> > > > >>>>>>>>>>>>>>>>>>>> error
> > > > >>>>>>>>>>>>>>>>>>>>>>> code
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ABORTABLE_ERROR
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that when encountered by
> > > > >>>>>> new
> > > > >>>>>>>>>> clients
> > > > >>>>>>>>>>>>>>> (gated
> > > > >>>>>>>>>>>>>>>> by
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> produce
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will simply abort the
> > > > >>>>>>>>> transaction.
> > > > >>>>>>>>>>>> This
> > > > >>>>>>>>>>>>>>>> allows
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> server
> > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> some
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> say
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in whether the client
> > > > >>>>>> aborts
> > > > >>>>>>>> and
> > > > >>>>>>>>>> makes
> > > > >>>>>>>>>>>>>>>> handling
> > > > >>>>>>>>>>>>>>>>>>>> much
> > > > >>>>>>>>>>>>>>>>>>>>>>>> simpler.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> future, we can also use
> > > > >>>>>> this
> > > > >>>>>>>>>> error in
> > > > >>>>>>>>>>>>>>> other
> > > > >>>>>>>>>>>>>>>>>>>> situations
> > > > >>>>>>>>>>>>>>>>>>>>>>>> where
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> want
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> abort the transactions. We
> > > > >>>>>>> can
> > > > >>>>>>>>>> even
> > > > >>>>>>>>>>>> use on
> > > > >>>>>>>>>>>>>>>>> other
> > > > >>>>>>>>>>>>>>>>>>>> apis.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've added this to the
> > > > >>>>>> KIP.
> > > > >>>>>>>> Let
> > > > >>>>>>>>> me
> > > > >>>>>>>>>>>> know if
> > > > >>>>>>>>>>>>>>>>> there
> > > > >>>>>>>>>>>>>>>>>>>> are
> > > > >>>>>>>>>>>>>>>>>>>>> any
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> questions
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> or
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 2, 2022 at
> > > > >>>>>> 10:22
> > > > >>>>>>>> AM
> > > > >>>>>>>>>>>> Justine
> > > > >>>>>>>>>>>>>>>> Olshan
> > > > >>>>>>>>>>>>>>>>> <
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> jolshan@confluent.io
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey Matthias,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30 — Maybe I also
> > > > >>>>>> didn't
> > > > >>>>>>>>>> express
> > > > >>>>>>>>>>>>>>> myself
> > > > >>>>>>>>>>>>>>>>>>>> clearly.
> > > > >>>>>>>>>>>>>>>>>>>>> For
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't have a way to
> > > > >>>>>>>> distinguish
> > > > >>>>>>>>>>>> between a
> > > > >>>>>>>>>>>>>>>>>> previous
> > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> current
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction since we
> > > > >>>>>> don't
> > > > >>>>>>>> have
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>> epoch
> > > > >>>>>>>>>>>>>>>>> bump.
> > > > >>>>>>>>>>>>>>>>>>>> This
> > > > >>>>>>>>>>>>>>>>>>>>>>> means
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message from the previous
> > > > >>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>> may be
> > > > >>>>>>>>>>>>>>>>>>>> added to
> > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> one.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> With
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older clients — we can't
> > > > >>>>>>>>>> guarantee
> > > > >>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>> won't
> > > > >>>>>>>>>>>>>>>>>>>> happen
> > > > >>>>>>>>>>>>>>>>>>>>>>> if we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> already
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call
> > > > >>>>>>> (why
> > > > >>>>>>>> we
> > > > >>>>>>>>>> make
> > > > >>>>>>>>>>>>>>> changes
> > > > >>>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> newer
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> client)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can at least gate some by
> > > > >>>>>>>>>> ensuring
> > > > >>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> partition
> > > > >>>>>>>>>>>>>>>>>>>>>>> has
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> been
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> added
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. The
> > > > >>>>>> rationale
> > > > >>>>>>>> here
> > > > >>>>>>>>>> is
> > > > >>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>> there
> > > > >>>>>>>>>>>>>>>>>> are
> > > > >>>>>>>>>>>>>>>>>>>>>>> likely
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> LESS
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrivals
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as time goes on, so
> > > > >>>>>>> hopefully
> > > > >>>>>>>>>> most
> > > > >>>>>>>>>>>> late
> > > > >>>>>>>>>>>>>>>>> arrivals
> > > > >>>>>>>>>>>>>>>>>>>> will
> > > > >>>>>>>>>>>>>>>>>>>>>>> come
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> BEFORE
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call.
> > > > >>>>>>>> Those
> > > > >>>>>>>>>> that
> > > > >>>>>>>>>>>>>>> arrive
> > > > >>>>>>>>>>>>>>>>>> before
> > > > >>>>>>>>>>>>>>>>>>>>> will
> > > > >>>>>>>>>>>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> properly
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> gated
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> > > > >>>>>>> describeTransactions
> > > > >>>>>>>>>>>> approach.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we take the approach
> > > > >>>>>> you
> > > > >>>>>>>>>>>> suggested,
> > > > >>>>>>>>>>>>>>> ANY
> > > > >>>>>>>>>>>>>>>>> late
> > > > >>>>>>>>>>>>>>>>>>>>> arrival
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> from a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> previous
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction will be
> > > > >>>>>> added.
> > > > >>>>>>>> And
> > > > >>>>>>>>> we
> > > > >>>>>>>>>>>> don't
> > > > >>>>>>>>>>>>>>> want
> > > > >>>>>>>>>>>>>>>>>>>> that. I
> > > > >>>>>>>>>>>>>>>>>>>>>>> also
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> see
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> benefit in sending
> > > > >>>>>>>>>> addPartitionsToTxn
> > > > >>>>>>>>>>>>>>> over
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>> describeTxns
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> They
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> both be one extra RPC to
> > > > >>>>>>> the
> > > > >>>>>>>>> Txn
> > > > >>>>>>>>>>>>>>>> coordinator.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To be clear — newer
> > > > >>>>>> clients
> > > > >>>>>>>>> will
> > > > >>>>>>>>>> use
> > > > >>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> instead
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTxns.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that if we
> > > > >>>>>>> have
> > > > >>>>>>>>>> some
> > > > >>>>>>>>>>>> delay
> > > > >>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>> client
> > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> bump
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it could continue to send
> > > > >>>>>>>> epoch
> > > > >>>>>>>>>> 73
> > > > >>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>> those
> > > > >>>>>>>>>>>>>>>>>>>> records
> > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fenced.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Perhaps this is not an
> > > > >>>>>>> issue
> > > > >>>>>>>> if
> > > > >>>>>>>>>> we
> > > > >>>>>>>>>>>> don't
> > > > >>>>>>>>>>>>>>>> allow
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>> next
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> produce
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> go
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through before the EndTxn
> > > > >>>>>>>>> request
> > > > >>>>>>>>>>>>>>> returns.
> > > > >>>>>>>>>>>>>>>> I'm
> > > > >>>>>>>>>>>>>>>>>>>> also
> > > > >>>>>>>>>>>>>>>>>>>>>>>> thinking
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> about
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cases of
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure. I will need to
> > > > >>>>>>> think
> > > > >>>>>>>>> on
> > > > >>>>>>>>>>>> this a
> > > > >>>>>>>>>>>>>>> bit.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I wasn't sure if it was
> > > > >>>>>>> that
> > > > >>>>>>>>>>>> confusing.
> > > > >>>>>>>>>>>>>>> But
> > > > >>>>>>>>>>>>>>>> if
> > > > >>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>> think it
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> is,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> investigate other ways.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure these are
> > > > >>>>>> the
> > > > >>>>>>>> same
> > > > >>>>>>>>>>>>>>> purgatories
> > > > >>>>>>>>>>>>>>>>>> since
> > > > >>>>>>>>>>>>>>>>>>>> one
> > > > >>>>>>>>>>>>>>>>>>>>>>> is a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> produce
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory (I was planning
> > > > >>>>>>> on
> > > > >>>>>>>>>> using a
> > > > >>>>>>>>>>>>>>>> callback
> > > > >>>>>>>>>>>>>>>>>>>> rather
> > > > >>>>>>>>>>>>>>>>>>>>>>> than
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the other is simply a
> > > > >>>>>>> request
> > > > >>>>>>>>> to
> > > > >>>>>>>>>>>> append
> > > > >>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> log.
> > > > >>>>>>>>>>>>>>>>>>>>> Not
> > > > >>>>>>>>>>>>>>>>>>>>>>>> sure
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> structure here for
> > > > >>>>>>> ordering,
> > > > >>>>>>>>> but
> > > > >>>>>>>>>> my
> > > > >>>>>>>>>>>>>>>>>> understanding
> > > > >>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handle the write request
> > > > >>>>>>>> before
> > > > >>>>>>>>>> it
> > > > >>>>>>>>>>>> hears
> > > > >>>>>>>>>>>>>>>> back
> > > > >>>>>>>>>>>>>>>>>> from
> > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> Txn
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Coordinator.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if I
> > > > >>>>>>>> misunderstood
> > > > >>>>>>>>>>>> something
> > > > >>>>>>>>>>>>>>> or
> > > > >>>>>>>>>>>>>>>>>>>> something
> > > > >>>>>>>>>>>>>>>>>>>>>>> was
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> unclear.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 1, 2022 at
> > > > >>>>>>> 12:15
> > > > >>>>>>>> PM
> > > > >>>>>>>>>>>> Matthias
> > > > >>>>>>>>>>>>>>> J.
> > > > >>>>>>>>>>>>>>>>> Sax
> > > > >>>>>>>>>>>>>>>>>> <
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the details
> > > > >>>>>>>>> Justine!
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side change
> > > > >>>>>>> for
> > > > >>>>>>>> 2
> > > > >>>>>>>>> is
> > > > >>>>>>>>>>>>>>> removing
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>> addPartitions
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need to
> > > > >>>>>>> make
> > > > >>>>>>>>>> this
> > > > >>>>>>>>>>>> from
> > > > >>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> producer
> > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think I did not
> > > > >>>>>> express
> > > > >>>>>>>>> myself
> > > > >>>>>>>>>>>>>>> clearly. I
> > > > >>>>>>>>>>>>>>>>>>>>> understand
> > > > >>>>>>>>>>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should) change the
> > > > >>>>>>> producer
> > > > >>>>>>>> to
> > > > >>>>>>>>>> not
> > > > >>>>>>>>>>>> send
> > > > >>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> longer. But I don't
> > > > >>>>>> thinks
> > > > >>>>>>>>> it's
> > > > >>>>>>>>>>>>>>> requirement
> > > > >>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>> change
> > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> broker?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What I am trying to say
> > > > >>>>>>> is:
> > > > >>>>>>>>> as a
> > > > >>>>>>>>>>>>>>> safe-guard
> > > > >>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>> improvement
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producers, the partition
> > > > >>>>>>>>> leader
> > > > >>>>>>>>>> can
> > > > >>>>>>>>>>>> just
> > > > >>>>>>>>>>>>>>>> send
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request to the
> > > > >>>>>>>> TX-coordinator
> > > > >>>>>>>>>> in any
> > > > >>>>>>>>>>>>>>> case
> > > > >>>>>>>>>>>>>>>> --
> > > > >>>>>>>>>>>>>>>>> if
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> old
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> correctly did send the
> > > > >>>>>>>>>>>> `addPartition`
> > > > >>>>>>>>>>>>>>>> request
> > > > >>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> already, the
> > > > >>>>>>> TX-coordinator
> > > > >>>>>>>>> can
> > > > >>>>>>>>>> just
> > > > >>>>>>>>>>>>>>>> "ignore"
> > > > >>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>> as
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> idempotent.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if the old producer has
> > > > >>>>>> a
> > > > >>>>>>>> bug
> > > > >>>>>>>>>> and
> > > > >>>>>>>>>>>> did
> > > > >>>>>>>>>>>>>>>> forget
> > > > >>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>> sent
> > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `addPartition`
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request, we would now
> > > > >>>>>>> ensure
> > > > >>>>>>>>>> that
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>> partition
> > > > >>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>> indeed
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> added
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX and thus fix a
> > > > >>>>>>> potential
> > > > >>>>>>>>>>>> producer bug
> > > > >>>>>>>>>>>>>>>>> (even
> > > > >>>>>>>>>>>>>>>>>>>> if we
> > > > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> get
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fencing via the bump
> > > > >>>>>>> epoch).
> > > > >>>>>>>>> --
> > > > >>>>>>>>>> It
> > > > >>>>>>>>>>>>>>> seems to
> > > > >>>>>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>>>> good
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Or
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there a reason to not do
> > > > >>>>>>>> this?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is ongoing
> > > > >>>>>> =
> > > > >>>>>>>>>> partition
> > > > >>>>>>>>>>>> was
> > > > >>>>>>>>>>>>>>>> added
> > > > >>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> via
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn. We
> > > > >>>>>>>> check
> > > > >>>>>>>>>> this
> > > > >>>>>>>>>>>> with
> > > > >>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this wasn't
> > > > >>>>>>>>>> sufficiently
> > > > >>>>>>>>>>>>>>>>> explained
> > > > >>>>>>>>>>>>>>>>>>>> here:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do what I propose
> > > > >>>>>> in
> > > > >>>>>>>>>> (20), we
> > > > >>>>>>>>>>>>>>> don't
> > > > >>>>>>>>>>>>>>>>>> really
> > > > >>>>>>>>>>>>>>>>>>>>> need
> > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> make
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `DescribeTransaction`
> > > > >>>>>>> call,
> > > > >>>>>>>> as
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>> partition
> > > > >>>>>>>>>>>>>>>>>>>> leader
> > > > >>>>>>>>>>>>>>>>>>>>>>> adds
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for older clients and we
> > > > >>>>>>> get
> > > > >>>>>>>>>> this
> > > > >>>>>>>>>>>> check
> > > > >>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>> free.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is that
> > > > >>>>>> if
> > > > >>>>>>>> any
> > > > >>>>>>>>>>>> messages
> > > > >>>>>>>>>>>>>>>>> somehow
> > > > >>>>>>>>>>>>>>>>>>>> come
> > > > >>>>>>>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> before
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the producer,
> > > > >>>>>>> they
> > > > >>>>>>>>>> will be
> > > > >>>>>>>>>>>>>>>> fenced.
> > > > >>>>>>>>>>>>>>>>>>>>> However,
> > > > >>>>>>>>>>>>>>>>>>>>>>> if
> > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it can be
> > > > >>>>>>>>>> discussed
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree that we should
> > > > >>>>>>> have
> > > > >>>>>>>>>> epoch
> > > > >>>>>>>>>>>>>>> fencing.
> > > > >>>>>>>>>>>>>>>> My
> > > > >>>>>>>>>>>>>>>>>>>>>>> question is
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> different:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Assume we are at epoch
> > > > >>>>>> 73,
> > > > >>>>>>>> and
> > > > >>>>>>>>>> we
> > > > >>>>>>>>>>>> have
> > > > >>>>>>>>>>>>>>> an
> > > > >>>>>>>>>>>>>>>>>> ongoing
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed. It seems
> > > > >>>>>>> natural
> > > > >>>>>>>> to
> > > > >>>>>>>>>>>> write the
> > > > >>>>>>>>>>>>>>>>>> "prepare
> > > > >>>>>>>>>>>>>>>>>>>>>>> commit"
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> marker
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
> > > > >>>>>>> both
> > > > >>>>>>>>> with
> > > > >>>>>>>>>>>> epoch
> > > > >>>>>>>>>>>>>>> 73,
> > > > >>>>>>>>>>>>>>>>> too,
> > > > >>>>>>>>>>>>>>>>>>>> as
> > > > >>>>>>>>>>>>>>>>>>>>> it
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> belongs
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current transaction. Of
> > > > >>>>>>>>> course,
> > > > >>>>>>>>>> we
> > > > >>>>>>>>>>>> now
> > > > >>>>>>>>>>>>>>> also
> > > > >>>>>>>>>>>>>>>>>> bump
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> expect
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the next requests to
> > > > >>>>>> have
> > > > >>>>>>>>> epoch
> > > > >>>>>>>>>> 74,
> > > > >>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>>>> reject
> > > > >>>>>>>>>>>>>>>>>>>>>>> an
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch 73, as the
> > > > >>>>>>>> corresponding
> > > > >>>>>>>>>> TX
> > > > >>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>> epoch
> > > > >>>>>>>>>>>>>>>>> 73
> > > > >>>>>>>>>>>>>>>>>>>> was
> > > > >>>>>>>>>>>>>>>>>>>>>>>> already
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems you propose to
> > > > >>>>>>>> write
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>>>> "prepare
> > > > >>>>>>>>>>>>>>>>>> commit
> > > > >>>>>>>>>>>>>>>>>>>>>>> marker"
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
> > > > >>>>>>> with
> > > > >>>>>>>>>> epoch 74
> > > > >>>>>>>>>>>>>>>> though,
> > > > >>>>>>>>>>>>>>>>>> what
> > > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> work,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems confusing. Is
> > > > >>>>>> there
> > > > >>>>>>> a
> > > > >>>>>>>>>> reason
> > > > >>>>>>>>>>>> why
> > > > >>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>>>> use
> > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> bumped
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 74
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of the current
> > > > >>>>>>> epoch
> > > > >>>>>>>>> 73?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are checking if
> > > > >>>>>>> the
> > > > >>>>>>>>>>>>>>> transaction is
> > > > >>>>>>>>>>>>>>>>>>>> ongoing,
> > > > >>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the leader
> > > > >>>>>>>>> partition
> > > > >>>>>>>>>> to
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for this
> > > > >>>>>>>>>> message to
> > > > >>>>>>>>>>>> come
> > > > >>>>>>>>>>>>>>>>> back,
> > > > >>>>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>>>>> theory
> > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
> > > > >>>>>> that
> > > > >>>>>>>>> would
> > > > >>>>>>>>>>>> make the
> > > > >>>>>>>>>>>>>>>>>> original
> > > > >>>>>>>>>>>>>>>>>>>>>>> result
> > > > >>>>>>>>>>>>>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out of
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why we
> > > > >>>>>> can
> > > > >>>>>>>>> check
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>>>> leader
> > > > >>>>>>>>>>>>>>>>>> state
> > > > >>>>>>>>>>>>>>>>>>>>>>> before
> > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks. Got it.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, is this really
> > > > >>>>>> an
> > > > >>>>>>>>>> issue?
> > > > >>>>>>>>>>>> We put
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> produce
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory, so how could
> > > > >>>>>> we
> > > > >>>>>>>>>> process
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> first?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Don't we need to put the
> > > > >>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
> > > > >>>>>>>>>>>>>>>>>>>>> into
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> too,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for this case, and
> > > > >>>>>> process
> > > > >>>>>>>>> both
> > > > >>>>>>>>>>>> request
> > > > >>>>>>>>>>>>>>>>>> in-order?
> > > > >>>>>>>>>>>>>>>>>>>>>>> (Again,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> my
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> knowledge is limited and
> > > > >>>>>>>> maybe
> > > > >>>>>>>>>> we
> > > > >>>>>>>>>>>> don't
> > > > >>>>>>>>>>>>>>>>>> maintain
> > > > >>>>>>>>>>>>>>>>>>>>>>> request
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> order
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case, what seems to be
> > > > >>>>>> an
> > > > >>>>>>>>> issue
> > > > >>>>>>>>>>>> IMHO,
> > > > >>>>>>>>>>>>>>> and I
> > > > >>>>>>>>>>>>>>>>> am
> > > > >>>>>>>>>>>>>>>>>>>>>>> wondering
> > > > >>>>>>>>>>>>>>>>>>>>>>>> if
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> changing
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request handling to
> > > > >>>>>>> preserve
> > > > >>>>>>>>>> order
> > > > >>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>> case
> > > > >>>>>>>>>>>>>>>>>>>>>>> might be
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> cleaner
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution?)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/30/22 3:28 PM,
> > > > >>>>>> Artem
> > > > >>>>>>>>>> Livshits
> > > > >>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think the interesting
> > > > >>>>>>>> part
> > > > >>>>>>>>> is
> > > > >>>>>>>>>>>> not in
> > > > >>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>> logic
> > > > >>>>>>>>>>>>>>>>>>>>>>>> (because
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> tries
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> figure out when
> > > > >>>>>>>>>>>> UNKNOWN_PRODUCER_ID is
> > > > >>>>>>>>>>>>>>>>>> retriable
> > > > >>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>> if
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> it's
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retryable,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's definitely not
> > > > >>>>>>> fatal),
> > > > >>>>>>>>> but
> > > > >>>>>>>>>>>> what
> > > > >>>>>>>>>>>>>>>> happens
> > > > >>>>>>>>>>>>>>>>>>>> when
> > > > >>>>>>>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> logic
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'true' and falls
> > > > >>>>>> through.
> > > > >>>>>>>> In
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>> old
> > > > >>>>>>>>>>>>>>>>> clients
> > > > >>>>>>>>>>>>>>>>>> it
> > > > >>>>>>>>>>>>>>>>>>>>>>> seems
> > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> keep the behavior in
> > > > >>>>>> the
> > > > >>>>>>>> new
> > > > >>>>>>>>>>>> clients,
> > > > >>>>>>>>>>>>>>> I'd
> > > > >>>>>>>>>>>>>>>>>>>> expect it
> > > > >>>>>>>>>>>>>>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> well.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 29, 2022 at
> > > > >>>>>>>> 11:57
> > > > >>>>>>>>>> AM
> > > > >>>>>>>>>>>>>>> Justine
> > > > >>>>>>>>>>>>>>>>>> Olshan
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>> <jolshan@confluent.io.invalid
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Artem and Jeff,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
> > > > >>>>>> look
> > > > >>>>>>>> and
> > > > >>>>>>>>>>>> sorry for
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>> slow
> > > > >>>>>>>>>>>>>>>>>>>>>>>> response.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You both mentioned the
> > > > >>>>>>>>> change
> > > > >>>>>>>>>> to
> > > > >>>>>>>>>>>>>>> handle
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear — this error
> > > > >>>>>> code
> > > > >>>>>>>> will
> > > > >>>>>>>>>> only
> > > > >>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>> sent
> > > > >>>>>>>>>>>>>>>>>> again
> > > > >>>>>>>>>>>>>>>>>>>>> when
> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> client's
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version is high enough
> > > > >>>>>>> to
> > > > >>>>>>>>>> ensure
> > > > >>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>> handle
> > > > >>>>>>>>>>>>>>>>> it
> > > > >>>>>>>>>>>>>>>>>>>>>>>> correctly.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The current (Java)
> > > > >>>>>>> client
> > > > >>>>>>>>>> handles
> > > > >>>>>>>>>>>>>>> this by
> > > > >>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> following
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> (somewhat
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> long)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code snippet:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // An
> > > > >>>>>>> UNKNOWN_PRODUCER_ID
> > > > >>>>>>>>>> means
> > > > >>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>> have
> > > > >>>>>>>>>>>>>>>>>>>> lost
> > > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker. Depending on
> > > > >>>>>> the
> > > > >>>>>>>> log
> > > > >>>>>>>>>> start
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // offset, we may want
> > > > >>>>>>> to
> > > > >>>>>>>>>> retry
> > > > >>>>>>>>>>>>>>> these, as
> > > > >>>>>>>>>>>>>>>>>>>>> described
> > > > >>>>>>>>>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> each
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> case
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below. If
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> none of those apply,
> > > > >>>>>>> then
> > > > >>>>>>>>> for
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // idempotent
> > > > >>>>>> producer,
> > > > >>>>>>> we
> > > > >>>>>>>>>> will
> > > > >>>>>>>>>>>>>>> locally
> > > > >>>>>>>>>>>>>>>>> bump
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> reset
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence numbers of
> > > > >>>>>>>>> in-flight
> > > > >>>>>>>>>>>> batches
> > > > >>>>>>>>>>>>>>>> from
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // sequence 0, then
> > > > >>>>>>> retry
> > > > >>>>>>>>> the
> > > > >>>>>>>>>>>> failed
> > > > >>>>>>>>>>>>>>>> batch,
> > > > >>>>>>>>>>>>>>>>>>>> which
> > > > >>>>>>>>>>>>>>>>>>>>>>>> should
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> now
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> succeed.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the transactional
> > > > >>>>>>>> producer,
> > > > >>>>>>>>>> allow
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // batch to fail. When
> > > > >>>>>>>>>> processing
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>> failed
> > > > >>>>>>>>>>>>>>>>>>>>> batch,
> > > > >>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transition
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an abortable error and
> > > > >>>>>>>> set a
> > > > >>>>>>>>>> flag
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // indicating that we
> > > > >>>>>>> need
> > > > >>>>>>>>> to
> > > > >>>>>>>>>>>> bump the
> > > > >>>>>>>>>>>>>>>>> epoch
> > > > >>>>>>>>>>>>>>>>>>>> (if
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> supported
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> by
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker).
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if (error ==
> > > > >>>>>>>>>>>>>>>> Errors.*UNKNOWN_PRODUCER_ID*)
> > > > >>>>>>>>>>>>>>>>> {
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > > > >>>>>>>>>> (response.logStartOffset
> > > > >>>>>>>>>>>> ==
> > > > >>>>>>>>>>>>>>> -1)
> > > > >>>>>>>>>>>>>>>> {
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // We don't
> > > > >>>>>>> know
> > > > >>>>>>>>>> the log
> > > > >>>>>>>>>>>>>>> start
> > > > >>>>>>>>>>>>>>>>>> offset
> > > > >>>>>>>>>>>>>>>>>>>>> with
> > > > >>>>>>>>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> response.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the request
> > > > >>>>>>>> until
> > > > >>>>>>>>>> we
> > > > >>>>>>>>>>>> get
> > > > >>>>>>>>>>>>>>> it.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // The
> > > > >>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > >>>>>>>>>>>>>>>> error
> > > > >>>>>>>>>>>>>>>>>> code
> > > > >>>>>>>>>>>>>>>>>>>>> was
> > > > >>>>>>>>>>>>>>>>>>>>>>>> added
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> along
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProduceResponse which
> > > > >>>>>>>>>> includes the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              //
> > > > >>>>>>>> logStartOffset.
> > > > >>>>>>>>>> So
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>> '-1'
> > > > >>>>>>>>>>>>>>>>>>>> sentinel
> > > > >>>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> backward
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatibility.
> > > > >>>>>> Instead,
> > > > >>>>>>> it
> > > > >>>>>>>>> is
> > > > >>>>>>>>>>>> possible
> > > > >>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // a broker
> > > > >>>>>> to
> > > > >>>>>>>> not
> > > > >>>>>>>>>> know
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> logStartOffset at
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> when
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returning
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response because
> > > > >>>>>> the
> > > > >>>>>>>>>> partition
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // may have
> > > > >>>>>>>> moved
> > > > >>>>>>>>>> away
> > > > >>>>>>>>>>>> from
> > > > >>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> broker
> > > > >>>>>>>>>>>>>>>>>>>>>>> from
> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error was
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initially raised to
> > > > >>>>>> the
> > > > >>>>>>>> time
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // response
> > > > >>>>>>> was
> > > > >>>>>>>>>> being
> > > > >>>>>>>>>>>>>>>>> constructed.
> > > > >>>>>>>>>>>>>>>>>> In
> > > > >>>>>>>>>>>>>>>>>>>>>>> these
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> cases,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retry the request: we
> > > > >>>>>>> are
> > > > >>>>>>>>>>>> guaranteed
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // to
> > > > >>>>>>> eventually
> > > > >>>>>>>>>> get a
> > > > >>>>>>>>>>>>>>>>>> logStartOffset
> > > > >>>>>>>>>>>>>>>>>>>>> once
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> things
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> settle
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> down.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > > > >>>>>>>>>>>>>>> (batch.sequenceHasBeenReset()) {
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // When the
> > > > >>>>>>>> first
> > > > >>>>>>>>>>>> inflight
> > > > >>>>>>>>>>>>>>>> batch
> > > > >>>>>>>>>>>>>>>>>>>> fails
> > > > >>>>>>>>>>>>>>>>>>>>>>> due to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> truncation
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> then the sequences of
> > > > >>>>>>> all
> > > > >>>>>>>>> the
> > > > >>>>>>>>>>>> other
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // in flight
> > > > >>>>>>>>> batches
> > > > >>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>> have
> > > > >>>>>>>>>>>>>>>>>> been
> > > > >>>>>>>>>>>>>>>>>>>>>>>> restarted
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> from
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> beginning.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, when those
> > > > >>>>>>>>> responses
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // come back
> > > > >>>>>>>> from
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>>>> broker,
> > > > >>>>>>>>>>>>>>>>> they
> > > > >>>>>>>>>>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>>>>>>> also
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> come
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > >>>>>>> error.
> > > > >>>>>>>>> In
> > > > >>>>>>>>>> this
> > > > >>>>>>>>>>>>>>> case,
> > > > >>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>> should
> > > > >>>>>>>>>>>>>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // reset the
> > > > >>>>>>>>>> sequence
> > > > >>>>>>>>>>>>>>> numbers
> > > > >>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>> beginning.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          } else if
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>> (lastAckedOffset(batch.topicPartition).orElse(
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> response.logStartOffset) {
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // The head
> > > > >>>>>> of
> > > > >>>>>>>> the
> > > > >>>>>>>>>> log
> > > > >>>>>>>>>>>> has
> > > > >>>>>>>>>>>>>>> been
> > > > >>>>>>>>>>>>>>>>>>>> removed,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> probably
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> due
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retention time
> > > > >>>>>> elapsing.
> > > > >>>>>>>> In
> > > > >>>>>>>>>> this
> > > > >>>>>>>>>>>> case,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // we expect
> > > > >>>>>>> to
> > > > >>>>>>>>>> lose the
> > > > >>>>>>>>>>>>>>>> producer
> > > > >>>>>>>>>>>>>>>>>>>> state.
> > > > >>>>>>>>>>>>>>>>>>>>>>> For
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer, reset the
> > > > >>>>>>>>> sequences
> > > > >>>>>>>>>> of
> > > > >>>>>>>>>>>> all
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // inflight
> > > > >>>>>>>>> batches
> > > > >>>>>>>>>> to
> > > > >>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>> from
> > > > >>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> beginning
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> retry
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the transaction
> > > > >>>>>>> does
> > > > >>>>>>>>> not
> > > > >>>>>>>>>>>> need to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // be
> > > > >>>>>> aborted.
> > > > >>>>>>>> For
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>> idempotent
> > > > >>>>>>>>>>>>>>>>>>>>>>> producer,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> bump
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> avoid
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reusing (sequence,
> > > > >>>>>>> epoch)
> > > > >>>>>>>>>> pairs
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              if
> > > > >>>>>>>>>> (isTransactional()) {
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>
> > > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>> this.producerIdAndEpoch);
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              } else {
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> requestEpochBumpForPartition(batch.topicPartition);
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              }
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > > > >>>>>>>>> (!isTransactional())
> > > > >>>>>>>>>> {
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // For the
> > > > >>>>>>>>>> idempotent
> > > > >>>>>>>>>>>>>>> producer,
> > > > >>>>>>>>>>>>>>>>>>>> always
> > > > >>>>>>>>>>>>>>>>>>>>>>> retry
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors. If the batch
> > > > >>>>>> has
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>> current
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // producer
> > > > >>>>>> ID
> > > > >>>>>>>> and
> > > > >>>>>>>>>>>> epoch,
> > > > >>>>>>>>>>>>>>>>> request a
> > > > >>>>>>>>>>>>>>>>>>>> bump
> > > > >>>>>>>>>>>>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Otherwise
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the
> > > > >>>>>> produce.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>> requestEpochBumpForPartition(batch.topicPartition);
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I was considering
> > > > >>>>>>> keeping
> > > > >>>>>>>>> this
> > > > >>>>>>>>>>>>>>> behavior —
> > > > >>>>>>>>>>>>>>>>> but
> > > > >>>>>>>>>>>>>>>>>>>> am
> > > > >>>>>>>>>>>>>>>>>>>>>>> open
> > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simplifying
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We are leaving changes
> > > > >>>>>>> to
> > > > >>>>>>>>>> older
> > > > >>>>>>>>>>>>>>> clients
> > > > >>>>>>>>>>>>>>>> off
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> table
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> here
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> since
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> caused
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many issues for
> > > > >>>>>> clients
> > > > >>>>>>> in
> > > > >>>>>>>>> the
> > > > >>>>>>>>>>>> past.
> > > > >>>>>>>>>>>>>>>>>> Previously
> > > > >>>>>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>>>>> was
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we didn't have the
> > > > >>>>>>>>> mechanisms
> > > > >>>>>>>>>> in
> > > > >>>>>>>>>>>>>>> place to
> > > > >>>>>>>>>>>>>>>>>>>> detect
> > > > >>>>>>>>>>>>>>>>>>>>>>> when
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> was
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> legitimate
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case vs some bug or
> > > > >>>>>> gap
> > > > >>>>>>> in
> > > > >>>>>>>>> the
> > > > >>>>>>>>>>>>>>> protocol.
> > > > >>>>>>>>>>>>>>>>>>>> Ensuring
> > > > >>>>>>>>>>>>>>>>>>>>>>> each
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> its
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> own epoch should close
> > > > >>>>>>>> this
> > > > >>>>>>>>>> gap.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> And to address Jeff's
> > > > >>>>>>>> second
> > > > >>>>>>>>>>>> point:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *does the typical
> > > > >>>>>>> produce
> > > > >>>>>>>>>> request
> > > > >>>>>>>>>>>> path
> > > > >>>>>>>>>>>>>>>>> append
> > > > >>>>>>>>>>>>>>>>>>>>>>> records
> > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> local
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> log
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along*
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *with the
> > > > >>>>>>>>>> currentTxnFirstOffset
> > > > >>>>>>>>>>>>>>>>> information?
> > > > >>>>>>>>>>>>>>>>>> I
> > > > >>>>>>>>>>>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>>>>>>>> like
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand*
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *when the field is
> > > > >>>>>>> written
> > > > >>>>>>>>> to
> > > > >>>>>>>>>>>> disk.*
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, the first produce
> > > > >>>>>>>>> request
> > > > >>>>>>>>>>>>>>> populates
> > > > >>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>> field
> > > > >>>>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> writes
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> offset
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as part of the record
> > > > >>>>>>>> batch
> > > > >>>>>>>>>> and
> > > > >>>>>>>>>>>> also
> > > > >>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>> producer
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> state
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> snapshot.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we reload the records
> > > > >>>>>> on
> > > > >>>>>>>>>> restart
> > > > >>>>>>>>>>>>>>> and/or
> > > > >>>>>>>>>>>>>>>>>>>>>>> reassignment,
> > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> repopulate
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> field with the
> > > > >>>>>> snapshot
> > > > >>>>>>>> from
> > > > >>>>>>>>>> disk
> > > > >>>>>>>>>>>>>>> along
> > > > >>>>>>>>>>>>>>>>> with
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> rest
> > > > >>>>>>>>>>>>>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if there
> > > > >>>>>> are
> > > > >>>>>>>>>> further
> > > > >>>>>>>>>>>>>>> comments
> > > > >>>>>>>>>>>>>>>>>>>> and/or
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> questions.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > > > >>>>>> at
> > > > >>>>>>>> 9:00
> > > > >>>>>>>>>> PM
> > > > >>>>>>>>>>>> Jeff
> > > > >>>>>>>>>>>>>>> Kim
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>> <jeff.kim@confluent.io.invalid
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP! I
> > > > >>>>>>>> have
> > > > >>>>>>>>>> two
> > > > >>>>>>>>>>>>>>>> questions:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1) For new clients,
> > > > >>>>>> we
> > > > >>>>>>>> can
> > > > >>>>>>>>>> once
> > > > >>>>>>>>>>>> again
> > > > >>>>>>>>>>>>>>>>> return
> > > > >>>>>>>>>>>>>>>>>>>> an
> > > > >>>>>>>>>>>>>>>>>>>>>>> error
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for sequences
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that are non-zero
> > > > >>>>>> when
> > > > >>>>>>>>> there
> > > > >>>>>>>>>> is
> > > > >>>>>>>>>>>> no
> > > > >>>>>>>>>>>>>>>>> producer
> > > > >>>>>>>>>>>>>>>>>>>> state
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> present
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> on
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This will indicate we
> > > > >>>>>>>>> missed
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>> 0
> > > > >>>>>>>>>>>>>>>>> sequence
> > > > >>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> yet
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> want
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the log.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would like to
> > > > >>>>>>>> understand
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>>>> current
> > > > >>>>>>>>>>>>>>>>>>>> behavior
> > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> handle
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and if there are any
> > > > >>>>>>>>> changes
> > > > >>>>>>>>>> we
> > > > >>>>>>>>>>>> are
> > > > >>>>>>>>>>>>>>>>> making.
> > > > >>>>>>>>>>>>>>>>>>>> Maybe
> > > > >>>>>>>>>>>>>>>>>>>>>>> I'm
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> missing
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but we would want to
> > > > >>>>>>>>> identify
> > > > >>>>>>>>>>>>>>> whether we
> > > > >>>>>>>>>>>>>>>>>>>> missed
> > > > >>>>>>>>>>>>>>>>>>>>>>> the 0
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> sequence
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients, no?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2) Upon returning
> > > > >>>>>> from
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>>>>> coordinator, we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> set
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as ongoing on the
> > > > >>>>>>> leader
> > > > >>>>>>>> by
> > > > >>>>>>>>>>>>>>> populating
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> currentTxnFirstOffset
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through the typical
> > > > >>>>>>>> produce
> > > > >>>>>>>>>>>> request
> > > > >>>>>>>>>>>>>>>>>> handling.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does the typical
> > > > >>>>>>> produce
> > > > >>>>>>>>>> request
> > > > >>>>>>>>>>>> path
> > > > >>>>>>>>>>>>>>>>> append
> > > > >>>>>>>>>>>>>>>>>>>>>>> records
> > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> local
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> log
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> > > > >>>>>>>>>> currentTxnFirstOffset
> > > > >>>>>>>>>>>>>>>>> information?
> > > > >>>>>>>>>>>>>>>>>> I
> > > > >>>>>>>>>>>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>>>>>>>> like
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when the field is
> > > > >>>>>>> written
> > > > >>>>>>>>> to
> > > > >>>>>>>>>>>> disk.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeff
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > > > >>>>>> at
> > > > >>>>>>>>> 4:44
> > > > >>>>>>>>>> PM
> > > > >>>>>>>>>>>> Artem
> > > > >>>>>>>>>>>>>>>>>> Livshits
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
> > > > >>>>>>> alivshits@confluent.io
> > > > >>>>>>>>>> .invalid>
> > > > >>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for the
> > > > >>>>>> KIP.
> > > > >>>>>>>> I
> > > > >>>>>>>>>> have
> > > > >>>>>>>>>>>> one
> > > > >>>>>>>>>>>>>>>>>> question.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 5) For new clients,
> > > > >>>>>> we
> > > > >>>>>>>> can
> > > > >>>>>>>>>> once
> > > > >>>>>>>>>>>>>>> again
> > > > >>>>>>>>>>>>>>>>>> return
> > > > >>>>>>>>>>>>>>>>>>>> an
> > > > >>>>>>>>>>>>>>>>>>>>>>> error
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe we had
> > > > >>>>>>>> problems
> > > > >>>>>>>>>> in the
> > > > >>>>>>>>>>>>>>> past
> > > > >>>>>>>>>>>>>>>>> with
> > > > >>>>>>>>>>>>>>>>>>>>>>> returning
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because it was
> > > > >>>>>>>> considered
> > > > >>>>>>>>>> fatal
> > > > >>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>> required
> > > > >>>>>>>>>>>>>>>>>>>>>>> client
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> restart.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> It
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good to spell out
> > > > >>>>>> the
> > > > >>>>>>>> new
> > > > >>>>>>>>>> client
> > > > >>>>>>>>>>>>>>>> behavior
> > > > >>>>>>>>>>>>>>>>>>>> when
> > > > >>>>>>>>>>>>>>>>>>>>> it
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> receives
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > > > >>>>>>> at
> > > > >>>>>>>>>> 10:00 AM
> > > > >>>>>>>>>>>>>>>> Justine
> > > > >>>>>>>>>>>>>>>>>>>> Olshan
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>> <jo...@confluent.io.invalid>
> > > > >>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
> > > > >>>>>>>> look
> > > > >>>>>>>>>>>> Matthias.
> > > > >>>>>>>>>>>>>>>> I've
> > > > >>>>>>>>>>>>>>>>>>>> tried
> > > > >>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> answer
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> your
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> questions
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 10)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Right — so the
> > > > >>>>>>> hanging
> > > > >>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>> only
> > > > >>>>>>>>>>>>>>>>>>>> occurs
> > > > >>>>>>>>>>>>>>>>>>>>>>> when
> > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> come in and the
> > > > >>>>>>>> partition
> > > > >>>>>>>>>> is
> > > > >>>>>>>>>>>> never
> > > > >>>>>>>>>>>>>>>> added
> > > > >>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> again.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never add the
> > > > >>>>>>> partition
> > > > >>>>>>>>> to
> > > > >>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>> transaction,
> > > > >>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>> will
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> never
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> marker
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never advance the
> > > > >>>>>>> LSO.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do end up
> > > > >>>>>>> adding
> > > > >>>>>>>>> the
> > > > >>>>>>>>>>>>>>> partition
> > > > >>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (I
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suppose
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happen before or
> > > > >>>>>>> after
> > > > >>>>>>>>> the
> > > > >>>>>>>>>> late
> > > > >>>>>>>>>>>>>>>> message
> > > > >>>>>>>>>>>>>>>>>>>> comes
> > > > >>>>>>>>>>>>>>>>>>>>> in)
> > > > >>>>>>>>>>>>>>>>>>>>>>>> then
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> include
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late message in the
> > > > >>>>>>>> next
> > > > >>>>>>>>>>>>>>> (incorrect)
> > > > >>>>>>>>>>>>>>>>>>>>> transaction.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> So perhaps it is
> > > > >>>>>>>> clearer
> > > > >>>>>>>>> to
> > > > >>>>>>>>>>>> make
> > > > >>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>> distinction
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> between
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> eventually get
> > > > >>>>>> added
> > > > >>>>>>> to
> > > > >>>>>>>>> the
> > > > >>>>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>> (but
> > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>> wrong
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> one)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> or
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that never get
> > > > >>>>>> added
> > > > >>>>>>>> and
> > > > >>>>>>>>>> become
> > > > >>>>>>>>>>>>>>>> hanging.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side
> > > > >>>>>>> change
> > > > >>>>>>>>> for
> > > > >>>>>>>>>> 2 is
> > > > >>>>>>>>>>>>>>>> removing
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> addPartitions
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need
> > > > >>>>>>> to
> > > > >>>>>>>>> make
> > > > >>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>> from
> > > > >>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> producer
> > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In my opinion, the
> > > > >>>>>>>> issue
> > > > >>>>>>>>>> with
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> call
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is that we don't
> > > > >>>>>> have
> > > > >>>>>>>> the
> > > > >>>>>>>>>> epoch
> > > > >>>>>>>>>>>>>>> bump,
> > > > >>>>>>>>>>>>>>>> so
> > > > >>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>> don't
> > > > >>>>>>>>>>>>>>>>>>>>>>>> know
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> belongs to the
> > > > >>>>>>> previous
> > > > >>>>>>>>>>>>>>> transaction or
> > > > >>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>> one.
> > > > >>>>>>>>>>>>>>>>>>>>>>> We
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition has been
> > > > >>>>>>>> added
> > > > >>>>>>>>> to
> > > > >>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>> transaction.
> > > > >>>>>>>>>>>>>>>>>>>>> Of
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> course,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> means
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't completely
> > > > >>>>>>> cover
> > > > >>>>>>>>> the
> > > > >>>>>>>>>> case
> > > > >>>>>>>>>>>>>>> where
> > > > >>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>> have a
> > > > >>>>>>>>>>>>>>>>>>>>>>>> really
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have added the
> > > > >>>>>>>> partition
> > > > >>>>>>>>> to
> > > > >>>>>>>>>>>> the new
> > > > >>>>>>>>>>>>>>>>>>>>> transaction,
> > > > >>>>>>>>>>>>>>>>>>>>>>> but
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> that's
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something we will
> > > > >>>>>>> need
> > > > >>>>>>>>> the
> > > > >>>>>>>>>> new
> > > > >>>>>>>>>>>>>>> clients
> > > > >>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>> cover.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is
> > > > >>>>>>> ongoing
> > > > >>>>>>>> =
> > > > >>>>>>>>>>>> partition
> > > > >>>>>>>>>>>>>>> was
> > > > >>>>>>>>>>>>>>>>>>>> added to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> via
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn.
> > > > >>>>>>> We
> > > > >>>>>>>>>> check
> > > > >>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>> with
> > > > >>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this
> > > > >>>>>>> wasn't
> > > > >>>>>>>>>>>> sufficiently
> > > > >>>>>>>>>>>>>>>>>>>> explained
> > > > >>>>>>>>>>>>>>>>>>>>>>> here:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>
> > > > >>>>>>
> > > > >>>>
> > > > >>
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is
> > > > >>>>>> that
> > > > >>>>>>>> if
> > > > >>>>>>>>>> any
> > > > >>>>>>>>>>>>>>> messages
> > > > >>>>>>>>>>>>>>>>>>>> somehow
> > > > >>>>>>>>>>>>>>>>>>>>>>> come
> > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> before
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the
> > > > >>>>>>> producer,
> > > > >>>>>>>>> they
> > > > >>>>>>>>>>>> will be
> > > > >>>>>>>>>>>>>>>>>> fenced.
> > > > >>>>>>>>>>>>>>>>>>>>>>> However,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it
> > > > >>>>>> can
> > > > >>>>>>> be
> > > > >>>>>>>>>>>> discussed
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 50)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It should be
> > > > >>>>>>>> synchronous
> > > > >>>>>>>>>>>> because
> > > > >>>>>>>>>>>>>>> if we
> > > > >>>>>>>>>>>>>>>>>> have
> > > > >>>>>>>>>>>>>>>>>>>> an
> > > > >>>>>>>>>>>>>>>>>>>>>>> event
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (ie,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> an
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes us to need
> > > > >>>>>> to
> > > > >>>>>>>>> abort
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>> transaction,
> > > > >>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>> need
> > > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions to send
> > > > >>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>> markers
> > > > >>>>>>>>>>>>>>>>> to.
> > > > >>>>>>>>>>>>>>>>>>>> We
> > > > >>>>>>>>>>>>>>>>>>>>>>> know
> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we added them to
> > > > >>>>>> the
> > > > >>>>>>>>>>>> coordinator
> > > > >>>>>>>>>>>>>>> via
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Previously we have
> > > > >>>>>>> had
> > > > >>>>>>>>>>>> asynchronous
> > > > >>>>>>>>>>>>>>>>> calls
> > > > >>>>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>> past
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (ie,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit markers when
> > > > >>>>>>> the
> > > > >>>>>>>>>>>>>>> transaction is
> > > > >>>>>>>>>>>>>>>>>>>>> completed)
> > > > >>>>>>>>>>>>>>>>>>>>>>>> but
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> often
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes confusion as
> > > > >>>>>>> we
> > > > >>>>>>>>>> need to
> > > > >>>>>>>>>>>> wait
> > > > >>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>> some
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> operations
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complete.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing commit
> > > > >>>>>>> markers
> > > > >>>>>>>>>> case,
> > > > >>>>>>>>>>>>>>> clients
> > > > >>>>>>>>>>>>>>>>> often
> > > > >>>>>>>>>>>>>>>>>>>> see
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>> CONCURRENT_TRANSACTIONs
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error messages and
> > > > >>>>>>> that
> > > > >>>>>>>>>> can be
> > > > >>>>>>>>>>>>>>>>> confusing.
> > > > >>>>>>>>>>>>>>>>>>>> For
> > > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> reason,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> may
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simpler to just
> > > > >>>>>> have
> > > > >>>>>>>>>>>> synchronous
> > > > >>>>>>>>>>>>>>>> calls —
> > > > >>>>>>>>>>>>>>>>>>>>>>> especially
> > > > >>>>>>>>>>>>>>>>>>>>>>>> if
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> block
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some operation's
> > > > >>>>>>>>> completion
> > > > >>>>>>>>>>>> anyway
> > > > >>>>>>>>>>>>>>>>> before
> > > > >>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>> can
> > > > >>>>>>>>>>>>>>>>>>>>>>>> start
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> next
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. And
> > > > >>>>>>> yes, I
> > > > >>>>>>>>>> meant
> > > > >>>>>>>>>>>>>>>>>> coordinator. I
> > > > >>>>>>>>>>>>>>>>>>>>> will
> > > > >>>>>>>>>>>>>>>>>>>>>>>> fix
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> that.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are
> > > > >>>>>> checking
> > > > >>>>>>> if
> > > > >>>>>>>>> the
> > > > >>>>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>> ongoing,
> > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> need
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the
> > > > >>>>>> leader
> > > > >>>>>>>>>> partition
> > > > >>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for
> > > > >>>>>>> this
> > > > >>>>>>>>>>>> message to
> > > > >>>>>>>>>>>>>>>> come
> > > > >>>>>>>>>>>>>>>>>>>> back,
> > > > >>>>>>>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> theory
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
> > > > >>>>>>>> that
> > > > >>>>>>>>>> would
> > > > >>>>>>>>>>>> make
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>> original
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> result
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why
> > > > >>>>>> we
> > > > >>>>>>>> can
> > > > >>>>>>>>>> check
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>> leader
> > > > >>>>>>>>>>>>>>>>>>>> state
> > > > >>>>>>>>>>>>>>>>>>>>>>>> before
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm happy to update
> > > > >>>>>>> the
> > > > >>>>>>>>>> KIP if
> > > > >>>>>>>>>>>>>>> some of
> > > > >>>>>>>>>>>>>>>>>> these
> > > > >>>>>>>>>>>>>>>>>>>>>>> things
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> were
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Nov 21,
> > > > >>>>>> 2022
> > > > >>>>>>> at
> > > > >>>>>>>>>> 7:11 PM
> > > > >>>>>>>>>>>>>>>> Matthias
> > > > >>>>>>>>>>>>>>>>>> J.
> > > > >>>>>>>>>>>>>>>>>>>>> Sax <
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the
> > > > >>>>>> KIP.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Couple of
> > > > >>>>>>>> clarification
> > > > >>>>>>>>>>>> questions
> > > > >>>>>>>>>>>>>>> (I
> > > > >>>>>>>>>>>>>>>> am
> > > > >>>>>>>>>>>>>>>>>>>> not a
> > > > >>>>>>>>>>>>>>>>>>>>>>>> broker
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> expert
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> do
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some question are
> > > > >>>>>>>>> obvious
> > > > >>>>>>>>>> for
> > > > >>>>>>>>>>>>>>> others,
> > > > >>>>>>>>>>>>>>>>> but
> > > > >>>>>>>>>>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>>>>>> me
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> my
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lack
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker knowledge).
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (10)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What happens if
> > > > >>>>>> the
> > > > >>>>>>>>>> message
> > > > >>>>>>>>>>>> come
> > > > >>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>> before
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> next
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request? It seems
> > > > >>>>>>> the
> > > > >>>>>>>>>> broker
> > > > >>>>>>>>>>>>>>> hosting
> > > > >>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> data
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anything about it
> > > > >>>>>>> and
> > > > >>>>>>>>>> append
> > > > >>>>>>>>>>>> it to
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> partition,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> too?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> What
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> difference between
> > > > >>>>>>>> both
> > > > >>>>>>>>>> cases?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, it seems a
> > > > >>>>>> TX
> > > > >>>>>>>>> would
> > > > >>>>>>>>>> only
> > > > >>>>>>>>>>>>>>> hang,
> > > > >>>>>>>>>>>>>>>> if
> > > > >>>>>>>>>>>>>>>>>>>> there
> > > > >>>>>>>>>>>>>>>>>>>>>>> is no
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> following
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> either committer
> > > > >>>>>> or
> > > > >>>>>>>>>> aborted?
> > > > >>>>>>>>>>>> Thus,
> > > > >>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>> case
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> above,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> TX
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> might
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually not hang
> > > > >>>>>>> (of
> > > > >>>>>>>>>> course,
> > > > >>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>> might
> > > > >>>>>>>>>>>>>>>>>> get
> > > > >>>>>>>>>>>>>>>>>>>> an
> > > > >>>>>>>>>>>>>>>>>>>>>>> EOS
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> violation
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX was aborted and
> > > > >>>>>>> the
> > > > >>>>>>>>>> second
> > > > >>>>>>>>>>>>>>>>> committed,
> > > > >>>>>>>>>>>>>>>>>> or
> > > > >>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>> other
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> way
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around).
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (20)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Of course, 1 and
> > > > >>>>>> 2
> > > > >>>>>>>>>> require
> > > > >>>>>>>>>>>>>>>> client-side
> > > > >>>>>>>>>>>>>>>>>>>>>>> changes, so
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> those approaches
> > > > >>>>>>> won’t
> > > > >>>>>>>>>> apply.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For (1) I
> > > > >>>>>> understand
> > > > >>>>>>>>> why a
> > > > >>>>>>>>>>>> client
> > > > >>>>>>>>>>>>>>>>> change
> > > > >>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> necessary,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we need a client
> > > > >>>>>>>> change
> > > > >>>>>>>>>> for
> > > > >>>>>>>>>>>> (2).
> > > > >>>>>>>>>>>>>>> Can
> > > > >>>>>>>>>>>>>>>>> you
> > > > >>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> --
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Later
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> explain
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that we should
> > > > >>>>>> send
> > > > >>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>>>>>> DescribeTransactionRequest,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> but I
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> am
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Can't we not just
> > > > >>>>>> do
> > > > >>>>>>>> an
> > > > >>>>>>>>>>>> implicit
> > > > >>>>>>>>>>>>>>>>>>>>>>> AddPartiitonToTx,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> too?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> If
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> old
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer correctly
> > > > >>>>>>>>>> registered
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>> partition
> > > > >>>>>>>>>>>>>>>>>>>>>>>> already,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can just ignore it
> > > > >>>>>>> as
> > > > >>>>>>>>>> it's an
> > > > >>>>>>>>>>>>>>>>> idempotent
> > > > >>>>>>>>>>>>>>>>>>>>>>> operation?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (30)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To cover older
> > > > >>>>>>>> clients,
> > > > >>>>>>>>>> we
> > > > >>>>>>>>>>>> will
> > > > >>>>>>>>>>>>>>>>> ensure a
> > > > >>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ongoing
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> before
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we write to a
> > > > >>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Not sure what you
> > > > >>>>>>> mean
> > > > >>>>>>>>> by
> > > > >>>>>>>>>>>> this?
> > > > >>>>>>>>>>>>>>> Can
> > > > >>>>>>>>>>>>>>>> you
> > > > >>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (40)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [the
> > > > >>>>>>> TX-coordinator]
> > > > >>>>>>>>> will
> > > > >>>>>>>>>>>> write
> > > > >>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> prepare
> > > > >>>>>>>>>>>>>>>>>>>>>>> commit
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch and send
> > > > >>>>>>>>>>>>>>> WriteTxnMarkerRequests
> > > > >>>>>>>>>>>>>>>>>> with
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>> bumped
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why do we use the
> > > > >>>>>>>> bumped
> > > > >>>>>>>>>>>> epoch for
> > > > >>>>>>>>>>>>>>>>> both?
> > > > >>>>>>>>>>>>>>>>>> It
> > > > >>>>>>>>>>>>>>>>>>>>>>> seems
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> more
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> intuitive
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the current epoch,
> > > > >>>>>>> and
> > > > >>>>>>>>>> only
> > > > >>>>>>>>>>>> return
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> bumped
> > > > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (50) "Implicit
> > > > >>>>>>>>>>>>>>>>> AddPartitionToTransaction"
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why does the
> > > > >>>>>>>> implicitly
> > > > >>>>>>>>>> sent
> > > > >>>>>>>>>>>>>>> request
> > > > >>>>>>>>>>>>>>>>> need
> > > > >>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> synchronous?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also says
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in case we need
> > > > >>>>>> to
> > > > >>>>>>>>> abort
> > > > >>>>>>>>>> and
> > > > >>>>>>>>>>>>>>> need to
> > > > >>>>>>>>>>>>>>>>>> know
> > > > >>>>>>>>>>>>>>>>>>>>> which
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What do you mean
> > > > >>>>>> by
> > > > >>>>>>>>> this?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we don’t want to
> > > > >>>>>>>> write
> > > > >>>>>>>>>> to it
> > > > >>>>>>>>>>>>>>> before
> > > > >>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>> store
> > > > >>>>>>>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> manager
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Do you mean
> > > > >>>>>>>>> TX-coordinator
> > > > >>>>>>>>>>>>>>> instead of
> > > > >>>>>>>>>>>>>>>>>>>>> "manager"?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (60)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For older clients
> > > > >>>>>>> and
> > > > >>>>>>>>>> ensuring
> > > > >>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>> TX
> > > > >>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> ongoing,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> describe a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> race condition. I
> > > > >>>>>> am
> > > > >>>>>>>> not
> > > > >>>>>>>>>> sure
> > > > >>>>>>>>>>>> if I
> > > > >>>>>>>>>>>>>>>> can
> > > > >>>>>>>>>>>>>>>>>>>> follow
> > > > >>>>>>>>>>>>>>>>>>>>>>> here.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> Can
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/18/22 1:21
> > > > >>>>>> PM,
> > > > >>>>>>>>>> Justine
> > > > >>>>>>>>>>>>>>> Olshan
> > > > >>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey all!
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to
> > > > >>>>>> start a
> > > > >>>>>>>>>>>> discussion
> > > > >>>>>>>>>>>>>>> on my
> > > > >>>>>>>>>>>>>>>>>>>> proposal
> > > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>> add
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> some
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checks on
> > > > >>>>>>>> transactions
> > > > >>>>>>>>> to
> > > > >>>>>>>>>>>> avoid
> > > > >>>>>>>>>>>>>>>>> hanging
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> transactions.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> been
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an issue for some
> > > > >>>>>>>> time,
> > > > >>>>>>>>>> so I
> > > > >>>>>>>>>>>>>>> really
> > > > >>>>>>>>>>>>>>>>> hope
> > > > >>>>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>>>> KIP
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> helpful
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users of EOS.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The KIP includes
> > > > >>>>>>>>> changes
> > > > >>>>>>>>>> that
> > > > >>>>>>>>>>>>>>> will
> > > > >>>>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>>>>> compatible
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> with
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> old
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> changes to
> > > > >>>>>> improve
> > > > >>>>>>>>>>>> performance
> > > > >>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>> correctness
> > > > >>>>>>>>>>>>>>>>>>>>>>> on
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> new
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Please take a
> > > > >>>>>> look
> > > > >>>>>>>> and
> > > > >>>>>>>>>> leave
> > > > >>>>>>>>>>>> any
> > > > >>>>>>>>>>>>>>>>>> comments
> > > > >>>>>>>>>>>>>>>>>>>> you
> > > > >>>>>>>>>>>>>>>>>>>>>>> may
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> have!
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>
> > > > >>>>>>
> > > > >>>>
> > > > >>
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> JIRA:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-14402
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>
> > > > >>>>>>
> > > > >>>>>
> > > > >>>>
> > > > >>>
> > > > >>
> > > > >
> > >
> >

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Artem Livshits <al...@confluent.io.INVALID>.
I agree.  I also think we should go a little further and be more
prescriptive with our recommendations for error handling (i.e. not just
think if there is a chance, but propose design patterns for typical
cases).  For example, if a transaction is aborted, the application must
either keep enough state to be able to redo the operation from the
beginning, or reset all state and fully recover from Kafka (i.e. re-consume
data from the last committed offsets).

For failed commits, we need a way for the application to either learn the
result (if commit went through, then continue, otherwise redo operation) or
just reset all state and fully recover from Kafka (i.e. re-consume data
from the last committed offsets).

-Artem

On Fri, Jan 27, 2023 at 9:31 AM Justine Olshan <jo...@confluent.io.invalid>
wrote:

> Thanks for clarifying Guozhang,
>
> I like the framing:
> * “if we did abort the txn and continue, could the app ever has a chance to
> recover and not have any side effects violating EOS guarantees”.*
>
> With this principle in mind, if we could convince ourselves that we won't
> have side effects, then we could move forward with a non-fatal error. I
> appreciate laying out the cases we've discussed. I think I also want to
> take a look in very close detail and convince myself of the right option.
> I'll get back to this thread with the final conclusion as well as update
> the KIP.
>
> Thanks for also bringing up KIP-691. I should take a look there as well. :)
>
> Justine
>
> On Thu, Jan 26, 2023 at 5:28 AM Guozhang Wang <gu...@gmail.com>
> wrote:
>
> > Hello Justine,
> >
> > Regarding which errors should be fatal v.s. abortable, I think the
> > principle would be “if we did abort the txn and continue, could the
> > app ever has a chance to recover and not have any side effects
> > violating EOS guarantees”.Of course we can discuss whether that’s a
> > good principle  e.g. one can argue that maybe it’s okay to violate EOS
> > some times rather than killing the whole app, whereas my rationale is
> > that since we want to make EOS the default config, it’s crucial that
> > we try to guarantee it as much as possible and whenever we detected if
> > there’s a chance that it maybe broken, we should let users know. And
> > if we just log an error in app and try to abort and continue, it’s
> > very likely that users would not be notified and probably only realize
> > that after a while, which may cause an even harder scenario to
> > debug.If people agree on that principle, we can go back and check this
> > KIP’s scenario: there are several scenarios causing a partition leader
> > detects an out of date epoch from a produce requests:
> >
> > * The old produce requests arrive late after a network partition, and
> > the old producer is already gone.
> > * The old produce requests arrive late after a network partition, but
> > the old producer is still around.
> > * A buggy producer client that did not follow the protocol.
> >
> > For the third case, we should try to let user know asap, and as clear
> > as possible, and hence it’s best to just stop the client app; for the
> > first case, it does not matter since the producer is already gone;
> > only for the second, probably the least likely case, we need to
> > ponder, and there my rationale again is that at that time, we may have
> > already violated the EOS guarantees since there are some partial txn
> > records that should be rejected while the txn itself has been
> > committed. In this case I think it’s better to let clients/users know
> > as soon as possible as well.
> >
> > Regarding the past scenarios where a fatal error killing the whole
> > apps, I believe part of that reason is that we were doing an
> > sub-optimal job on clients side handling various error cases and
> > that’s what KIP-691 is trying to resolve, and hence personally I would
> > suggest we do not weight in too much on that if we can trust that
> > KIP-691 will be successfully eliminate those not-necessary-hard-fail
> > scenarios.
> >
> >
> > Guozhang
> >
> > On Wed, Jan 25, 2023 at 5:51 PM Matthias J. Sax <mj...@apache.org>
> wrote:
> > >
> > > So the timestamp would be set when the write happens and thus no
> > > out-of-order data (base in time) can be introduced with "append_time"
> > > config even if a request sits in purgatory first while we check the TX
> > > status.
> > >
> > > That does make sense. Thanks for confirming, that there is no
> > > out-of-order issue for this case.
> > >
> > >
> > > -Matthias
> > >
> > > On 1/25/23 5:04 PM, Justine Olshan wrote:
> > > > Hey Matthias,
> > > > Let me put it this way, if a producer is checking if a transaction is
> > > > ongoing, then no writes to the partition from the producer will go
> > through
> > > > until the transaction is confirmed ongoing.
> > > >  From then, I think I can apply the writes in the order they came in.
> > Does
> > > > that make sense?
> > > >
> > > > Let me know if I'm missing something.
> > > > Justine
> > > >
> > > > On Wed, Jan 25, 2023 at 4:57 PM Matthias J. Sax <mj...@apache.org>
> > wrote:
> > > >
> > > >>> would it build an offset map with just the latest timestamp for a
> > key?
> > > >>
> > > >> Cannot remember the details without reading the KIP, but yes,
> > something
> > > >> like this (I believe it actually needs to track both, offset and
> > > >> timestamp per key).
> > > >>
> > > >>> I wonder if ordering assumptions are baked in there, why not use
> > > >> offset-based compaction.
> > > >>
> > > >> The use case is a compacted topic that does contain out-of-order
> data.
> > > >> If you set key k1=v1 @ 5 offset 100 and later key1 = v0 @ 3 at
> offset
> > > >> 200 we want to cleanup v0 with higher offset because it's
> out-of-order
> > > >> based on time, but keep v1 what is the actual latest version of k1.
> > > >>
> > > >>
> > > >>> I was also not aware of this "guarantee" with regards to broker
> side
> > > >> time.
> > > >>
> > > >> As already said: I am not sure if it's a public contract, but based
> on
> > > >> my experience, people might reply on it as "implicit contract". --
> > Maybe
> > > >> somebody else knows if it's public or not, and if it would be ok to
> > > >> "break" it.
> > > >>
> > > >>> Let me know if you have any concerns here.
> > > >>
> > > >> My understanding is: While we cannot make an offset-order guarantee
> > for
> > > >> interleaved writes of different producer, if the topic is configures
> > > >> with "append_time", we "guarantee" (cf. my comment above") timestamp
> > > >> order... If that's the case, it would be an issue if we break this
> > > >> "guarantee".
> > > >>
> > > >> I am not sure when the broker sets the timestamp for "append_time"
> > > >> config? If we do it before putting the request into purgatory, we
> > have a
> > > >> problem. However, if we set the timestamp when we actually process
> the
> > > >> request and do the actual append, it seems there is no issue, as the
> > > >> request that was waiting in purgatory get the "newest" timestamp and
> > > >> thus cannot introduce out-of-order data.
> > > >>
> > > >>
> > > >> -Matthias
> > > >>
> > > >>
> > > >> On 1/24/23 10:44 AM, Justine Olshan wrote:
> > > >>> Hey Matthias,
> > > >>>
> > > >>> I have actually never heard of KIP-280 so thanks for bringing it
> up.
> > That
> > > >>> seems interesting. I wonder how it would work though -- would it
> > build an
> > > >>> offset map with just the latest timestamp for a key? I wonder if
> > ordering
> > > >>> assumptions are baked in there, why not use offset-based
> compaction.
> > > >>>
> > > >>> I was also not aware of this "guarantee" with regards to broker
> side
> > > >> time.
> > > >>> I think that we can do in order handling for a given producer, but
> > not
> > > >>> across all producers. However, we can't guarantee that anyway.
> > > >>>
> > > >>> Let me know if you have any concerns here.
> > > >>>
> > > >>> Thanks,
> > > >>> Justine
> > > >>>
> > > >>> On Mon, Jan 23, 2023 at 6:33 PM Matthias J. Sax <mj...@apache.org>
> > > >> wrote:
> > > >>>
> > > >>>> Just a side note about Guozhang comments about timestamps.
> > > >>>>
> > > >>>> If the producer sets the timestamp, putting the record into
> > purgatory
> > > >>>> seems not to be an issue (as already said: for this case we don't
> > > >>>> guarantee timestamp order between writes of different producers
> > anyway).
> > > >>>> However, if the broker sets the timestamp, the expectation is that
> > there
> > > >>>> is no out-of-order data in the partition ever; if we would
> introduce
> > > >>>> out-of-order data for this case (for interleaved writes of
> different
> > > >>>> producers), it seems we would violate the current contract? (To be
> > fair:
> > > >>>> I don't know if that's an official contract, but I assume people
> > rely on
> > > >>>> this behavior -- and it "advertised" in many public talks...)
> > > >>>>
> > > >>>> About compaction: there is actually KIP-280 that adds timestamp
> > based
> > > >>>> compaction what is a very useful feature for Kafka Streams with
> > regard
> > > >>>> to out-of-order data handling. So the impact if we introduce
> > > >>>> out-of-order data could be larger scoped.
> > > >>>>
> > > >>>>
> > > >>>> -Matthias
> > > >>>>
> > > >>>>
> > > >>>> On 1/20/23 4:48 PM, Justine Olshan wrote:
> > > >>>>> Hey Artem,
> > > >>>>>
> > > >>>>> I see there is a check for transactional producers. I'm wondering
> > if we
> > > >>>>> don't handle the epoch overflow case. I'm also not sure it will
> be
> > a
> > > >> huge
> > > >>>>> issue to extend to transactional producers, but maybe I'm missing
> > > >>>> something.
> > > >>>>>
> > > >>>>> As for the recovery path -- I think Guozhang's point was if we
> > have a
> > > >> bad
> > > >>>>> client that repeatedly tries to produce without adding to the
> > > >> transaction
> > > >>>>> we would do the following:
> > > >>>>> a) if not fatal, we just fail the produce request over and over
> > > >>>>> b) if fatal, we fence the producer
> > > >>>>>
> > > >>>>> Here with B, the issue with the client would be made clear more
> > > >> quickly.
> > > >>>> I
> > > >>>>> suppose there are some intermediate cases where the issue only
> > occurs
> > > >>>>> sometimes, but I wonder if we should consider how to recover with
> > > >> clients
> > > >>>>> who don't behave as expected anyway.
> > > >>>>>
> > > >>>>> I think there is a place for the abortable error that we are
> > adding --
> > > >>>> just
> > > >>>>> abort and try again. But I think there are also some cases where
> > trying
> > > >>>> to
> > > >>>>> recover overcomplicates some logic. Especially if we are
> > considering
> > > >>>> older
> > > >>>>> clients -- there I'm not sure if there's a ton we can do besides
> > fail
> > > >> the
> > > >>>>> batch or fence the producer. With newer clients, we can consider
> > more
> > > >>>>> options for what can just be recovered after aborting. But epochs
> > might
> > > >>>> be
> > > >>>>> a hard one unless we also want to reset producer ID.
> > > >>>>>
> > > >>>>> Thanks,
> > > >>>>> Justine
> > > >>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>> On Fri, Jan 20, 2023 at 3:59 PM Artem Livshits
> > > >>>>> <al...@confluent.io.invalid> wrote:
> > > >>>>>
> > > >>>>>>>     besides the poorly written client case
> > > >>>>>>
> > > >>>>>> A poorly written client could create a lot of grief to people
> who
> > run
> > > >>>> Kafka
> > > >>>>>> brokers :-), so when deciding to make an error fatal I would see
> > if
> > > >>>> there
> > > >>>>>> is a reasonable recovery path rather than how often it could
> > happen.
> > > >>>> If we
> > > >>>>>> have solid implementation of transactions (which I hope we'll do
> > as a
> > > >>>>>> result of this KIP), it would help to recover from a large class
> > of
> > > >>>> errors
> > > >>>>>> by just aborting a transaction, even if the cause of error is a
> > race
> > > >>>>>> condition or etc.
> > > >>>>>>
> > > >>>>>> -Artem
> > > >>>>>>
> > > >>>>>> On Fri, Jan 20, 2023 at 3:26 PM Justine Olshan
> > > >>>>>> <jo...@confluent.io.invalid>
> > > >>>>>> wrote:
> > > >>>>>>
> > > >>>>>>> Artem --
> > > >>>>>>> I guess the discussion path we were going down is when we
> expect
> > to
> > > >> see
> > > >>>>>>> this error. I mentioned that it was hard to come up with cases
> > for
> > > >> when
> > > >>>>>> the
> > > >>>>>>> producer would still be around to receive the error besides the
> > > >> poorly
> > > >>>>>>> written client case.
> > > >>>>>>> If we don't expect to have a producer to receive the response,
> it
> > > >> sort
> > > >>>> of
> > > >>>>>>> makes sense for it to be fatal.
> > > >>>>>>>
> > > >>>>>>> I had some discussion with Jason offline about the epoch being
> > off
> > > >>>> cases
> > > >>>>>>> and I'm not sure we could find a ton (outside of produce
> > requests)
> > > >>>> where
> > > >>>>>> we
> > > >>>>>>> could/should recover. I'd be happy to hear some examples
> though,
> > > >> maybe
> > > >>>>>> I'm
> > > >>>>>>> missing something.
> > > >>>>>>>
> > > >>>>>>> Thanks,
> > > >>>>>>> Justine
> > > >>>>>>>
> > > >>>>>>> On Fri, Jan 20, 2023 at 3:19 PM Artem Livshits
> > > >>>>>>> <al...@confluent.io.invalid> wrote:
> > > >>>>>>>
> > > >>>>>>>> In general, I'd like to avoid fatal errors as much as
> possible,
> > in
> > > >>>> some
> > > >>>>>>>> sense fatal errors just push out recovery logic to the
> > application
> > > >>>>>> which
> > > >>>>>>>> either complicates the application or leads to disruption
> (we've
> > > >> seen
> > > >>>>>>> cases
> > > >>>>>>>> when a transient broker error could lead to work stoppage when
> > > >>>>>>> applications
> > > >>>>>>>> need to be manually restarted).  I think we should strive to
> > define
> > > >>>>>>>> recovery logic for most errors (and/or encapsulate it in the
> > Kafka
> > > >>>>>> client
> > > >>>>>>>> as much as possible).
> > > >>>>>>>>
> > > >>>>>>>> One benefit of transactions is that they simplify recovery
> from
> > > >>>> errors,
> > > >>>>>>>> pretty much any error (that's not handled transparently by
> > retries
> > > >> in
> > > >>>>>>> Kafka
> > > >>>>>>>> client) can be handled by the application via aborting the
> > > >> transaction
> > > >>>>>>> and
> > > >>>>>>>> repeating the transactional logic again.  One tricky error is
> an
> > > >> error
> > > >>>>>>>> during commit, because we don't know the outcome.  For commit
> > > >> errors,
> > > >>>>>> the
> > > >>>>>>>> recommendation should be to retry the commit until it returns
> > the
> > > >>>>>>> specific
> > > >>>>>>>> result (committed or aborted).
> > > >>>>>>>>
> > > >>>>>>>> -Artem
> > > >>>>>>>>
> > > >>>>>>>> On Fri, Jan 20, 2023 at 2:52 PM Justine Olshan
> > > >>>>>>>> <jo...@confluent.io.invalid>
> > > >>>>>>>> wrote:
> > > >>>>>>>>
> > > >>>>>>>>> That's a fair point about other clients.
> > > >>>>>>>>>
> > > >>>>>>>>> I think the abortable error case is interesting because I'm
> > curious
> > > >>>>>> how
> > > >>>>>>>>> other clients would handle this. I assume they would need to
> > > >>>>>> implement
> > > >>>>>>>>> handling for the error code unless they did something like
> "any
> > > >>>>>> unknown
> > > >>>>>>>>> error codes/any codes that aren't x,y,z are retriable." I
> would
> > > >> hope
> > > >>>>>>> that
> > > >>>>>>>>> unknown error codes were fatal, and if the code was
> > implemented it
> > > >>>>>>> would
> > > >>>>>>>>> abort the transaction. But I will think on this too.
> > > >>>>>>>>>
> > > >>>>>>>>> As for InvalidRecord -- you mentioned it was not fatal, but
> I'm
> > > >>>>>> taking
> > > >>>>>>> a
> > > >>>>>>>>> look through the code. We would see this on handling the
> > produce
> > > >>>>>>>> response.
> > > >>>>>>>>> If I recall correctly, we check if errors are retriable. I
> > think
> > > >> this
> > > >>>>>>>> error
> > > >>>>>>>>> would not be retriable. But I guess the concern here is that
> > it is
> > > >>>>>> not
> > > >>>>>>>>> enough for just that batch to fail. I guess I hadn't
> considered
> > > >> fully
> > > >>>>>>>>> fencing the old producer but there are valid arguments here
> > why we
> > > >>>>>>> would
> > > >>>>>>>>> want to.
> > > >>>>>>>>>
> > > >>>>>>>>> Thanks,
> > > >>>>>>>>> Justine
> > > >>>>>>>>>
> > > >>>>>>>>> On Fri, Jan 20, 2023 at 2:35 PM Guozhang Wang <
> > > >>>>>>>> guozhang.wang.us@gmail.com>
> > > >>>>>>>>> wrote:
> > > >>>>>>>>>
> > > >>>>>>>>>> Thanks Justine for the replies! I agree with most of your
> > > >> thoughts.
> > > >>>>>>>>>>
> > > >>>>>>>>>> Just for 3/7), though I agree for our own AK producer, since
> > we do
> > > >>>>>>>>>> "nextRequest(boolean hasIncompleteBatches)", we guarantee
> the
> > > >>>>>> end-txn
> > > >>>>>>>>>> would not be sent until we've effectively flushed, but I was
> > > >>>>>>> referring
> > > >>>>>>>>>> to any future bugs or other buggy clients that the same
> > client may
> > > >>>>>>> get
> > > >>>>>>>>>> into this situation, in which case we should give the
> client a
> > > >>>>>> clear
> > > >>>>>>>>>> msg that "you did something wrong, and hence now you should
> > > >> fatally
> > > >>>>>>>>>> close yourself". What I'm concerned about is that, by seeing
> > an
> > > >>>>>>>>>> "abortable error" or in some rare cases an "invalid record",
> > the
> > > >>>>>>>>>> client could not realize "something that's really bad
> > happened".
> > > >> So
> > > >>>>>>>>>> it's not about adding a new error, it's mainly about those
> > real
> > > >>>>>> buggy
> > > >>>>>>>>>> situations causing such "should never happen" cases, the
> > errors
> > > >>>>>>> return
> > > >>>>>>>>>> would not be informative enough.
> > > >>>>>>>>>>
> > > >>>>>>>>>> Thinking in other ways, if we believe that for most cases
> such
> > > >>>>>> error
> > > >>>>>>>>>> codes would not reach the original clients since they would
> be
> > > >>>>>>>>>> disconnected or even gone by that time, and only in some
> rare
> > > >> cases
> > > >>>>>>>>>> they would still be seen by the sending clients, then why
> not
> > make
> > > >>>>>>>>>> them more fatal and more specific than generic.
> > > >>>>>>>>>>
> > > >>>>>>>>>> Guozhang
> > > >>>>>>>>>>
> > > >>>>>>>>>> On Fri, Jan 20, 2023 at 1:59 PM Justine Olshan
> > > >>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Hey Guozhang. Thanks for taking a look and for the detailed
> > > >>>>>>> comments!
> > > >>>>>>>>>> I'll
> > > >>>>>>>>>>> do my best to address below.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> 1. I see what you are saying here, but I think I need to
> look
> > > >>>>>>> through
> > > >>>>>>>>> the
> > > >>>>>>>>>>> sequence of events you mention. Typically we've seen this
> > issue
> > > >>>>>> in
> > > >>>>>>> a
> > > >>>>>>>>> few
> > > >>>>>>>>>>> cases.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>     One is when we have a producer disconnect when trying
> to
> > > >>>>>> produce.
> > > >>>>>>>>>>> Typically in these cases, we abort the transaction. We've
> > seen
> > > >>>>>> that
> > > >>>>>>>>> after
> > > >>>>>>>>>>> the markers are written, the disconnection can sometimes
> > cause
> > > >>>>>> the
> > > >>>>>>>>>> request
> > > >>>>>>>>>>> to get flushed to the broker. In this case, we don't need
> > client
> > > >>>>>>>>> handling
> > > >>>>>>>>>>> because the producer we are responding to is gone. We just
> > needed
> > > >>>>>>> to
> > > >>>>>>>>> make
> > > >>>>>>>>>>> sure we didn't write to the log on the broker side. I'm
> > trying to
> > > >>>>>>>> think
> > > >>>>>>>>>> of
> > > >>>>>>>>>>> a case where we do have the client to return to. I'd think
> > the
> > > >>>>>> same
> > > >>>>>>>>>> client
> > > >>>>>>>>>>> couldn't progress to committing the transaction unless the
> > > >>>>>> produce
> > > >>>>>>>>>> request
> > > >>>>>>>>>>> returned right? Of course, there is the incorrectly written
> > > >>>>>> clients
> > > >>>>>>>>> case.
> > > >>>>>>>>>>> I'll think on this a bit more and let you know if I come up
> > with
> > > >>>>>>>>> another
> > > >>>>>>>>>>> scenario when we would return to an active client when the
> > > >>>>>>>> transaction
> > > >>>>>>>>> is
> > > >>>>>>>>>>> no longer ongoing.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> I was not aware that we checked the result of a send after
> we
> > > >>>>>>> commit
> > > >>>>>>>>>>> though. I'll need to look into that a bit more.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> 2. There were some questions about this in the discussion.
> > The
> > > >>>>>> plan
> > > >>>>>>>> is
> > > >>>>>>>>> to
> > > >>>>>>>>>>> handle overflow with the mechanism we currently have in the
> > > >>>>>>> producer.
> > > >>>>>>>>> If
> > > >>>>>>>>>> we
> > > >>>>>>>>>>> try to bump and the epoch will overflow, we actually
> > allocate a
> > > >>>>>> new
> > > >>>>>>>>>>> producer ID. I need to confirm the fencing logic on the
> last
> > > >>>>>> epoch
> > > >>>>>>>> (ie,
> > > >>>>>>>>>> we
> > > >>>>>>>>>>> probably shouldn't allow any records to be produced with
> the
> > > >>>>>> final
> > > >>>>>>>>> epoch
> > > >>>>>>>>>>> since we can never properly fence that one).
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> 3. I can agree with you that the current error handling is
> > > >>>>>> messy. I
> > > >>>>>>>>>> recall
> > > >>>>>>>>>>> taking a look at your KIP a while back, but I think I
> mostly
> > saw
> > > >>>>>>> the
> > > >>>>>>>>>>> section about how the errors were wrapped. Maybe I need to
> > take
> > > >>>>>>>> another
> > > >>>>>>>>>>> look. As for abortable error, the idea was that the
> handling
> > > >>>>>> would
> > > >>>>>>> be
> > > >>>>>>>>>>> simple -- if this error is seen, the transaction should be
> > > >>>>>> aborted
> > > >>>>>>> --
> > > >>>>>>>>> no
> > > >>>>>>>>>>> other logic about previous state or requests necessary. Is
> > your
> > > >>>>>>>> concern
> > > >>>>>>>>>>> simply about adding new errors? We were hoping to have an
> > error
> > > >>>>>>> that
> > > >>>>>>>>>> would
> > > >>>>>>>>>>> have one meaning and many of the current errors have a
> > history of
> > > >>>>>>>>> meaning
> > > >>>>>>>>>>> different things on different client versions. That was the
> > main
> > > >>>>>>>>>> motivation
> > > >>>>>>>>>>> for adding a new error.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> 4. This is a good point about record timestamp reordering.
> > > >>>>>>> Timestamps
> > > >>>>>>>>>> don't
> > > >>>>>>>>>>> affect compaction, but they do affect retention deletion.
> For
> > > >>>>>> that,
> > > >>>>>>>>> kafka
> > > >>>>>>>>>>> considers the largest timestamp in the segment, so I think
> a
> > > >>>>>> small
> > > >>>>>>>>> amount
> > > >>>>>>>>>>> of reordering (hopefully on the order of milliseconds or
> even
> > > >>>>>>>> seconds)
> > > >>>>>>>>>> will
> > > >>>>>>>>>>> be ok. We take timestamps from clients so there is already
> a
> > > >>>>>>>>> possibility
> > > >>>>>>>>>>> for some drift and non-monotonically increasing timestamps.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> 5. Thanks for catching. The error is there, but it's
> actually
> > > >>>>>> that
> > > >>>>>>>>> those
> > > >>>>>>>>>>> fields should be 4+! Due to how the message generator
> works,
> > I
> > > >>>>>>>> actually
> > > >>>>>>>>>>> have to redefine those fields inside the
> > > >>>>>>>>> `"AddPartitionsToTxnTransaction`
> > > >>>>>>>>>>> block for it to build correctly. I'll fix it to be correct.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> 6. Correct -- we will only add the request to purgatory if
> > the
> > > >>>>>>> cache
> > > >>>>>>>>> has
> > > >>>>>>>>>> no
> > > >>>>>>>>>>> ongoing transaction. I can change the wording to make that
> > > >>>>>> clearer
> > > >>>>>>>> that
> > > >>>>>>>>>> we
> > > >>>>>>>>>>> only place the request in purgatory if we need to contact
> the
> > > >>>>>>>>> transaction
> > > >>>>>>>>>>> coordinator.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> 7. We did take a look at some of the errors and it was hard
> > to
> > > >>>>>> come
> > > >>>>>>>> up
> > > >>>>>>>>>> with
> > > >>>>>>>>>>> a good one. I agree that InvalidTxnStateException is ideal
> > except
> > > >>>>>>> for
> > > >>>>>>>>> the
> > > >>>>>>>>>>> fact that it hasn't been returned on Produce requests
> > before. The
> > > >>>>>>>> error
> > > >>>>>>>>>>> handling for clients is a bit vague (which is why I opened
> > > >>>>>>>> KAFKA-14439
> > > >>>>>>>>>>> <https://issues.apache.org/jira/browse/KAFKA-14439>), but
> > the
> > > >>>>>>>> decision
> > > >>>>>>>>>> we
> > > >>>>>>>>>>> made here was to only return errors that have been
> previously
> > > >>>>>>>> returned
> > > >>>>>>>>> to
> > > >>>>>>>>>>> producers. As for not being fatal, I think part of the
> > theory was
> > > >>>>>>>> that
> > > >>>>>>>>> in
> > > >>>>>>>>>>> many cases, the producer would be disconnected. (See point
> > 1) and
> > > >>>>>>>> this
> > > >>>>>>>>>>> would just be an error to return from the server. I did
> plan
> > to
> > > >>>>>>> think
> > > >>>>>>>>>> about
> > > >>>>>>>>>>> other cases, so let me know if you think of any as well!
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Lots to say! Let me know if you have further thoughts!
> > > >>>>>>>>>>> Justine
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang <
> > > >>>>>>>>>> guozhang.wang.us@gmail.com>
> > > >>>>>>>>>>> wrote:
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>> Hello Justine,
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> Thanks for the great write-up! I made a quick pass through
> > it
> > > >>>>>> and
> > > >>>>>>>>> here
> > > >>>>>>>>>>>> are some thoughts (I have not been able to read through
> this
> > > >>>>>>> thread
> > > >>>>>>>>> so
> > > >>>>>>>>>>>> pardon me if they have overlapped or subsumed by previous
> > > >>>>>>>> comments):
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> First are some meta ones:
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> 1. I think we need to also improve the client's experience
> > once
> > > >>>>>>> we
> > > >>>>>>>>>>>> have this defence in place. More concretely, say a user's
> > > >>>>>>> producer
> > > >>>>>>>>>>>> code is like following:
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> future = producer.send();
> > > >>>>>>>>>>>> // producer.flush();
> > > >>>>>>>>>>>> producer.commitTransaction();
> > > >>>>>>>>>>>> future.get();
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> Which resulted in the order of a) produce-request sent by
> > > >>>>>>> producer,
> > > >>>>>>>>> b)
> > > >>>>>>>>>>>> end-txn-request sent by producer, c) end-txn-response sent
> > > >>>>>> back,
> > > >>>>>>> d)
> > > >>>>>>>>>>>> txn-marker-request sent from coordinator to partition
> > leader,
> > > >>>>>> e)
> > > >>>>>>>>>>>> produce-request finally received by the partition leader,
> > > >>>>>> before
> > > >>>>>>>> this
> > > >>>>>>>>>>>> KIP e) step would be accepted causing a dangling txn; now
> it
> > > >>>>>>> would
> > > >>>>>>>> be
> > > >>>>>>>>>>>> rejected in step e) which is good. But from the client's
> > point
> > > >>>>>> of
> > > >>>>>>>>> view
> > > >>>>>>>>>>>> now it becomes confusing since the `commitTransaction()`
> > > >>>>>> returns
> > > >>>>>>>>>>>> successfully, but the "future" throws an invalid-epoch
> > error,
> > > >>>>>> and
> > > >>>>>>>>> they
> > > >>>>>>>>>>>> are not sure if the transaction did succeed or not. In
> > fact, it
> > > >>>>>>>>>>>> "partially succeeded" with some msgs being rejected but
> > others
> > > >>>>>>>>>>>> committed successfully.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> Of course the easy way to avoid this is, always call
> > > >>>>>>>>>>>> "producer.flush()" before commitTxn and that's what we do
> > > >>>>>>>> ourselves,
> > > >>>>>>>>>>>> and what we recommend users do. But I suspect not everyone
> > does
> > > >>>>>>> it.
> > > >>>>>>>>> In
> > > >>>>>>>>>>>> fact I just checked the javadoc in KafkaProducer and our
> > code
> > > >>>>>>>> snippet
> > > >>>>>>>>>>>> does not include a `flush()` call. So I'm thinking maybe
> we
> > can
> > > >>>>>>> in
> > > >>>>>>>>>>>> side the `commitTxn` code to enforce flushing before
> sending
> > > >>>>>> the
> > > >>>>>>>>>>>> end-txn request.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> 2. I'd like to clarify a bit details on "just add
> > partitions to
> > > >>>>>>> the
> > > >>>>>>>>>>>> transaction on the first produce request during a
> > transaction".
> > > >>>>>>> My
> > > >>>>>>>>>>>> understanding is that the partition leader's cache has the
> > > >>>>>>> producer
> > > >>>>>>>>> id
> > > >>>>>>>>>>>> / sequence / epoch for the latest txn, either on-going or
> is
> > > >>>>>>>>> completed
> > > >>>>>>>>>>>> (upon receiving the marker request from coordinator).
> When a
> > > >>>>>>>> produce
> > > >>>>>>>>>>>> request is received, if
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> * producer's epoch < cached epoch, or producer's epoch ==
> > > >>>>>> cached
> > > >>>>>>>>> epoch
> > > >>>>>>>>>>>> but the latest txn is completed, leader directly reject
> with
> > > >>>>>>>>>>>> invalid-epoch.
> > > >>>>>>>>>>>> * producer's epoch > cached epoch, park the the request
> and
> > > >>>>>> send
> > > >>>>>>>>>>>> add-partitions request to coordinator.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> In order to do it, does the coordinator need to bump the
> > > >>>>>> sequence
> > > >>>>>>>> and
> > > >>>>>>>>>>>> reset epoch to 0 when the next epoch is going to overflow?
> > If
> > > >>>>>> no
> > > >>>>>>>> need
> > > >>>>>>>>>>>> to do so, then how we handle the (admittedly rare, but
> still
> > > >>>>>> may
> > > >>>>>>>>>>>> happen) epoch overflow situation?
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> 3. I'm a bit concerned about adding a generic
> > "ABORTABLE_ERROR"
> > > >>>>>>>> given
> > > >>>>>>>>>>>> we already have a pretty messy error classification and
> > error
> > > >>>>>>>>> handling
> > > >>>>>>>>>>>> on the producer clients side --- I have a summary about
> the
> > > >>>>>>> issues
> > > >>>>>>>>> and
> > > >>>>>>>>>>>> a proposal to address this in
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>
> > > >>>>>>
> > > >>>>
> > > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
> > > >>>>>>>>>>>> -- I understand we do not want to use
> "UNKNOWN_PRODUCER_ID"
> > > >>>>>>> anymore
> > > >>>>>>>>>>>> and in fact we intend to deprecate it in KIP-360 and
> > eventually
> > > >>>>>>>>> remove
> > > >>>>>>>>>>>> it; but I'm wondering can we still use specific error
> codes.
> > > >>>>>> E.g.
> > > >>>>>>>>> what
> > > >>>>>>>>>>>> about "InvalidProducerEpochException" since for new
> clients,
> > > >>>>>> the
> > > >>>>>>>>>>>> actual reason this would actually be rejected is indeed
> > because
> > > >>>>>>> the
> > > >>>>>>>>>>>> epoch on the coordinator caused the add-partitions-request
> > from
> > > >>>>>>> the
> > > >>>>>>>>>>>> brokers to be rejected anyways?
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> 4. It seems we put the producer request into purgatory
> > before
> > > >>>>>> we
> > > >>>>>>>> ever
> > > >>>>>>>>>>>> append the records, while other producer's records may
> > still be
> > > >>>>>>>>>>>> appended during the time; and that potentially may result
> in
> > > >>>>>> some
> > > >>>>>>>>>>>> re-ordering compared with reception order. I'm not super
> > > >>>>>>> concerned
> > > >>>>>>>>>>>> about it since Kafka does not guarantee reception ordering
> > > >>>>>> across
> > > >>>>>>>>>>>> producers anyways, but it may make the timestamps of
> records
> > > >>>>>>>> inside a
> > > >>>>>>>>>>>> partition to be more out-of-ordered. Are we aware of any
> > > >>>>>>> scenarios
> > > >>>>>>>>>>>> such as future enhancements on log compactions that may be
> > > >>>>>>> affected
> > > >>>>>>>>> by
> > > >>>>>>>>>>>> this effect?
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> Below are just minor comments:
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> 5. In "AddPartitionsToTxnTransaction" field of
> > > >>>>>>>>>>>> "AddPartitionsToTxnRequest" RPC, the versions of those
> inner
> > > >>>>>>> fields
> > > >>>>>>>>>>>> are "0-3" while I thought they should be "0+" still?
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> 6. Regarding "we can place the request in a purgatory of
> > sorts
> > > >>>>>>> and
> > > >>>>>>>>>>>> check if there is any state for the transaction on the
> > > >>>>>> broker": i
> > > >>>>>>>>>>>> think at this time when we just do the checks against the
> > > >>>>>> cached
> > > >>>>>>>>>>>> state, we do not need to put the request to purgatory yet?
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> 7. This is related to 3) above. I feel using
> > > >>>>>>>> "InvalidRecordException"
> > > >>>>>>>>>>>> for older clients may also be a bit confusing, and also it
> > is
> > > >>>>>> not
> > > >>>>>>>>>>>> fatal -- for old clients, it better to be fatal since this
> > > >>>>>>>> indicates
> > > >>>>>>>>>>>> the clients is doing something wrong and hence it should
> be
> > > >>>>>>> closed.
> > > >>>>>>>>>>>> And in general I'd prefer to use slightly more specific
> > meaning
> > > >>>>>>>> error
> > > >>>>>>>>>>>> codes for clients. That being said, I also feel
> > > >>>>>>>>>>>> "InvalidProducerEpochException" is not suitable for old
> > > >>>>>> versioned
> > > >>>>>>>>>>>> clients, and we'd have to pick one that old clients
> > recognize.
> > > >>>>>>> I'd
> > > >>>>>>>>>>>> prefer "InvalidTxnStateException" but that one is supposed
> > to
> > > >>>>>> be
> > > >>>>>>>>>>>> returned from txn coordinators only today. I'd suggest we
> > do a
> > > >>>>>>>> quick
> > > >>>>>>>>>>>> check in the current client's code path and see if that
> one
> > > >>>>>> would
> > > >>>>>>>> be
> > > >>>>>>>>>>>> handled if it's from a produce-response, and if yes, use
> > this
> > > >>>>>>> one;
> > > >>>>>>>>>>>> otherwise, use "ProducerFencedException" which is much
> less
> > > >>>>>>>>> meaningful
> > > >>>>>>>>>>>> but it's still a fatal error.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>> Guozhang
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
> > > >>>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Yeah -- looks like we already have code to handle bumping
> > the
> > > >>>>>>>> epoch
> > > >>>>>>>>>> and
> > > >>>>>>>>>>>>> when the epoch is Short.MAX_VALUE, we get a new producer
> > ID.
> > > >>>>>>>> Since
> > > >>>>>>>>>> this
> > > >>>>>>>>>>>> is
> > > >>>>>>>>>>>>> already the behavior, do we want to change it further?
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Justine
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <
> > > >>>>>>>>> jolshan@confluent.io
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Hey all, just wanted to quickly update and say I've
> > > >>>>>> modified
> > > >>>>>>>> the
> > > >>>>>>>>>> KIP to
> > > >>>>>>>>>>>>>> explicitly mention that AddOffsetCommitsToTxnRequest
> will
> > > >>>>>> be
> > > >>>>>>>>>> replaced
> > > >>>>>>>>>>>> by
> > > >>>>>>>>>>>>>> a coordinator-side (inter-broker) AddPartitionsToTxn
> > > >>>>>> implicit
> > > >>>>>>>>>> request.
> > > >>>>>>>>>>>> This
> > > >>>>>>>>>>>>>> mirrors the user partitions and will implicitly add
> offset
> > > >>>>>>>>>> partitions
> > > >>>>>>>>>>>> to
> > > >>>>>>>>>>>>>> transactions when we commit offsets on them. We will
> > > >>>>>>> deprecate
> > > >>>>>>>>>>>> AddOffsetCommitsToTxnRequest
> > > >>>>>>>>>>>>>> for new clients.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Also to address Artem's comments --
> > > >>>>>>>>>>>>>> I'm a bit unsure if the changes here will change the
> > > >>>>>> previous
> > > >>>>>>>>>> behavior
> > > >>>>>>>>>>>> for
> > > >>>>>>>>>>>>>> fencing producers. In the case you mention in the first
> > > >>>>>>>>> paragraph,
> > > >>>>>>>>>> are
> > > >>>>>>>>>>>> you
> > > >>>>>>>>>>>>>> saying we bump the epoch before we try to abort the
> > > >>>>>>>> transaction?
> > > >>>>>>>>> I
> > > >>>>>>>>>>>> think I
> > > >>>>>>>>>>>>>> need to understand the scenarios you mention a bit
> better.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> As for the second part -- I think it makes sense to have
> > > >>>>>> some
> > > >>>>>>>>> sort
> > > >>>>>>>>>> of
> > > >>>>>>>>>>>>>> "sentinel" epoch to signal epoch is about to overflow (I
> > > >>>>>>> think
> > > >>>>>>>> we
> > > >>>>>>>>>> sort
> > > >>>>>>>>>>>> of
> > > >>>>>>>>>>>>>> have this value in place in some ways) so we can codify
> it
> > > >>>>>> in
> > > >>>>>>>> the
> > > >>>>>>>>>> KIP.
> > > >>>>>>>>>>>> I'll
> > > >>>>>>>>>>>>>> look into that and try to update soon.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>>>> Justine.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
> > > >>>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> It's good to know that KIP-588 addressed some of the
> > > >>>>>> issues.
> > > >>>>>>>>>> Looking
> > > >>>>>>>>>>>> at
> > > >>>>>>>>>>>>>>> the code, it still looks like there are some cases that
> > > >>>>>>> would
> > > >>>>>>>>>> result
> > > >>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>> fatal error, e.g. PRODUCER_FENCED is issued by the
> > > >>>>>>> transaction
> > > >>>>>>>>>>>> coordinator
> > > >>>>>>>>>>>>>>> if epoch doesn't match, and the client treats it as a
> > > >>>>>> fatal
> > > >>>>>>>>> error
> > > >>>>>>>>>>>> (code in
> > > >>>>>>>>>>>>>>> TransactionManager request handling).  If we consider,
> > for
> > > >>>>>>>>>> example,
> > > >>>>>>>>>>>>>>> committing a transaction that returns a timeout, but
> > > >>>>>>> actually
> > > >>>>>>>>>>>> succeeds,
> > > >>>>>>>>>>>>>>> trying to abort it or re-commit may result in
> > > >>>>>>> PRODUCER_FENCED
> > > >>>>>>>>>> error
> > > >>>>>>>>>>>>>>> (because of epoch bump).
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> For failed commits, specifically, we need to know the
> > > >>>>>> actual
> > > >>>>>>>>>> outcome,
> > > >>>>>>>>>>>>>>> because if we return an error the application may think
> > > >>>>>> that
> > > >>>>>>>> the
> > > >>>>>>>>>>>>>>> transaction is aborted and redo the work, leading to
> > > >>>>>>>> duplicates.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Re: overflowing epoch.  We could either do it on the TC
> > > >>>>>> and
> > > >>>>>>>>> return
> > > >>>>>>>>>>>> both
> > > >>>>>>>>>>>>>>> producer id and epoch (e.g. change the protocol), or
> > > >>>>>> signal
> > > >>>>>>>> the
> > > >>>>>>>>>> client
> > > >>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>> it needs to get a new producer id.  Checking for max
> > epoch
> > > >>>>>>>> could
> > > >>>>>>>>>> be a
> > > >>>>>>>>>>>>>>> reasonable signal, the value to check should probably
> be
> > > >>>>>>>> present
> > > >>>>>>>>>> in
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>> KIP
> > > >>>>>>>>>>>>>>> as this is effectively a part of the contract.  Also,
> the
> > > >>>>>> TC
> > > >>>>>>>>>> should
> > > >>>>>>>>>>>>>>> probably return an error if the client didn't change
> > > >>>>>>> producer
> > > >>>>>>>> id
> > > >>>>>>>>>> after
> > > >>>>>>>>>>>>>>> hitting max epoch.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> -Artem
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
> > > >>>>>>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Thanks for the discussion Artem.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> With respect to the handling of fenced producers, we
> > > >>>>>> have
> > > >>>>>>>> some
> > > >>>>>>>>>>>> behavior
> > > >>>>>>>>>>>>>>>> already in place. As of KIP-588:
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>
> > > >>>>>>
> > > >>>>
> > > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> > > >>>>>>>>>>>>>>>> ,
> > > >>>>>>>>>>>>>>>> we handle timeouts more gracefully. The producer can
> > > >>>>>>>> recover.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Produce requests can also recover from epoch fencing
> by
> > > >>>>>>>>>> aborting the
> > > >>>>>>>>>>>>>>>> transaction and starting over.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> What other cases were you considering that would cause
> > > >>>>>> us
> > > >>>>>>> to
> > > >>>>>>>>>> have a
> > > >>>>>>>>>>>>>>> fenced
> > > >>>>>>>>>>>>>>>> epoch but we'd want to recover?
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> The first point about handling epoch overflows is
> fair.
> > > >>>>>> I
> > > >>>>>>>>> think
> > > >>>>>>>>>>>> there is
> > > >>>>>>>>>>>>>>>> some logic we'd need to consider. (ie, if we are one
> > > >>>>>> away
> > > >>>>>>>> from
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>> max
> > > >>>>>>>>>>>>>>>> epoch, we need to reset the producer ID.) I'm still
> > > >>>>>>>> wondering
> > > >>>>>>>>> if
> > > >>>>>>>>>>>> there
> > > >>>>>>>>>>>>>>> is a
> > > >>>>>>>>>>>>>>>> way to direct this from the response, or if everything
> > > >>>>>>>> should
> > > >>>>>>>>> be
> > > >>>>>>>>>>>> done on
> > > >>>>>>>>>>>>>>>> the client side. Let me know if you have any thoughts
> > > >>>>>>> here.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>>>>>> Justine
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
> > > >>>>>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> There are some workflows in the client that are
> > > >>>>>> implied
> > > >>>>>>> by
> > > >>>>>>>>>>>> protocol
> > > >>>>>>>>>>>>>>>>> changes, e.g.:
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> - for new clients, epoch changes with every
> > > >>>>>> transaction
> > > >>>>>>>> and
> > > >>>>>>>>>> can
> > > >>>>>>>>>>>>>>> overflow,
> > > >>>>>>>>>>>>>>>>> in old clients this condition was handled
> > > >>>>>> transparently,
> > > >>>>>>>>>> because
> > > >>>>>>>>>>>> epoch
> > > >>>>>>>>>>>>>>>> was
> > > >>>>>>>>>>>>>>>>> bumped in InitProducerId and it would return a new
> > > >>>>>>>> producer
> > > >>>>>>>>>> id if
> > > >>>>>>>>>>>>>>> epoch
> > > >>>>>>>>>>>>>>>>> overflows, the new clients would need to implement
> > > >>>>>> some
> > > >>>>>>>>>> workflow
> > > >>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>> refresh
> > > >>>>>>>>>>>>>>>>> producer id
> > > >>>>>>>>>>>>>>>>> - how to handle fenced producers, for new clients
> > > >>>>>> epoch
> > > >>>>>>>>>> changes
> > > >>>>>>>>>>>> with
> > > >>>>>>>>>>>>>>>> every
> > > >>>>>>>>>>>>>>>>> transaction, so in presence of failures during
> > > >>>>>> commits /
> > > >>>>>>>>>> aborts,
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>> producer could get easily fenced, old clients would
> > > >>>>>>> pretty
> > > >>>>>>>>>> much
> > > >>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>> get
> > > >>>>>>>>>>>>>>>>> fenced when a new incarnation of the producer was
> > > >>>>>>>>> initialized
> > > >>>>>>>>>> with
> > > >>>>>>>>>>>>>>>>> InitProducerId so it's ok to treat as a fatal error,
> > > >>>>>> the
> > > >>>>>>>> new
> > > >>>>>>>>>>>> clients
> > > >>>>>>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>> need to implement some workflow to handle that error,
> > > >>>>>>>>>> otherwise
> > > >>>>>>>>>>>> they
> > > >>>>>>>>>>>>>>>> could
> > > >>>>>>>>>>>>>>>>> get fenced by themselves
> > > >>>>>>>>>>>>>>>>> - in particular (as a subset of the previous issue),
> > > >>>>>>> what
> > > >>>>>>>>>> would
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>> client
> > > >>>>>>>>>>>>>>>>> do if it got a timeout during commit?  commit
> could've
> > > >>>>>>>>>> succeeded
> > > >>>>>>>>>>>> or
> > > >>>>>>>>>>>>>>>> failed
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> Not sure if this has to be defined in the KIP as
> > > >>>>>>>>> implementing
> > > >>>>>>>>>>>> those
> > > >>>>>>>>>>>>>>>>> probably wouldn't require protocol changes, but we
> > > >>>>>> have
> > > >>>>>>>>>> multiple
> > > >>>>>>>>>>>>>>>>> implementations of Kafka clients, so probably would
> be
> > > >>>>>>>> good
> > > >>>>>>>>> to
> > > >>>>>>>>>>>> have
> > > >>>>>>>>>>>>>>> some
> > > >>>>>>>>>>>>>>>>> client implementation guidance.  Could also be done
> > > >>>>>> as a
> > > >>>>>>>>>> separate
> > > >>>>>>>>>>>> doc.
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> -Artem
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
> > > >>>>>>>>>>>>>>>> <jolshan@confluent.io.invalid
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> Hey all, I've updated the KIP to incorporate Jason's
> > > >>>>>>>>>>>> suggestions.
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>
> > > >>>>>>
> > > >>>>
> > > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> 1. Use AddPartitionsToTxn + verify flag to check on
> > > >>>>>>> old
> > > >>>>>>>>>> clients
> > > >>>>>>>>>>>>>>>>>> 2. Updated AddPartitionsToTxn API to support
> > > >>>>>>> transaction
> > > >>>>>>>>>>>> batching
> > > >>>>>>>>>>>>>>>>>> 3. Mention IBP bump
> > > >>>>>>>>>>>>>>>>>> 4. Mention auth change on new AddPartitionsToTxn
> > > >>>>>>>> version.
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> I'm planning on opening a vote soon.
> > > >>>>>>>>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>>>>>>>> Justine
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <
> > > >>>>>>>>>>>> jolshan@confluent.io
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> Thanks Jason. Those changes make sense to me. I
> > > >>>>>> will
> > > >>>>>>>>>> update
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>> KIP.
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
> > > >>>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
> > > >>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> Hey Justine,
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> I was wondering about compatibility here. When
> > > >>>>>> we
> > > >>>>>>>>> send
> > > >>>>>>>>>>>> requests
> > > >>>>>>>>>>>>>>>>>>>> between brokers, we want to ensure that the
> > > >>>>>>> receiving
> > > >>>>>>>>>> broker
> > > >>>>>>>>>>>>>>>>> understands
> > > >>>>>>>>>>>>>>>>>>>> the request (specifically the new fields).
> > > >>>>>>> Typically
> > > >>>>>>>>>> this is
> > > >>>>>>>>>>>> done
> > > >>>>>>>>>>>>>>>> via
> > > >>>>>>>>>>>>>>>>>>>> IBP/metadata version.
> > > >>>>>>>>>>>>>>>>>>>> I'm trying to think if there is a way around it
> > > >>>>>> but
> > > >>>>>>>> I'm
> > > >>>>>>>>>> not
> > > >>>>>>>>>>>> sure
> > > >>>>>>>>>>>>>>>> there
> > > >>>>>>>>>>>>>>>>>> is.
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> Yes. I think we would gate usage of this behind
> > > >>>>>> an
> > > >>>>>>>> IBP
> > > >>>>>>>>>> bump.
> > > >>>>>>>>>>>> Does
> > > >>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>> seem
> > > >>>>>>>>>>>>>>>>>>>> reasonable?
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> As for the improvements -- can you clarify how
> > > >>>>>>> the
> > > >>>>>>>>>> multiple
> > > >>>>>>>>>>>>>>>>>>>> transactional
> > > >>>>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking of a case
> > > >>>>>>>> where
> > > >>>>>>>>> we
> > > >>>>>>>>>>>>>>> wait/batch
> > > >>>>>>>>>>>>>>>>>>>> multiple produce requests together? My
> > > >>>>>>> understanding
> > > >>>>>>>>> for
> > > >>>>>>>>>> now
> > > >>>>>>>>>>>> was
> > > >>>>>>>>>>>>>>> 1
> > > >>>>>>>>>>>>>>>>>>>> transactional ID and one validation per 1 produce
> > > >>>>>>>>>> request.
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> Each call to `AddPartitionsToTxn` is essentially
> > > >>>>>> a
> > > >>>>>>>>> write
> > > >>>>>>>>>> to
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>> log and must block on replication. The more we
> > > >>>>>> can
> > > >>>>>>>> fit
> > > >>>>>>>>>> into a
> > > >>>>>>>>>>>>>>> single
> > > >>>>>>>>>>>>>>>>>>>> request, the more writes we can do in parallel.
> > > >>>>>> The
> > > >>>>>>>>>>>> alternative
> > > >>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>> make
> > > >>>>>>>>>>>>>>>>>>>> use of more connections, but usually we prefer
> > > >>>>>>>> batching
> > > >>>>>>>>>>>> since the
> > > >>>>>>>>>>>>>>>>>> network
> > > >>>>>>>>>>>>>>>>>>>> stack is not really optimized for high
> > > >>>>>>>>> connection/request
> > > >>>>>>>>>>>> loads.
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> Finally with respect to the authorizations, I
> > > >>>>>>> think
> > > >>>>>>>>> it
> > > >>>>>>>>>>>> makes
> > > >>>>>>>>>>>>>>> sense
> > > >>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>> skip
> > > >>>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit confused by
> > > >>>>>> the
> > > >>>>>>>>>> "leader
> > > >>>>>>>>>>>> ID"
> > > >>>>>>>>>>>>>>>> field.
> > > >>>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the request as
> > > >>>>>> from a
> > > >>>>>>>>>> broker
> > > >>>>>>>>>>>> (does
> > > >>>>>>>>>>>>>>> it
> > > >>>>>>>>>>>>>>>>>> matter
> > > >>>>>>>>>>>>>>>>>>>> which one?).
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> We could also make it version-based. For the next
> > > >>>>>>>>>> version, we
> > > >>>>>>>>>>>>>>> could
> > > >>>>>>>>>>>>>>>>>>>> require
> > > >>>>>>>>>>>>>>>>>>>> CLUSTER auth. So clients would not be able to use
> > > >>>>>>> the
> > > >>>>>>>>> API
> > > >>>>>>>>>>>>>>> anymore,
> > > >>>>>>>>>>>>>>>>> which
> > > >>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>> probably what we want.
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> -Jason
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> > > >>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> As a follow up, I was just thinking about the
> > > >>>>>>>>> batching
> > > >>>>>>>>>> a
> > > >>>>>>>>>>>> bit
> > > >>>>>>>>>>>>>>> more.
> > > >>>>>>>>>>>>>>>>>>>>> I suppose if we have one request in flight and
> > > >>>>>> we
> > > >>>>>>>>>> queue up
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>> other
> > > >>>>>>>>>>>>>>>>>>>>> produce requests in some sort of purgatory, we
> > > >>>>>>>> could
> > > >>>>>>>>>> send
> > > >>>>>>>>>>>>>>>>> information
> > > >>>>>>>>>>>>>>>>>>>> out
> > > >>>>>>>>>>>>>>>>>>>>> for all of them rather than one by one. So that
> > > >>>>>>>> would
> > > >>>>>>>>>> be a
> > > >>>>>>>>>>>>>>> benefit
> > > >>>>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>>>> batching partitions to add per transaction.
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> I'll need to think a bit more on the design of
> > > >>>>>>> this
> > > >>>>>>>>>> part
> > > >>>>>>>>>>>> of the
> > > >>>>>>>>>>>>>>>> KIP,
> > > >>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>> will update the KIP in the next few days.
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>>>>>>>>>>> Justine
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan
> > > >>>>>> <
> > > >>>>>>>>>>>>>>>>> jolshan@confluent.io>
> > > >>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> Hey Jason -- thanks for the input -- I was
> > > >>>>>> just
> > > >>>>>>>>>> digging
> > > >>>>>>>>>>>> a bit
> > > >>>>>>>>>>>>>>>>> deeper
> > > >>>>>>>>>>>>>>>>>>>> into
> > > >>>>>>>>>>>>>>>>>>>>>> the design + implementation of the validation
> > > >>>>>>>> calls
> > > >>>>>>>>>> here
> > > >>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>> what
> > > >>>>>>>>>>>>>>>>>> you
> > > >>>>>>>>>>>>>>>>>>>> say
> > > >>>>>>>>>>>>>>>>>>>>>> makes sense.
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> I was wondering about compatibility here.
> > > >>>>>> When
> > > >>>>>>> we
> > > >>>>>>>>>> send
> > > >>>>>>>>>>>>>>> requests
> > > >>>>>>>>>>>>>>>>>>>>>> between brokers, we want to ensure that the
> > > >>>>>>>>> receiving
> > > >>>>>>>>>>>> broker
> > > >>>>>>>>>>>>>>>>>>>> understands
> > > >>>>>>>>>>>>>>>>>>>>>> the request (specifically the new fields).
> > > >>>>>>>>> Typically
> > > >>>>>>>>>>>> this is
> > > >>>>>>>>>>>>>>>> done
> > > >>>>>>>>>>>>>>>>>> via
> > > >>>>>>>>>>>>>>>>>>>>>> IBP/metadata version.
> > > >>>>>>>>>>>>>>>>>>>>>> I'm trying to think if there is a way around
> > > >>>>>> it
> > > >>>>>>>> but
> > > >>>>>>>>>> I'm
> > > >>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>> sure
> > > >>>>>>>>>>>>>>>>>> there
> > > >>>>>>>>>>>>>>>>>>>>> is.
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> As for the improvements -- can you clarify
> > > >>>>>> how
> > > >>>>>>>> the
> > > >>>>>>>>>>>> multiple
> > > >>>>>>>>>>>>>>>>>>>> transactional
> > > >>>>>>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking of a
> > > >>>>>>> case
> > > >>>>>>>>>> where we
> > > >>>>>>>>>>>>>>>>> wait/batch
> > > >>>>>>>>>>>>>>>>>>>>>> multiple produce requests together? My
> > > >>>>>>>>> understanding
> > > >>>>>>>>>> for
> > > >>>>>>>>>>>> now
> > > >>>>>>>>>>>>>>>> was 1
> > > >>>>>>>>>>>>>>>>>>>>>> transactional ID and one validation per 1
> > > >>>>>>> produce
> > > >>>>>>>>>>>> request.
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> Finally with respect to the authorizations, I
> > > >>>>>>>> think
> > > >>>>>>>>>> it
> > > >>>>>>>>>>>> makes
> > > >>>>>>>>>>>>>>>> sense
> > > >>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>> skip
> > > >>>>>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit confused
> > > >>>>>> by
> > > >>>>>>>> the
> > > >>>>>>>>>>>> "leader
> > > >>>>>>>>>>>>>>> ID"
> > > >>>>>>>>>>>>>>>>>> field.
> > > >>>>>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the request as
> > > >>>>>>>> from a
> > > >>>>>>>>>>>> broker
> > > >>>>>>>>>>>>>>> (does
> > > >>>>>>>>>>>>>>>>> it
> > > >>>>>>>>>>>>>>>>>>>>> matter
> > > >>>>>>>>>>>>>>>>>>>>>> which one?).
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> I think I want to adopt these suggestions,
> > > >>>>>> just
> > > >>>>>>>> had
> > > >>>>>>>>>> a few
> > > >>>>>>>>>>>>>>>>> questions
> > > >>>>>>>>>>>>>>>>>> on
> > > >>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>> details.
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>>>>>>>>>>>> Justine
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> On Thu, Jan 5, 2023 at 5:05 PM Jason
> > > >>>>>> Gustafson
> > > >>>>>>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
> > > >>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> Thanks for the proposal.
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> I was thinking about the implementation a
> > > >>>>>>> little
> > > >>>>>>>>>> bit.
> > > >>>>>>>>>>>> In the
> > > >>>>>>>>>>>>>>>>>> current
> > > >>>>>>>>>>>>>>>>>>>>>>> proposal, the behavior depends on whether we
> > > >>>>>>>> have
> > > >>>>>>>>> an
> > > >>>>>>>>>>>> old or
> > > >>>>>>>>>>>>>>> new
> > > >>>>>>>>>>>>>>>>>>>> client.
> > > >>>>>>>>>>>>>>>>>>>>>>> For
> > > >>>>>>>>>>>>>>>>>>>>>>> old clients, we send `DescribeTransactions`
> > > >>>>>>> and
> > > >>>>>>>>>> verify
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>> result
> > > >>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>>>>>>> new clients, we send `AddPartitionsToTxn`.
> > > >>>>>> We
> > > >>>>>>>>> might
> > > >>>>>>>>>> be
> > > >>>>>>>>>>>> able
> > > >>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>> simplify
> > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> implementation if we can use the same
> > > >>>>>> request
> > > >>>>>>>>> type.
> > > >>>>>>>>>> For
> > > >>>>>>>>>>>>>>>> example,
> > > >>>>>>>>>>>>>>>>>>>> what if
> > > >>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>> bump the protocol version for
> > > >>>>>>>> `AddPartitionsToTxn`
> > > >>>>>>>>>> and
> > > >>>>>>>>>>>> add a
> > > >>>>>>>>>>>>>>>>>>>>>>> `validateOnly`
> > > >>>>>>>>>>>>>>>>>>>>>>> flag? For older versions, we can set
> > > >>>>>>>>>>>> `validateOnly=true` so
> > > >>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> request only returns successfully if the
> > > >>>>>>>> partition
> > > >>>>>>>>>> had
> > > >>>>>>>>>>>>>>> already
> > > >>>>>>>>>>>>>>>>> been
> > > >>>>>>>>>>>>>>>>>>>>> added.
> > > >>>>>>>>>>>>>>>>>>>>>>> For new versions, we can set
> > > >>>>>>>> `validateOnly=false`
> > > >>>>>>>>>> and
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>> partition
> > > >>>>>>>>>>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>>>>> added to the transaction. The other slightly
> > > >>>>>>>>>> annoying
> > > >>>>>>>>>>>> thing
> > > >>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>>>>>>>> get around is the need to collect the
> > > >>>>>>>> transaction
> > > >>>>>>>>>> state
> > > >>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>> all
> > > >>>>>>>>>>>>>>>>>>>>> partitions
> > > >>>>>>>>>>>>>>>>>>>>>>> even when we only care about a subset.
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> Some additional improvements to consider:
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> - We can give `AddPartitionsToTxn` better
> > > >>>>>>> batch
> > > >>>>>>>>>> support
> > > >>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>>>> inter-broker
> > > >>>>>>>>>>>>>>>>>>>>>>> usage. Currently we only allow one
> > > >>>>>>>>>> `TransactionalId` to
> > > >>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>> specified,
> > > >>>>>>>>>>>>>>>>>>>>> but
> > > >>>>>>>>>>>>>>>>>>>>>>> the broker may get some benefit being able
> > > >>>>>> to
> > > >>>>>>>>> batch
> > > >>>>>>>>>>>> across
> > > >>>>>>>>>>>>>>>>> multiple
> > > >>>>>>>>>>>>>>>>>>>>>>> transactions.
> > > >>>>>>>>>>>>>>>>>>>>>>> - Another small improvement is skipping
> > > >>>>>> topic
> > > >>>>>>>>>>>> authorization
> > > >>>>>>>>>>>>>>>>> checks
> > > >>>>>>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>>>>>>> `AddPartitionsToTxn` when the request is
> > > >>>>>> from
> > > >>>>>>> a
> > > >>>>>>>>>> broker.
> > > >>>>>>>>>>>>>>> Perhaps
> > > >>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>> can
> > > >>>>>>>>>>>>>>>>>>>>> add
> > > >>>>>>>>>>>>>>>>>>>>>>> a field for the `LeaderId` or something like
> > > >>>>>>>> that
> > > >>>>>>>>>> and
> > > >>>>>>>>>>>>>>> require
> > > >>>>>>>>>>>>>>>>>> CLUSTER
> > > >>>>>>>>>>>>>>>>>>>>>>> permission when set.
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> Best,
> > > >>>>>>>>>>>>>>>>>>>>>>> Jason
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
> > > >>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation. It makes sense
> > > >>>>>>> to
> > > >>>>>>>> me
> > > >>>>>>>>>> now.
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 1:42 PM Justine
> > > >>>>>>> Olshan
> > > >>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>> My understanding of the mechanism is
> > > >>>>>> that
> > > >>>>>>>> when
> > > >>>>>>>>>> we
> > > >>>>>>>>>>>> get to
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>> last
> > > >>>>>>>>>>>>>>>>>>>>>>> epoch,
> > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>> increment to the fencing/last epoch and
> > > >>>>>> if
> > > >>>>>>>> any
> > > >>>>>>>>>>>> further
> > > >>>>>>>>>>>>>>>>> requests
> > > >>>>>>>>>>>>>>>>>>>> come
> > > >>>>>>>>>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>>>>>>>>> this producer ID they are fenced. Then
> > > >>>>>> the
> > > >>>>>>>>>> producer
> > > >>>>>>>>>>>>>>> gets a
> > > >>>>>>>>>>>>>>>>> new
> > > >>>>>>>>>>>>>>>>>> ID
> > > >>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>>> restarts with epoch/sequence 0. The
> > > >>>>>> fenced
> > > >>>>>>>>> epoch
> > > >>>>>>>>>>>> sticks
> > > >>>>>>>>>>>>>>>>> around
> > > >>>>>>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>> duration of producer.id.expiration.ms
> > > >>>>>> and
> > > >>>>>>>>>> blocks
> > > >>>>>>>>>>>> any
> > > >>>>>>>>>>>>>>> late
> > > >>>>>>>>>>>>>>>>>>>> messages
> > > >>>>>>>>>>>>>>>>>>>>>>>> there.
> > > >>>>>>>>>>>>>>>>>>>>>>>>> The new ID will get to take advantage of
> > > >>>>>>> the
> > > >>>>>>>>>>>> improved
> > > >>>>>>>>>>>>>>>>> semantics
> > > >>>>>>>>>>>>>>>>>>>>> around
> > > >>>>>>>>>>>>>>>>>>>>>>>>> non-zero start sequences. So I think we
> > > >>>>>>> are
> > > >>>>>>>>>> covered.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>> The only potential issue is overloading
> > > >>>>>>> the
> > > >>>>>>>>>> cache,
> > > >>>>>>>>>>>> but
> > > >>>>>>>>>>>>>>>>>> hopefully
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>> improvements (lowered
> > > >>>>>>>>> producer.id.expiration.ms
> > > >>>>>>>>>> )
> > > >>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>> help
> > > >>>>>>>>>>>>>>>>>> with
> > > >>>>>>>>>>>>>>>>>>>>> that.
> > > >>>>>>>>>>>>>>>>>>>>>>>> Let
> > > >>>>>>>>>>>>>>>>>>>>>>>>> me know if you still have concerns.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
> > > >>>>>>>>>>>>>>>>>>>> <ju...@confluent.io.invalid>
> > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> 70. The proposed fencing logic doesn't
> > > >>>>>>>> apply
> > > >>>>>>>>>> when
> > > >>>>>>>>>>>> pid
> > > >>>>>>>>>>>>>>>>>> changes,
> > > >>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> right? If so, I am not sure how
> > > >>>>>> complete
> > > >>>>>>>> we
> > > >>>>>>>>>> are
> > > >>>>>>>>>>>>>>>> addressing
> > > >>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>> issue
> > > >>>>>>>>>>>>>>>>>>>>>>>> if
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> the pid changes more frequently.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 16, 2022 at 9:16 AM
> > > >>>>>> Justine
> > > >>>>>>>>> Olshan
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for replying!
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> 70.We already do the overflow
> > > >>>>>>> mechanism,
> > > >>>>>>>>> so
> > > >>>>>>>>>> my
> > > >>>>>>>>>>>>>>> change
> > > >>>>>>>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>>>>> just
> > > >>>>>>>>>>>>>>>>>>>>>>> make
> > > >>>>>>>>>>>>>>>>>>>>>>>>> it
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> happen more often.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I was also not suggesting a new
> > > >>>>>> field
> > > >>>>>>> in
> > > >>>>>>>>> the
> > > >>>>>>>>>>>> log,
> > > >>>>>>>>>>>>>>> but
> > > >>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>> response,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> which would be gated by the client
> > > >>>>>>>>> version.
> > > >>>>>>>>>>>> Sorry if
> > > >>>>>>>>>>>>>>>>>>>> something
> > > >>>>>>>>>>>>>>>>>>>>>>> there
> > > >>>>>>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> unclear. I think we are starting to
> > > >>>>>>>>> diverge.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> The goal of this KIP is to not
> > > >>>>>> change
> > > >>>>>>> to
> > > >>>>>>>>> the
> > > >>>>>>>>>>>> marker
> > > >>>>>>>>>>>>>>>>> format
> > > >>>>>>>>>>>>>>>>>> at
> > > >>>>>>>>>>>>>>>>>>>>> all.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> 71. Yes, I guess I was going under
> > > >>>>>> the
> > > >>>>>>>>>>>> assumption
> > > >>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> log
> > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>>>>>>>>>> just
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> look at its last epoch and treat it
> > > >>>>>> as
> > > >>>>>>>> the
> > > >>>>>>>>>>>> current
> > > >>>>>>>>>>>>>>>>> epoch. I
> > > >>>>>>>>>>>>>>>>>>>>>>> suppose
> > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> have some special logic that if the
> > > >>>>>>> last
> > > >>>>>>>>>> epoch
> > > >>>>>>>>>>>> was
> > > >>>>>>>>>>>>>>> on a
> > > >>>>>>>>>>>>>>>>>>>> marker
> > > >>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> actually
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> expect the next epoch or something
> > > >>>>>>> like
> > > >>>>>>>>>> that. We
> > > >>>>>>>>>>>>>>> just
> > > >>>>>>>>>>>>>>>>> need
> > > >>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>> distinguish
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> based on whether we had a
> > > >>>>>> commit/abort
> > > >>>>>>>>>> marker.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> 72.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> if the producer epoch hasn't been
> > > >>>>>>>> bumped
> > > >>>>>>>>>> on
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
> > > >>>>>>>> message
> > > >>>>>>>>>> will
> > > >>>>>>>>>>>> fail
> > > >>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> sequence
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> validation
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the producer
> > > >>>>>>>> epoch
> > > >>>>>>>>>> has
> > > >>>>>>>>>>>> been
> > > >>>>>>>>>>>>>>>>> bumped,
> > > >>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>> ignore
> > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck message
> > > >>>>>>>> could
> > > >>>>>>>>>> be
> > > >>>>>>>>>>>>>>> appended
> > > >>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>> log.
> > > >>>>>>>>>>>>>>>>>>>>>>>> So,
> > > >>>>>>>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
> > > >>>>>> guard?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I follow that "the
> > > >>>>>>> message
> > > >>>>>>>>> will
> > > >>>>>>>>>>>> fail
> > > >>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> sequence
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> validation".
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> In some of these cases, we had an
> > > >>>>>>> abort
> > > >>>>>>>>>> marker
> > > >>>>>>>>>>>> (due
> > > >>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>> an
> > > >>>>>>>>>>>>>>>>>>>> error)
> > > >>>>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>>> then
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the late message comes in with the
> > > >>>>>>>> correct
> > > >>>>>>>>>>>> sequence
> > > >>>>>>>>>>>>>>>>> number.
> > > >>>>>>>>>>>>>>>>>>>> This
> > > >>>>>>>>>>>>>>>>>>>>>>> is a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> case
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> covered by the KIP.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> The latter case is actually not
> > > >>>>>>>> something
> > > >>>>>>>>>> we've
> > > >>>>>>>>>>>>>>>>> considered
> > > >>>>>>>>>>>>>>>>>>>>> here. I
> > > >>>>>>>>>>>>>>>>>>>>>>>>> think
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> generally when we bump the epoch, we
> > > >>>>>>> are
> > > >>>>>>>>>>>> accepting
> > > >>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> sequence
> > > >>>>>>>>>>>>>>>>>>>>>>>>> does
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> not need to be checked anymore. My
> > > >>>>>>>>>>>> understanding is
> > > >>>>>>>>>>>>>>>> also
> > > >>>>>>>>>>>>>>>>>>>> that we
> > > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> typically bump epoch mid transaction
> > > >>>>>>>>> (based
> > > >>>>>>>>>> on a
> > > >>>>>>>>>>>>>>> quick
> > > >>>>>>>>>>>>>>>>> look
> > > >>>>>>>>>>>>>>>>>>>> at
> > > >>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>> code)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but let me know if that is the case.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 15, 2022 at 12:23 PM Jun
> > > >>>>>>> Rao
> > > >>>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the reply.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Assigning a new pid on int
> > > >>>>>>>> overflow
> > > >>>>>>>>>> seems
> > > >>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>> bit
> > > >>>>>>>>>>>>>>>>>> hacky.
> > > >>>>>>>>>>>>>>>>>>>> If
> > > >>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>> need a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> level id, it will be better to
> > > >>>>>> model
> > > >>>>>>>>> this
> > > >>>>>>>>>>>>>>> explicitly.
> > > >>>>>>>>>>>>>>>>>>>> Adding a
> > > >>>>>>>>>>>>>>>>>>>>>>> new
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> field
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> would require a bit more work
> > > >>>>>> since
> > > >>>>>>> it
> > > >>>>>>>>>>>> requires a
> > > >>>>>>>>>>>>>>> new
> > > >>>>>>>>>>>>>>>>> txn
> > > >>>>>>>>>>>>>>>>>>>>> marker
> > > >>>>>>>>>>>>>>>>>>>>>>>>> format
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the log. So, we probably need to
> > > >>>>>>> guard
> > > >>>>>>>>> it
> > > >>>>>>>>>>>> with an
> > > >>>>>>>>>>>>>>> IBP
> > > >>>>>>>>>>>>>>>>> or
> > > >>>>>>>>>>>>>>>>>>>>>>> metadata
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> version
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and document the impact on
> > > >>>>>> downgrade
> > > >>>>>>>>> once
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>> new
> > > >>>>>>>>>>>>>>>>> format
> > > >>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>> written
> > > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 71. Hmm, once the marker is
> > > >>>>>> written,
> > > >>>>>>>> the
> > > >>>>>>>>>>>> partition
> > > >>>>>>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>>>>>>> expect
> > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>> next
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> append to be on the next epoch.
> > > >>>>>> Does
> > > >>>>>>>>> that
> > > >>>>>>>>>>>> cover
> > > >>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>> case
> > > >>>>>>>>>>>>>>>>>>>> you
> > > >>>>>>>>>>>>>>>>>>>>>>>>> mentioned?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 72. Also, just to be clear on the
> > > >>>>>>>>> stucked
> > > >>>>>>>>>>>> message
> > > >>>>>>>>>>>>>>>> issue
> > > >>>>>>>>>>>>>>>>>>>>>>> described
> > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> motivation. With EoS, we also
> > > >>>>>>> validate
> > > >>>>>>>>> the
> > > >>>>>>>>>>>>>>> sequence
> > > >>>>>>>>>>>>>>>> id
> > > >>>>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>>>>>>>>> idempotency.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> So,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> with the current logic, if the
> > > >>>>>>>> producer
> > > >>>>>>>>>> epoch
> > > >>>>>>>>>>>>>>> hasn't
> > > >>>>>>>>>>>>>>>>> been
> > > >>>>>>>>>>>>>>>>>>>>>>> bumped on
> > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
> > > >>>>>>>>> message
> > > >>>>>>>>>> will
> > > >>>>>>>>>>>>>>> fail
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>> sequence
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> validation
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the
> > > >>>>>> producer
> > > >>>>>>>>>> epoch has
> > > >>>>>>>>>>>>>>> been
> > > >>>>>>>>>>>>>>>>>>>> bumped, we
> > > >>>>>>>>>>>>>>>>>>>>>>>> ignore
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck
> > > >>>>>> message
> > > >>>>>>>>>> could be
> > > >>>>>>>>>>>>>>>> appended
> > > >>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> log.
> > > >>>>>>>>>>>>>>>>>>>>>>>>> So,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
> > > >>>>>>> guard?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 10:44 AM
> > > >>>>>>>> Justine
> > > >>>>>>>>>>>> Olshan
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > > >>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias — thanks again for
> > > >>>>>> taking
> > > >>>>>>>>> time
> > > >>>>>>>>>> to
> > > >>>>>>>>>>>> look
> > > >>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>> this.
> > > >>>>>>>>>>>>>>>>>>>> You
> > > >>>>>>>>>>>>>>>>>>>>>>> said:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My proposal was only focusing
> > > >>>>>> to
> > > >>>>>>>>> avoid
> > > >>>>>>>>>>>>>>> dangling
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
> > > >>>>>> added
> > > >>>>>>>>>> without
> > > >>>>>>>>>>>>>>>> registered
> > > >>>>>>>>>>>>>>>>>>>>>>> partition.
> > > >>>>>>>>>>>>>>>>>>>>>>>> --
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more details
> > > >>>>>> to
> > > >>>>>>>> the
> > > >>>>>>>>>> KIP
> > > >>>>>>>>>>>> about
> > > >>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>> scenario
> > > >>>>>>>>>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> better
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I understand what
> > > >>>>>> you
> > > >>>>>>>>> mean
> > > >>>>>>>>>>>> here.
> > > >>>>>>>>>>>>>>> The
> > > >>>>>>>>>>>>>>>>>>>> motivation
> > > >>>>>>>>>>>>>>>>>>>>>>>>> section
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> describes two scenarios about
> > > >>>>>> how
> > > >>>>>>>> the
> > > >>>>>>>>>> record
> > > >>>>>>>>>>>>>>> can be
> > > >>>>>>>>>>>>>>>>>> added
> > > >>>>>>>>>>>>>>>>>>>>>>>> without a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> registered partition:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Another way hanging
> > > >>>>>> transactions
> > > >>>>>>>> can
> > > >>>>>>>>>>>> occur is
> > > >>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>>>>> client
> > > >>>>>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> buggy
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> may somehow try to write to a
> > > >>>>>>>>> partition
> > > >>>>>>>>>>>> before
> > > >>>>>>>>>>>>>>> it
> > > >>>>>>>>>>>>>>>>> adds
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>> partition
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> For the first example of this
> > > >>>>>>> would
> > > >>>>>>>> it
> > > >>>>>>>>>> be
> > > >>>>>>>>>>>>>>> helpful
> > > >>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>> say
> > > >>>>>>>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> comes in after the abort, but
> > > >>>>>>> before
> > > >>>>>>>>> the
> > > >>>>>>>>>>>>>>> partition
> > > >>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>> added
> > > >>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> next
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction so it becomes
> > > >>>>>>> "hanging."
> > > >>>>>>>>>>>> Perhaps the
> > > >>>>>>>>>>>>>>>> next
> > > >>>>>>>>>>>>>>>>>>>>> sentence
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> describing
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the message becoming part of the
> > > >>>>>>>> next
> > > >>>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>> (a
> > > >>>>>>>>>>>>>>>>>>>>> different
> > > >>>>>>>>>>>>>>>>>>>>>>>>> case)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> was
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not properly differentiated.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun — thanks for reading the
> > > >>>>>> KIP.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. The int typing was a
> > > >>>>>> concern.
> > > >>>>>>>>>> Currently
> > > >>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>> have a
> > > >>>>>>>>>>>>>>>>>>>>>>> mechanism
> > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> place
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> fence the final epoch when the
> > > >>>>>>> epoch
> > > >>>>>>>>> is
> > > >>>>>>>>>>>> about to
> > > >>>>>>>>>>>>>>>>>> overflow
> > > >>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>>> assign
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer ID with epoch 0. Of
> > > >>>>>>> course,
> > > >>>>>>>>>> this
> > > >>>>>>>>>>>> is a
> > > >>>>>>>>>>>>>>> bit
> > > >>>>>>>>>>>>>>>>>> tricky
> > > >>>>>>>>>>>>>>>>>>>>>>> when it
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> comes
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response back to the client.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Making this a long could be
> > > >>>>>>> another
> > > >>>>>>>>>> option,
> > > >>>>>>>>>>>> but
> > > >>>>>>>>>>>>>>> I
> > > >>>>>>>>>>>>>>>>>> wonder
> > > >>>>>>>>>>>>>>>>>>>> are
> > > >>>>>>>>>>>>>>>>>>>>>>>> there
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> implications on changing this
> > > >>>>>>> field
> > > >>>>>>>> if
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>>> epoch is
> > > >>>>>>>>>>>>>>>>>>>>> persisted
> > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> disk?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to check the usages.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 71.This was something Matthias
> > > >>>>>>> asked
> > > >>>>>>>>>> about
> > > >>>>>>>>>>>> as
> > > >>>>>>>>>>>>>>>> well. I
> > > >>>>>>>>>>>>>>>>>> was
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> considering a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible edge case where a
> > > >>>>>> produce
> > > >>>>>>>>>> request
> > > >>>>>>>>>>>> from
> > > >>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>> new
> > > >>>>>>>>>>>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> somehow
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> gets sent right after the marker
> > > >>>>>>> is
> > > >>>>>>>>>>>> written, but
> > > >>>>>>>>>>>>>>>>> before
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> alerted of the newly bumped
> > > >>>>>> epoch.
> > > >>>>>>>> In
> > > >>>>>>>>>> this
> > > >>>>>>>>>>>>>>> case, we
> > > >>>>>>>>>>>>>>>>> may
> > > >>>>>>>>>>>>>>>>>>>>>>> include
> > > >>>>>>>>>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> record
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we don't want to. I suppose
> > > >>>>>>> we
> > > >>>>>>>>>> could
> > > >>>>>>>>>>>> try
> > > >>>>>>>>>>>>>>> to do
> > > >>>>>>>>>>>>>>>>>>>>> something
> > > >>>>>>>>>>>>>>>>>>>>>>>>> client
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> side
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to bump the epoch after sending
> > > >>>>>> an
> > > >>>>>>>>>> endTxn as
> > > >>>>>>>>>>>>>>> well
> > > >>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>>> scenario
> > > >>>>>>>>>>>>>>>>>>>>>>>>> —
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> I
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wonder how it would work when
> > > >>>>>> the
> > > >>>>>>>>>> server is
> > > >>>>>>>>>>>>>>>> aborting
> > > >>>>>>>>>>>>>>>>>>>> based
> > > >>>>>>>>>>>>>>>>>>>>> on
> > > >>>>>>>>>>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> error. I could also be missing
> > > >>>>>>>>>> something and
> > > >>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>> scenario
> > > >>>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> actually
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again to everyone reading
> > > >>>>>>> and
> > > >>>>>>>>>>>> commenting.
> > > >>>>>>>>>>>>>>>> Let
> > > >>>>>>>>>>>>>>>>> me
> > > >>>>>>>>>>>>>>>>>>>> know
> > > >>>>>>>>>>>>>>>>>>>>>>>> about
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> further questions or comments.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 9:41 AM
> > > >>>>>>> Jun
> > > >>>>>>>>> Rao
> > > >>>>>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP. A couple
> > > >>>>>> of
> > > >>>>>>>>>> comments.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Currently, the producer
> > > >>>>>>> epoch
> > > >>>>>>>> is
> > > >>>>>>>>>> an
> > > >>>>>>>>>>>> int.
> > > >>>>>>>>>>>>>>> I am
> > > >>>>>>>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>>>>>>> sure
> > > >>>>>>>>>>>>>>>>>>>>>>> if
> > > >>>>>>>>>>>>>>>>>>>>>>>>> it's
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> enough
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to accommodate all
> > > >>>>>> transactions
> > > >>>>>>> in
> > > >>>>>>>>> the
> > > >>>>>>>>>>>>>>> lifetime
> > > >>>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>>>>>>> producer.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> Should
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> change that to a long or add a
> > > >>>>>>> new
> > > >>>>>>>>>> long
> > > >>>>>>>>>>>> field
> > > >>>>>>>>>>>>>>>> like
> > > >>>>>>>>>>>>>>>>>>>> txnId?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 71. "it will write the prepare
> > > >>>>>>>>> commit
> > > >>>>>>>>>>>> message
> > > >>>>>>>>>>>>>>>> with
> > > >>>>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>>>>> bumped
> > > >>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> send
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WriteTxnMarkerRequests with
> > > >>>>>> the
> > > >>>>>>>>> bumped
> > > >>>>>>>>>>>> epoch."
> > > >>>>>>>>>>>>>>>> Hmm,
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > > >>>>>>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> associated
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the current txn right?
> > > >>>>>> So,
> > > >>>>>>> it
> > > >>>>>>>>>> seems
> > > >>>>>>>>>>>>>>> weird to
> > > >>>>>>>>>>>>>>>>>>>> write a
> > > >>>>>>>>>>>>>>>>>>>>>>>> commit
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with a bumped epoch. Should we
> > > >>>>>>>> only
> > > >>>>>>>>>> bump
> > > >>>>>>>>>>>> up
> > > >>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>> epoch
> > > >>>>>>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> EndTxnResponse
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rename the field to sth like
> > > >>>>>>>>>>>>>>> nextProducerEpoch?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 12, 2022 at 8:54
> > > >>>>>> PM
> > > >>>>>>>>>> Matthias
> > > >>>>>>>>>>>> J.
> > > >>>>>>>>>>>>>>> Sax <
> > > >>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the background.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30: SGTM. My proposal was
> > > >>>>>>>> only
> > > >>>>>>>>>>>> focusing
> > > >>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>> avoid
> > > >>>>>>>>>>>>>>>>>>>>>>> dangling
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
> > > >>>>>>>> added
> > > >>>>>>>>>>>> without
> > > >>>>>>>>>>>>>>>>>> registered
> > > >>>>>>>>>>>>>>>>>>>>>>>> partition.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> --
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more
> > > >>>>>> details
> > > >>>>>>>> to
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>> KIP
> > > >>>>>>>>>>>>>>> about
> > > >>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>> scenario
> > > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40: I think you hit a fair
> > > >>>>>>> point
> > > >>>>>>>>>> about
> > > >>>>>>>>>>>> race
> > > >>>>>>>>>>>>>>>>>>>> conditions
> > > >>>>>>>>>>>>>>>>>>>>> or
> > > >>>>>>>>>>>>>>>>>>>>>>>>> client
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> bugs
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (incorrectly not bumping the
> > > >>>>>>>>>> epoch). The
> > > >>>>>>>>>>>>>>>>>>>>>>> complexity/confusion
> > > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> using
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the bumped epoch I see, is
> > > >>>>>>>> mainly
> > > >>>>>>>>>> for
> > > >>>>>>>>>>>>>>> internal
> > > >>>>>>>>>>>>>>>>>>>>> debugging,
> > > >>>>>>>>>>>>>>>>>>>>>>> ie,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> inspecting
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log segment dumps -- it
> > > >>>>>> seems
> > > >>>>>>>>>> harder to
> > > >>>>>>>>>>>>>>> reason
> > > >>>>>>>>>>>>>>>>>> about
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>> system
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> us
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> humans. But if we get better
> > > >>>>>>>>>>>> guarantees, it
> > > >>>>>>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>>>>> worth to
> > > >>>>>>>>>>>>>>>>>>>>>>>>> use
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped epoch.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60: as I mentioned already,
> > > >>>>>> I
> > > >>>>>>>>> don't
> > > >>>>>>>>>>>> know the
> > > >>>>>>>>>>>>>>>>> broker
> > > >>>>>>>>>>>>>>>>>>>>>>> internals
> > > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> provide
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more input. So if nobody
> > > >>>>>> else
> > > >>>>>>>>> chimes
> > > >>>>>>>>>>>> in, we
> > > >>>>>>>>>>>>>>>>> should
> > > >>>>>>>>>>>>>>>>>>>> just
> > > >>>>>>>>>>>>>>>>>>>>>>> move
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> forward
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with your proposal.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 12/6/22 4:22 PM, Justine
> > > >>>>>>>> Olshan
> > > >>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi all,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> After Artem's questions
> > > >>>>>>> about
> > > >>>>>>>>>> error
> > > >>>>>>>>>>>>>>> behavior,
> > > >>>>>>>>>>>>>>>>>> I've
> > > >>>>>>>>>>>>>>>>>>>>>>>>> re-evaluated
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unknown producer ID
> > > >>>>>>> exception
> > > >>>>>>>>> and
> > > >>>>>>>>>> had
> > > >>>>>>>>>>>> some
> > > >>>>>>>>>>>>>>>>>>>> discussions
> > > >>>>>>>>>>>>>>>>>>>>>>>>> offline.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think generally it makes
> > > >>>>>>>> sense
> > > >>>>>>>>>> to
> > > >>>>>>>>>>>>>>> simplify
> > > >>>>>>>>>>>>>>>>>> error
> > > >>>>>>>>>>>>>>>>>>>>>>> handling
> > > >>>>>>>>>>>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> cases
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this and the
> > > >>>>>>>> UNKNOWN_PRODUCER_ID
> > > >>>>>>>>>> error
> > > >>>>>>>>>>>>>>> has a
> > > >>>>>>>>>>>>>>>>>> pretty
> > > >>>>>>>>>>>>>>>>>>>>> long
> > > >>>>>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complicated
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> history. Because of this,
> > > >>>>>> I
> > > >>>>>>>>>> propose
> > > >>>>>>>>>>>>>>> adding a
> > > >>>>>>>>>>>>>>>>> new
> > > >>>>>>>>>>>>>>>>>>>> error
> > > >>>>>>>>>>>>>>>>>>>>>>> code
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ABORTABLE_ERROR
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that when encountered by
> > > >>>>>> new
> > > >>>>>>>>>> clients
> > > >>>>>>>>>>>>>>> (gated
> > > >>>>>>>>>>>>>>>> by
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> produce
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will simply abort the
> > > >>>>>>>>> transaction.
> > > >>>>>>>>>>>> This
> > > >>>>>>>>>>>>>>>> allows
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> server
> > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> some
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> say
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in whether the client
> > > >>>>>> aborts
> > > >>>>>>>> and
> > > >>>>>>>>>> makes
> > > >>>>>>>>>>>>>>>> handling
> > > >>>>>>>>>>>>>>>>>>>> much
> > > >>>>>>>>>>>>>>>>>>>>>>>> simpler.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> future, we can also use
> > > >>>>>> this
> > > >>>>>>>>>> error in
> > > >>>>>>>>>>>>>>> other
> > > >>>>>>>>>>>>>>>>>>>> situations
> > > >>>>>>>>>>>>>>>>>>>>>>>> where
> > > >>>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> want
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> abort the transactions. We
> > > >>>>>>> can
> > > >>>>>>>>>> even
> > > >>>>>>>>>>>> use on
> > > >>>>>>>>>>>>>>>>> other
> > > >>>>>>>>>>>>>>>>>>>> apis.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've added this to the
> > > >>>>>> KIP.
> > > >>>>>>>> Let
> > > >>>>>>>>> me
> > > >>>>>>>>>>>> know if
> > > >>>>>>>>>>>>>>>>> there
> > > >>>>>>>>>>>>>>>>>>>> are
> > > >>>>>>>>>>>>>>>>>>>>> any
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> questions
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> or
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 2, 2022 at
> > > >>>>>> 10:22
> > > >>>>>>>> AM
> > > >>>>>>>>>>>> Justine
> > > >>>>>>>>>>>>>>>> Olshan
> > > >>>>>>>>>>>>>>>>> <
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> jolshan@confluent.io
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey Matthias,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30 — Maybe I also
> > > >>>>>> didn't
> > > >>>>>>>>>> express
> > > >>>>>>>>>>>>>>> myself
> > > >>>>>>>>>>>>>>>>>>>> clearly.
> > > >>>>>>>>>>>>>>>>>>>>> For
> > > >>>>>>>>>>>>>>>>>>>>>>>>> older
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't have a way to
> > > >>>>>>>> distinguish
> > > >>>>>>>>>>>> between a
> > > >>>>>>>>>>>>>>>>>> previous
> > > >>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> current
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction since we
> > > >>>>>> don't
> > > >>>>>>>> have
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>> epoch
> > > >>>>>>>>>>>>>>>>> bump.
> > > >>>>>>>>>>>>>>>>>>>> This
> > > >>>>>>>>>>>>>>>>>>>>>>> means
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message from the previous
> > > >>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>> may be
> > > >>>>>>>>>>>>>>>>>>>> added to
> > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>> new
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> one.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> With
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older clients — we can't
> > > >>>>>>>>>> guarantee
> > > >>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>> won't
> > > >>>>>>>>>>>>>>>>>>>> happen
> > > >>>>>>>>>>>>>>>>>>>>>>> if we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> already
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call
> > > >>>>>>> (why
> > > >>>>>>>> we
> > > >>>>>>>>>> make
> > > >>>>>>>>>>>>>>> changes
> > > >>>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> newer
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> client)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can at least gate some by
> > > >>>>>>>>>> ensuring
> > > >>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> partition
> > > >>>>>>>>>>>>>>>>>>>>>>> has
> > > >>>>>>>>>>>>>>>>>>>>>>>>> been
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> added
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. The
> > > >>>>>> rationale
> > > >>>>>>>> here
> > > >>>>>>>>>> is
> > > >>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>> there
> > > >>>>>>>>>>>>>>>>>> are
> > > >>>>>>>>>>>>>>>>>>>>>>> likely
> > > >>>>>>>>>>>>>>>>>>>>>>>>> LESS
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrivals
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as time goes on, so
> > > >>>>>>> hopefully
> > > >>>>>>>>>> most
> > > >>>>>>>>>>>> late
> > > >>>>>>>>>>>>>>>>> arrivals
> > > >>>>>>>>>>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>>>>>>>>>> come
> > > >>>>>>>>>>>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> BEFORE
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call.
> > > >>>>>>>> Those
> > > >>>>>>>>>> that
> > > >>>>>>>>>>>>>>> arrive
> > > >>>>>>>>>>>>>>>>>> before
> > > >>>>>>>>>>>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> properly
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> gated
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> > > >>>>>>> describeTransactions
> > > >>>>>>>>>>>> approach.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we take the approach
> > > >>>>>> you
> > > >>>>>>>>>>>> suggested,
> > > >>>>>>>>>>>>>>> ANY
> > > >>>>>>>>>>>>>>>>> late
> > > >>>>>>>>>>>>>>>>>>>>> arrival
> > > >>>>>>>>>>>>>>>>>>>>>>>>> from a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> previous
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction will be
> > > >>>>>> added.
> > > >>>>>>>> And
> > > >>>>>>>>> we
> > > >>>>>>>>>>>> don't
> > > >>>>>>>>>>>>>>> want
> > > >>>>>>>>>>>>>>>>>>>> that. I
> > > >>>>>>>>>>>>>>>>>>>>>>> also
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> see
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> benefit in sending
> > > >>>>>>>>>> addPartitionsToTxn
> > > >>>>>>>>>>>>>>> over
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>> describeTxns
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> They
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> both be one extra RPC to
> > > >>>>>>> the
> > > >>>>>>>>> Txn
> > > >>>>>>>>>>>>>>>> coordinator.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To be clear — newer
> > > >>>>>> clients
> > > >>>>>>>>> will
> > > >>>>>>>>>> use
> > > >>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> instead
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTxns.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that if we
> > > >>>>>>> have
> > > >>>>>>>>>> some
> > > >>>>>>>>>>>> delay
> > > >>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>> client
> > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>> bump
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it could continue to send
> > > >>>>>>>> epoch
> > > >>>>>>>>>> 73
> > > >>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>> those
> > > >>>>>>>>>>>>>>>>>>>> records
> > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fenced.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Perhaps this is not an
> > > >>>>>>> issue
> > > >>>>>>>> if
> > > >>>>>>>>>> we
> > > >>>>>>>>>>>> don't
> > > >>>>>>>>>>>>>>>> allow
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>> next
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> produce
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> go
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through before the EndTxn
> > > >>>>>>>>> request
> > > >>>>>>>>>>>>>>> returns.
> > > >>>>>>>>>>>>>>>> I'm
> > > >>>>>>>>>>>>>>>>>>>> also
> > > >>>>>>>>>>>>>>>>>>>>>>>> thinking
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> about
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cases of
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure. I will need to
> > > >>>>>>> think
> > > >>>>>>>>> on
> > > >>>>>>>>>>>> this a
> > > >>>>>>>>>>>>>>> bit.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I wasn't sure if it was
> > > >>>>>>> that
> > > >>>>>>>>>>>> confusing.
> > > >>>>>>>>>>>>>>> But
> > > >>>>>>>>>>>>>>>> if
> > > >>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>> think it
> > > >>>>>>>>>>>>>>>>>>>>>>>>> is,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> investigate other ways.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure these are
> > > >>>>>> the
> > > >>>>>>>> same
> > > >>>>>>>>>>>>>>> purgatories
> > > >>>>>>>>>>>>>>>>>> since
> > > >>>>>>>>>>>>>>>>>>>> one
> > > >>>>>>>>>>>>>>>>>>>>>>> is a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> produce
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory (I was planning
> > > >>>>>>> on
> > > >>>>>>>>>> using a
> > > >>>>>>>>>>>>>>>> callback
> > > >>>>>>>>>>>>>>>>>>>> rather
> > > >>>>>>>>>>>>>>>>>>>>>>> than
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the other is simply a
> > > >>>>>>> request
> > > >>>>>>>>> to
> > > >>>>>>>>>>>> append
> > > >>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> log.
> > > >>>>>>>>>>>>>>>>>>>>> Not
> > > >>>>>>>>>>>>>>>>>>>>>>>> sure
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> structure here for
> > > >>>>>>> ordering,
> > > >>>>>>>>> but
> > > >>>>>>>>>> my
> > > >>>>>>>>>>>>>>>>>> understanding
> > > >>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handle the write request
> > > >>>>>>>> before
> > > >>>>>>>>>> it
> > > >>>>>>>>>>>> hears
> > > >>>>>>>>>>>>>>>> back
> > > >>>>>>>>>>>>>>>>>> from
> > > >>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> Txn
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Coordinator.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if I
> > > >>>>>>>> misunderstood
> > > >>>>>>>>>>>> something
> > > >>>>>>>>>>>>>>> or
> > > >>>>>>>>>>>>>>>>>>>> something
> > > >>>>>>>>>>>>>>>>>>>>>>> was
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> unclear.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 1, 2022 at
> > > >>>>>>> 12:15
> > > >>>>>>>> PM
> > > >>>>>>>>>>>> Matthias
> > > >>>>>>>>>>>>>>> J.
> > > >>>>>>>>>>>>>>>>> Sax
> > > >>>>>>>>>>>>>>>>>> <
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the details
> > > >>>>>>>>> Justine!
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side change
> > > >>>>>>> for
> > > >>>>>>>> 2
> > > >>>>>>>>> is
> > > >>>>>>>>>>>>>>> removing
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>> addPartitions
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need to
> > > >>>>>>> make
> > > >>>>>>>>>> this
> > > >>>>>>>>>>>> from
> > > >>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> producer
> > > >>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think I did not
> > > >>>>>> express
> > > >>>>>>>>> myself
> > > >>>>>>>>>>>>>>> clearly. I
> > > >>>>>>>>>>>>>>>>>>>>> understand
> > > >>>>>>>>>>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (and
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should) change the
> > > >>>>>>> producer
> > > >>>>>>>> to
> > > >>>>>>>>>> not
> > > >>>>>>>>>>>> send
> > > >>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> longer. But I don't
> > > >>>>>> thinks
> > > >>>>>>>>> it's
> > > >>>>>>>>>>>>>>> requirement
> > > >>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>> change
> > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> broker?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What I am trying to say
> > > >>>>>>> is:
> > > >>>>>>>>> as a
> > > >>>>>>>>>>>>>>> safe-guard
> > > >>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>> improvement
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producers, the partition
> > > >>>>>>>>> leader
> > > >>>>>>>>>> can
> > > >>>>>>>>>>>> just
> > > >>>>>>>>>>>>>>>> send
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request to the
> > > >>>>>>>> TX-coordinator
> > > >>>>>>>>>> in any
> > > >>>>>>>>>>>>>>> case
> > > >>>>>>>>>>>>>>>> --
> > > >>>>>>>>>>>>>>>>> if
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> old
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> correctly did send the
> > > >>>>>>>>>>>> `addPartition`
> > > >>>>>>>>>>>>>>>> request
> > > >>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> already, the
> > > >>>>>>> TX-coordinator
> > > >>>>>>>>> can
> > > >>>>>>>>>> just
> > > >>>>>>>>>>>>>>>> "ignore"
> > > >>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>> as
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> idempotent.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if the old producer has
> > > >>>>>> a
> > > >>>>>>>> bug
> > > >>>>>>>>>> and
> > > >>>>>>>>>>>> did
> > > >>>>>>>>>>>>>>>> forget
> > > >>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>> sent
> > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `addPartition`
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request, we would now
> > > >>>>>>> ensure
> > > >>>>>>>>>> that
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>> partition
> > > >>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>> indeed
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> added
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX and thus fix a
> > > >>>>>>> potential
> > > >>>>>>>>>>>> producer bug
> > > >>>>>>>>>>>>>>>>> (even
> > > >>>>>>>>>>>>>>>>>>>> if we
> > > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> get
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fencing via the bump
> > > >>>>>>> epoch).
> > > >>>>>>>>> --
> > > >>>>>>>>>> It
> > > >>>>>>>>>>>>>>> seems to
> > > >>>>>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>>>>> good
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Or
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there a reason to not do
> > > >>>>>>>> this?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is ongoing
> > > >>>>>> =
> > > >>>>>>>>>> partition
> > > >>>>>>>>>>>> was
> > > >>>>>>>>>>>>>>>> added
> > > >>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> via
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn. We
> > > >>>>>>>> check
> > > >>>>>>>>>> this
> > > >>>>>>>>>>>> with
> > > >>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this wasn't
> > > >>>>>>>>>> sufficiently
> > > >>>>>>>>>>>>>>>>> explained
> > > >>>>>>>>>>>>>>>>>>>> here:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do what I propose
> > > >>>>>> in
> > > >>>>>>>>>> (20), we
> > > >>>>>>>>>>>>>>> don't
> > > >>>>>>>>>>>>>>>>>> really
> > > >>>>>>>>>>>>>>>>>>>>> need
> > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>> make
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `DescribeTransaction`
> > > >>>>>>> call,
> > > >>>>>>>> as
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>>>> partition
> > > >>>>>>>>>>>>>>>>>>>> leader
> > > >>>>>>>>>>>>>>>>>>>>>>> adds
> > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for older clients and we
> > > >>>>>>> get
> > > >>>>>>>>>> this
> > > >>>>>>>>>>>> check
> > > >>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>> free.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is that
> > > >>>>>> if
> > > >>>>>>>> any
> > > >>>>>>>>>>>> messages
> > > >>>>>>>>>>>>>>>>> somehow
> > > >>>>>>>>>>>>>>>>>>>> come
> > > >>>>>>>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>>>>>>> before
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the producer,
> > > >>>>>>> they
> > > >>>>>>>>>> will be
> > > >>>>>>>>>>>>>>>> fenced.
> > > >>>>>>>>>>>>>>>>>>>>> However,
> > > >>>>>>>>>>>>>>>>>>>>>>> if
> > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it can be
> > > >>>>>>>>>> discussed
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree that we should
> > > >>>>>>> have
> > > >>>>>>>>>> epoch
> > > >>>>>>>>>>>>>>> fencing.
> > > >>>>>>>>>>>>>>>> My
> > > >>>>>>>>>>>>>>>>>>>>>>> question is
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> different:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Assume we are at epoch
> > > >>>>>> 73,
> > > >>>>>>>> and
> > > >>>>>>>>>> we
> > > >>>>>>>>>>>> have
> > > >>>>>>>>>>>>>>> an
> > > >>>>>>>>>>>>>>>>>> ongoing
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed. It seems
> > > >>>>>>> natural
> > > >>>>>>>> to
> > > >>>>>>>>>>>> write the
> > > >>>>>>>>>>>>>>>>>> "prepare
> > > >>>>>>>>>>>>>>>>>>>>>>> commit"
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> marker
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
> > > >>>>>>> both
> > > >>>>>>>>> with
> > > >>>>>>>>>>>> epoch
> > > >>>>>>>>>>>>>>> 73,
> > > >>>>>>>>>>>>>>>>> too,
> > > >>>>>>>>>>>>>>>>>>>> as
> > > >>>>>>>>>>>>>>>>>>>>> it
> > > >>>>>>>>>>>>>>>>>>>>>>>>> belongs
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current transaction. Of
> > > >>>>>>>>> course,
> > > >>>>>>>>>> we
> > > >>>>>>>>>>>> now
> > > >>>>>>>>>>>>>>> also
> > > >>>>>>>>>>>>>>>>>> bump
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> expect
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the next requests to
> > > >>>>>> have
> > > >>>>>>>>> epoch
> > > >>>>>>>>>> 74,
> > > >>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>>>>> reject
> > > >>>>>>>>>>>>>>>>>>>>>>> an
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch 73, as the
> > > >>>>>>>> corresponding
> > > >>>>>>>>>> TX
> > > >>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>> epoch
> > > >>>>>>>>>>>>>>>>> 73
> > > >>>>>>>>>>>>>>>>>>>> was
> > > >>>>>>>>>>>>>>>>>>>>>>>> already
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems you propose to
> > > >>>>>>>> write
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>>> "prepare
> > > >>>>>>>>>>>>>>>>>> commit
> > > >>>>>>>>>>>>>>>>>>>>>>> marker"
> > > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
> > > >>>>>>> with
> > > >>>>>>>>>> epoch 74
> > > >>>>>>>>>>>>>>>> though,
> > > >>>>>>>>>>>>>>>>>> what
> > > >>>>>>>>>>>>>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>>>>>>>>>> work,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems confusing. Is
> > > >>>>>> there
> > > >>>>>>> a
> > > >>>>>>>>>> reason
> > > >>>>>>>>>>>> why
> > > >>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>>>>> use
> > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> bumped
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 74
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of the current
> > > >>>>>>> epoch
> > > >>>>>>>>> 73?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are checking if
> > > >>>>>>> the
> > > >>>>>>>>>>>>>>> transaction is
> > > >>>>>>>>>>>>>>>>>>>> ongoing,
> > > >>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>> need
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the leader
> > > >>>>>>>>> partition
> > > >>>>>>>>>> to
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for this
> > > >>>>>>>>>> message to
> > > >>>>>>>>>>>> come
> > > >>>>>>>>>>>>>>>>> back,
> > > >>>>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>>>>> theory
> > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
> > > >>>>>> that
> > > >>>>>>>>> would
> > > >>>>>>>>>>>> make the
> > > >>>>>>>>>>>>>>>>>> original
> > > >>>>>>>>>>>>>>>>>>>>>>> result
> > > >>>>>>>>>>>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out of
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why we
> > > >>>>>> can
> > > >>>>>>>>> check
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>>> leader
> > > >>>>>>>>>>>>>>>>>> state
> > > >>>>>>>>>>>>>>>>>>>>>>> before
> > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks. Got it.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, is this really
> > > >>>>>> an
> > > >>>>>>>>>> issue?
> > > >>>>>>>>>>>> We put
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> produce
> > > >>>>>>>>>>>>>>>>>>>>>>>>> request
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory, so how could
> > > >>>>>> we
> > > >>>>>>>>>> process
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> first?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Don't we need to put the
> > > >>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
> > > >>>>>>>>>>>>>>>>>>>>> into
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> too,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for this case, and
> > > >>>>>> process
> > > >>>>>>>>> both
> > > >>>>>>>>>>>> request
> > > >>>>>>>>>>>>>>>>>> in-order?
> > > >>>>>>>>>>>>>>>>>>>>>>> (Again,
> > > >>>>>>>>>>>>>>>>>>>>>>>>> my
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> knowledge is limited and
> > > >>>>>>>> maybe
> > > >>>>>>>>>> we
> > > >>>>>>>>>>>> don't
> > > >>>>>>>>>>>>>>>>>> maintain
> > > >>>>>>>>>>>>>>>>>>>>>>> request
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> order
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case, what seems to be
> > > >>>>>> an
> > > >>>>>>>>> issue
> > > >>>>>>>>>>>> IMHO,
> > > >>>>>>>>>>>>>>> and I
> > > >>>>>>>>>>>>>>>>> am
> > > >>>>>>>>>>>>>>>>>>>>>>> wondering
> > > >>>>>>>>>>>>>>>>>>>>>>>> if
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> changing
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request handling to
> > > >>>>>>> preserve
> > > >>>>>>>>>> order
> > > >>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>> case
> > > >>>>>>>>>>>>>>>>>>>>>>> might be
> > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> cleaner
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution?)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/30/22 3:28 PM,
> > > >>>>>> Artem
> > > >>>>>>>>>> Livshits
> > > >>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think the interesting
> > > >>>>>>>> part
> > > >>>>>>>>> is
> > > >>>>>>>>>>>> not in
> > > >>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>> logic
> > > >>>>>>>>>>>>>>>>>>>>>>>> (because
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> tries
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> figure out when
> > > >>>>>>>>>>>> UNKNOWN_PRODUCER_ID is
> > > >>>>>>>>>>>>>>>>>> retriable
> > > >>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>> if
> > > >>>>>>>>>>>>>>>>>>>>>>>>> it's
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retryable,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's definitely not
> > > >>>>>>> fatal),
> > > >>>>>>>>> but
> > > >>>>>>>>>>>> what
> > > >>>>>>>>>>>>>>>> happens
> > > >>>>>>>>>>>>>>>>>>>> when
> > > >>>>>>>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>>>> logic
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'true' and falls
> > > >>>>>> through.
> > > >>>>>>>> In
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>> old
> > > >>>>>>>>>>>>>>>>> clients
> > > >>>>>>>>>>>>>>>>>> it
> > > >>>>>>>>>>>>>>>>>>>>>>> seems
> > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> keep the behavior in
> > > >>>>>> the
> > > >>>>>>>> new
> > > >>>>>>>>>>>> clients,
> > > >>>>>>>>>>>>>>> I'd
> > > >>>>>>>>>>>>>>>>>>>> expect it
> > > >>>>>>>>>>>>>>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> well.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 29, 2022 at
> > > >>>>>>>> 11:57
> > > >>>>>>>>>> AM
> > > >>>>>>>>>>>>>>> Justine
> > > >>>>>>>>>>>>>>>>>> Olshan
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>> <jolshan@confluent.io.invalid
> > > >>>>>>>>>>
> > > >>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Artem and Jeff,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
> > > >>>>>> look
> > > >>>>>>>> and
> > > >>>>>>>>>>>> sorry for
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>> slow
> > > >>>>>>>>>>>>>>>>>>>>>>>> response.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You both mentioned the
> > > >>>>>>>>> change
> > > >>>>>>>>>> to
> > > >>>>>>>>>>>>>>> handle
> > > >>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear — this error
> > > >>>>>> code
> > > >>>>>>>> will
> > > >>>>>>>>>> only
> > > >>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>> sent
> > > >>>>>>>>>>>>>>>>>> again
> > > >>>>>>>>>>>>>>>>>>>>> when
> > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> client's
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version is high enough
> > > >>>>>>> to
> > > >>>>>>>>>> ensure
> > > >>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>> handle
> > > >>>>>>>>>>>>>>>>> it
> > > >>>>>>>>>>>>>>>>>>>>>>>> correctly.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The current (Java)
> > > >>>>>>> client
> > > >>>>>>>>>> handles
> > > >>>>>>>>>>>>>>> this by
> > > >>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> following
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> (somewhat
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> long)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code snippet:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // An
> > > >>>>>>> UNKNOWN_PRODUCER_ID
> > > >>>>>>>>>> means
> > > >>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>> have
> > > >>>>>>>>>>>>>>>>>>>> lost
> > > >>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker. Depending on
> > > >>>>>> the
> > > >>>>>>>> log
> > > >>>>>>>>>> start
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // offset, we may want
> > > >>>>>>> to
> > > >>>>>>>>>> retry
> > > >>>>>>>>>>>>>>> these, as
> > > >>>>>>>>>>>>>>>>>>>>> described
> > > >>>>>>>>>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> each
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> case
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below. If
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> none of those apply,
> > > >>>>>>> then
> > > >>>>>>>>> for
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // idempotent
> > > >>>>>> producer,
> > > >>>>>>> we
> > > >>>>>>>>>> will
> > > >>>>>>>>>>>>>>> locally
> > > >>>>>>>>>>>>>>>>> bump
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> reset
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence numbers of
> > > >>>>>>>>> in-flight
> > > >>>>>>>>>>>> batches
> > > >>>>>>>>>>>>>>>> from
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // sequence 0, then
> > > >>>>>>> retry
> > > >>>>>>>>> the
> > > >>>>>>>>>>>> failed
> > > >>>>>>>>>>>>>>>> batch,
> > > >>>>>>>>>>>>>>>>>>>> which
> > > >>>>>>>>>>>>>>>>>>>>>>>> should
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> now
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> succeed.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the transactional
> > > >>>>>>>> producer,
> > > >>>>>>>>>> allow
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // batch to fail. When
> > > >>>>>>>>>> processing
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>> failed
> > > >>>>>>>>>>>>>>>>>>>>> batch,
> > > >>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transition
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an abortable error and
> > > >>>>>>>> set a
> > > >>>>>>>>>> flag
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // indicating that we
> > > >>>>>>> need
> > > >>>>>>>>> to
> > > >>>>>>>>>>>> bump the
> > > >>>>>>>>>>>>>>>>> epoch
> > > >>>>>>>>>>>>>>>>>>>> (if
> > > >>>>>>>>>>>>>>>>>>>>>>>>> supported
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> by
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker).
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if (error ==
> > > >>>>>>>>>>>>>>>> Errors.*UNKNOWN_PRODUCER_ID*)
> > > >>>>>>>>>>>>>>>>> {
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > > >>>>>>>>>> (response.logStartOffset
> > > >>>>>>>>>>>> ==
> > > >>>>>>>>>>>>>>> -1)
> > > >>>>>>>>>>>>>>>> {
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // We don't
> > > >>>>>>> know
> > > >>>>>>>>>> the log
> > > >>>>>>>>>>>>>>> start
> > > >>>>>>>>>>>>>>>>>> offset
> > > >>>>>>>>>>>>>>>>>>>>> with
> > > >>>>>>>>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> response.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the request
> > > >>>>>>>> until
> > > >>>>>>>>>> we
> > > >>>>>>>>>>>> get
> > > >>>>>>>>>>>>>>> it.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // The
> > > >>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > >>>>>>>>>>>>>>>> error
> > > >>>>>>>>>>>>>>>>>> code
> > > >>>>>>>>>>>>>>>>>>>>> was
> > > >>>>>>>>>>>>>>>>>>>>>>>> added
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> along
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProduceResponse which
> > > >>>>>>>>>> includes the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              //
> > > >>>>>>>> logStartOffset.
> > > >>>>>>>>>> So
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>> '-1'
> > > >>>>>>>>>>>>>>>>>>>> sentinel
> > > >>>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> backward
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatibility.
> > > >>>>>> Instead,
> > > >>>>>>> it
> > > >>>>>>>>> is
> > > >>>>>>>>>>>> possible
> > > >>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // a broker
> > > >>>>>> to
> > > >>>>>>>> not
> > > >>>>>>>>>> know
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> logStartOffset at
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> when
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returning
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response because
> > > >>>>>> the
> > > >>>>>>>>>> partition
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // may have
> > > >>>>>>>> moved
> > > >>>>>>>>>> away
> > > >>>>>>>>>>>> from
> > > >>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> broker
> > > >>>>>>>>>>>>>>>>>>>>>>> from
> > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error was
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initially raised to
> > > >>>>>> the
> > > >>>>>>>> time
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // response
> > > >>>>>>> was
> > > >>>>>>>>>> being
> > > >>>>>>>>>>>>>>>>> constructed.
> > > >>>>>>>>>>>>>>>>>> In
> > > >>>>>>>>>>>>>>>>>>>>>>> these
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> cases,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retry the request: we
> > > >>>>>>> are
> > > >>>>>>>>>>>> guaranteed
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // to
> > > >>>>>>> eventually
> > > >>>>>>>>>> get a
> > > >>>>>>>>>>>>>>>>>> logStartOffset
> > > >>>>>>>>>>>>>>>>>>>>> once
> > > >>>>>>>>>>>>>>>>>>>>>>>>> things
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> settle
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> down.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > > >>>>>>>>>>>>>>> (batch.sequenceHasBeenReset()) {
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // When the
> > > >>>>>>>> first
> > > >>>>>>>>>>>> inflight
> > > >>>>>>>>>>>>>>>> batch
> > > >>>>>>>>>>>>>>>>>>>> fails
> > > >>>>>>>>>>>>>>>>>>>>>>> due to
> > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> truncation
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> then the sequences of
> > > >>>>>>> all
> > > >>>>>>>>> the
> > > >>>>>>>>>>>> other
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // in flight
> > > >>>>>>>>> batches
> > > >>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>> have
> > > >>>>>>>>>>>>>>>>>> been
> > > >>>>>>>>>>>>>>>>>>>>>>>> restarted
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> from
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> beginning.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, when those
> > > >>>>>>>>> responses
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // come back
> > > >>>>>>>> from
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>>> broker,
> > > >>>>>>>>>>>>>>>>> they
> > > >>>>>>>>>>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>>>>>>>> also
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> come
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > >>>>>>> error.
> > > >>>>>>>>> In
> > > >>>>>>>>>> this
> > > >>>>>>>>>>>>>>> case,
> > > >>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>> should
> > > >>>>>>>>>>>>>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // reset the
> > > >>>>>>>>>> sequence
> > > >>>>>>>>>>>>>>> numbers
> > > >>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>> beginning.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          } else if
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>> (lastAckedOffset(batch.topicPartition).orElse(
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> response.logStartOffset) {
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // The head
> > > >>>>>> of
> > > >>>>>>>> the
> > > >>>>>>>>>> log
> > > >>>>>>>>>>>> has
> > > >>>>>>>>>>>>>>> been
> > > >>>>>>>>>>>>>>>>>>>> removed,
> > > >>>>>>>>>>>>>>>>>>>>>>>>> probably
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> due
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retention time
> > > >>>>>> elapsing.
> > > >>>>>>>> In
> > > >>>>>>>>>> this
> > > >>>>>>>>>>>> case,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // we expect
> > > >>>>>>> to
> > > >>>>>>>>>> lose the
> > > >>>>>>>>>>>>>>>> producer
> > > >>>>>>>>>>>>>>>>>>>> state.
> > > >>>>>>>>>>>>>>>>>>>>>>> For
> > > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer, reset the
> > > >>>>>>>>> sequences
> > > >>>>>>>>>> of
> > > >>>>>>>>>>>> all
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // inflight
> > > >>>>>>>>> batches
> > > >>>>>>>>>> to
> > > >>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>> from
> > > >>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> beginning
> > > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> retry
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the transaction
> > > >>>>>>> does
> > > >>>>>>>>> not
> > > >>>>>>>>>>>> need to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // be
> > > >>>>>> aborted.
> > > >>>>>>>> For
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>>>> idempotent
> > > >>>>>>>>>>>>>>>>>>>>>>> producer,
> > > >>>>>>>>>>>>>>>>>>>>>>>>> bump
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> avoid
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reusing (sequence,
> > > >>>>>>> epoch)
> > > >>>>>>>>>> pairs
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              if
> > > >>>>>>>>>> (isTransactional()) {
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>
> > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>> this.producerIdAndEpoch);
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              } else {
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> requestEpochBumpForPartition(batch.topicPartition);
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              }
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > > >>>>>>>>> (!isTransactional())
> > > >>>>>>>>>> {
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // For the
> > > >>>>>>>>>> idempotent
> > > >>>>>>>>>>>>>>> producer,
> > > >>>>>>>>>>>>>>>>>>>> always
> > > >>>>>>>>>>>>>>>>>>>>>>> retry
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors. If the batch
> > > >>>>>> has
> > > >>>>>>>> the
> > > >>>>>>>>>>>> current
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // producer
> > > >>>>>> ID
> > > >>>>>>>> and
> > > >>>>>>>>>>>> epoch,
> > > >>>>>>>>>>>>>>>>> request a
> > > >>>>>>>>>>>>>>>>>>>> bump
> > > >>>>>>>>>>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Otherwise
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the
> > > >>>>>> produce.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>> requestEpochBumpForPartition(batch.topicPartition);
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I was considering
> > > >>>>>>> keeping
> > > >>>>>>>>> this
> > > >>>>>>>>>>>>>>> behavior —
> > > >>>>>>>>>>>>>>>>> but
> > > >>>>>>>>>>>>>>>>>>>> am
> > > >>>>>>>>>>>>>>>>>>>>>>> open
> > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simplifying
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We are leaving changes
> > > >>>>>>> to
> > > >>>>>>>>>> older
> > > >>>>>>>>>>>>>>> clients
> > > >>>>>>>>>>>>>>>> off
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> table
> > > >>>>>>>>>>>>>>>>>>>>>>>>> here
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> since
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> caused
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many issues for
> > > >>>>>> clients
> > > >>>>>>> in
> > > >>>>>>>>> the
> > > >>>>>>>>>>>> past.
> > > >>>>>>>>>>>>>>>>>> Previously
> > > >>>>>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>>> was
> > > >>>>>>>>>>>>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we didn't have the
> > > >>>>>>>>> mechanisms
> > > >>>>>>>>>> in
> > > >>>>>>>>>>>>>>> place to
> > > >>>>>>>>>>>>>>>>>>>> detect
> > > >>>>>>>>>>>>>>>>>>>>>>> when
> > > >>>>>>>>>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> was
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> legitimate
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case vs some bug or
> > > >>>>>> gap
> > > >>>>>>> in
> > > >>>>>>>>> the
> > > >>>>>>>>>>>>>>> protocol.
> > > >>>>>>>>>>>>>>>>>>>> Ensuring
> > > >>>>>>>>>>>>>>>>>>>>>>> each
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> its
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> own epoch should close
> > > >>>>>>>> this
> > > >>>>>>>>>> gap.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> And to address Jeff's
> > > >>>>>>>> second
> > > >>>>>>>>>>>> point:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *does the typical
> > > >>>>>>> produce
> > > >>>>>>>>>> request
> > > >>>>>>>>>>>> path
> > > >>>>>>>>>>>>>>>>> append
> > > >>>>>>>>>>>>>>>>>>>>>>> records
> > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> local
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> log
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along*
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *with the
> > > >>>>>>>>>> currentTxnFirstOffset
> > > >>>>>>>>>>>>>>>>> information?
> > > >>>>>>>>>>>>>>>>>> I
> > > >>>>>>>>>>>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>>>>>>>>> like
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand*
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *when the field is
> > > >>>>>>> written
> > > >>>>>>>>> to
> > > >>>>>>>>>>>> disk.*
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, the first produce
> > > >>>>>>>>> request
> > > >>>>>>>>>>>>>>> populates
> > > >>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>> field
> > > >>>>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> writes
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> offset
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as part of the record
> > > >>>>>>>> batch
> > > >>>>>>>>>> and
> > > >>>>>>>>>>>> also
> > > >>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>> producer
> > > >>>>>>>>>>>>>>>>>>>>>>>>> state
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> snapshot.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we reload the records
> > > >>>>>> on
> > > >>>>>>>>>> restart
> > > >>>>>>>>>>>>>>> and/or
> > > >>>>>>>>>>>>>>>>>>>>>>> reassignment,
> > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> repopulate
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> field with the
> > > >>>>>> snapshot
> > > >>>>>>>> from
> > > >>>>>>>>>> disk
> > > >>>>>>>>>>>>>>> along
> > > >>>>>>>>>>>>>>>>> with
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> rest
> > > >>>>>>>>>>>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if there
> > > >>>>>> are
> > > >>>>>>>>>> further
> > > >>>>>>>>>>>>>>> comments
> > > >>>>>>>>>>>>>>>>>>>> and/or
> > > >>>>>>>>>>>>>>>>>>>>>>>>> questions.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > > >>>>>> at
> > > >>>>>>>> 9:00
> > > >>>>>>>>>> PM
> > > >>>>>>>>>>>> Jeff
> > > >>>>>>>>>>>>>>> Kim
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>> <jeff.kim@confluent.io.invalid
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP! I
> > > >>>>>>>> have
> > > >>>>>>>>>> two
> > > >>>>>>>>>>>>>>>> questions:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1) For new clients,
> > > >>>>>> we
> > > >>>>>>>> can
> > > >>>>>>>>>> once
> > > >>>>>>>>>>>> again
> > > >>>>>>>>>>>>>>>>> return
> > > >>>>>>>>>>>>>>>>>>>> an
> > > >>>>>>>>>>>>>>>>>>>>>>> error
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for sequences
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that are non-zero
> > > >>>>>> when
> > > >>>>>>>>> there
> > > >>>>>>>>>> is
> > > >>>>>>>>>>>> no
> > > >>>>>>>>>>>>>>>>> producer
> > > >>>>>>>>>>>>>>>>>>>> state
> > > >>>>>>>>>>>>>>>>>>>>>>>>> present
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> on
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This will indicate we
> > > >>>>>>>>> missed
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>> 0
> > > >>>>>>>>>>>>>>>>> sequence
> > > >>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> yet
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> want
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the log.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would like to
> > > >>>>>>>> understand
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>>> current
> > > >>>>>>>>>>>>>>>>>>>> behavior
> > > >>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>> handle
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and if there are any
> > > >>>>>>>>> changes
> > > >>>>>>>>>> we
> > > >>>>>>>>>>>> are
> > > >>>>>>>>>>>>>>>>> making.
> > > >>>>>>>>>>>>>>>>>>>> Maybe
> > > >>>>>>>>>>>>>>>>>>>>>>> I'm
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> missing
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but we would want to
> > > >>>>>>>>> identify
> > > >>>>>>>>>>>>>>> whether we
> > > >>>>>>>>>>>>>>>>>>>> missed
> > > >>>>>>>>>>>>>>>>>>>>>>> the 0
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> sequence
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients, no?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2) Upon returning
> > > >>>>>> from
> > > >>>>>>>> the
> > > >>>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>>>>> coordinator, we
> > > >>>>>>>>>>>>>>>>>>>>>>>>> can
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> set
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as ongoing on the
> > > >>>>>>> leader
> > > >>>>>>>> by
> > > >>>>>>>>>>>>>>> populating
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> currentTxnFirstOffset
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through the typical
> > > >>>>>>>> produce
> > > >>>>>>>>>>>> request
> > > >>>>>>>>>>>>>>>>>> handling.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does the typical
> > > >>>>>>> produce
> > > >>>>>>>>>> request
> > > >>>>>>>>>>>> path
> > > >>>>>>>>>>>>>>>>> append
> > > >>>>>>>>>>>>>>>>>>>>>>> records
> > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> local
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> log
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> > > >>>>>>>>>> currentTxnFirstOffset
> > > >>>>>>>>>>>>>>>>> information?
> > > >>>>>>>>>>>>>>>>>> I
> > > >>>>>>>>>>>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>>>>>>>>> like
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when the field is
> > > >>>>>>> written
> > > >>>>>>>>> to
> > > >>>>>>>>>>>> disk.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeff
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > > >>>>>> at
> > > >>>>>>>>> 4:44
> > > >>>>>>>>>> PM
> > > >>>>>>>>>>>> Artem
> > > >>>>>>>>>>>>>>>>>> Livshits
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
> > > >>>>>>> alivshits@confluent.io
> > > >>>>>>>>>> .invalid>
> > > >>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for the
> > > >>>>>> KIP.
> > > >>>>>>>> I
> > > >>>>>>>>>> have
> > > >>>>>>>>>>>> one
> > > >>>>>>>>>>>>>>>>>> question.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 5) For new clients,
> > > >>>>>> we
> > > >>>>>>>> can
> > > >>>>>>>>>> once
> > > >>>>>>>>>>>>>>> again
> > > >>>>>>>>>>>>>>>>>> return
> > > >>>>>>>>>>>>>>>>>>>> an
> > > >>>>>>>>>>>>>>>>>>>>>>> error
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe we had
> > > >>>>>>>> problems
> > > >>>>>>>>>> in the
> > > >>>>>>>>>>>>>>> past
> > > >>>>>>>>>>>>>>>>> with
> > > >>>>>>>>>>>>>>>>>>>>>>> returning
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because it was
> > > >>>>>>>> considered
> > > >>>>>>>>>> fatal
> > > >>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>> required
> > > >>>>>>>>>>>>>>>>>>>>>>> client
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> restart.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> It
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good to spell out
> > > >>>>>> the
> > > >>>>>>>> new
> > > >>>>>>>>>> client
> > > >>>>>>>>>>>>>>>> behavior
> > > >>>>>>>>>>>>>>>>>>>> when
> > > >>>>>>>>>>>>>>>>>>>>> it
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> receives
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > > >>>>>>> at
> > > >>>>>>>>>> 10:00 AM
> > > >>>>>>>>>>>>>>>> Justine
> > > >>>>>>>>>>>>>>>>>>>> Olshan
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>> <jo...@confluent.io.invalid>
> > > >>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
> > > >>>>>>>> look
> > > >>>>>>>>>>>> Matthias.
> > > >>>>>>>>>>>>>>>> I've
> > > >>>>>>>>>>>>>>>>>>>> tried
> > > >>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>> answer
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> your
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> questions
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 10)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Right — so the
> > > >>>>>>> hanging
> > > >>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>> only
> > > >>>>>>>>>>>>>>>>>>>> occurs
> > > >>>>>>>>>>>>>>>>>>>>>>> when
> > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> come in and the
> > > >>>>>>>> partition
> > > >>>>>>>>>> is
> > > >>>>>>>>>>>> never
> > > >>>>>>>>>>>>>>>> added
> > > >>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> again.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never add the
> > > >>>>>>> partition
> > > >>>>>>>>> to
> > > >>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>> transaction,
> > > >>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>>>>>>>>>>>> never
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> marker
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never advance the
> > > >>>>>>> LSO.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do end up
> > > >>>>>>> adding
> > > >>>>>>>>> the
> > > >>>>>>>>>>>>>>> partition
> > > >>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> (I
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suppose
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happen before or
> > > >>>>>>> after
> > > >>>>>>>>> the
> > > >>>>>>>>>> late
> > > >>>>>>>>>>>>>>>> message
> > > >>>>>>>>>>>>>>>>>>>> comes
> > > >>>>>>>>>>>>>>>>>>>>> in)
> > > >>>>>>>>>>>>>>>>>>>>>>>> then
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> include
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late message in the
> > > >>>>>>>> next
> > > >>>>>>>>>>>>>>> (incorrect)
> > > >>>>>>>>>>>>>>>>>>>>> transaction.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> So perhaps it is
> > > >>>>>>>> clearer
> > > >>>>>>>>> to
> > > >>>>>>>>>>>> make
> > > >>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>> distinction
> > > >>>>>>>>>>>>>>>>>>>>>>>>> between
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> eventually get
> > > >>>>>> added
> > > >>>>>>> to
> > > >>>>>>>>> the
> > > >>>>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>> (but
> > > >>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>> wrong
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> one)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> or
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that never get
> > > >>>>>> added
> > > >>>>>>>> and
> > > >>>>>>>>>> become
> > > >>>>>>>>>>>>>>>> hanging.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side
> > > >>>>>>> change
> > > >>>>>>>>> for
> > > >>>>>>>>>> 2 is
> > > >>>>>>>>>>>>>>>> removing
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> addPartitions
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need
> > > >>>>>>> to
> > > >>>>>>>>> make
> > > >>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>> from
> > > >>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> producer
> > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In my opinion, the
> > > >>>>>>>> issue
> > > >>>>>>>>>> with
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> call
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is that we don't
> > > >>>>>> have
> > > >>>>>>>> the
> > > >>>>>>>>>> epoch
> > > >>>>>>>>>>>>>>> bump,
> > > >>>>>>>>>>>>>>>> so
> > > >>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>> don't
> > > >>>>>>>>>>>>>>>>>>>>>>>> know
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> belongs to the
> > > >>>>>>> previous
> > > >>>>>>>>>>>>>>> transaction or
> > > >>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>> one.
> > > >>>>>>>>>>>>>>>>>>>>>>> We
> > > >>>>>>>>>>>>>>>>>>>>>>>>> need
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition has been
> > > >>>>>>>> added
> > > >>>>>>>>> to
> > > >>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>> transaction.
> > > >>>>>>>>>>>>>>>>>>>>> Of
> > > >>>>>>>>>>>>>>>>>>>>>>>>> course,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> means
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't completely
> > > >>>>>>> cover
> > > >>>>>>>>> the
> > > >>>>>>>>>> case
> > > >>>>>>>>>>>>>>> where
> > > >>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>> have a
> > > >>>>>>>>>>>>>>>>>>>>>>>> really
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have added the
> > > >>>>>>>> partition
> > > >>>>>>>>> to
> > > >>>>>>>>>>>> the new
> > > >>>>>>>>>>>>>>>>>>>>> transaction,
> > > >>>>>>>>>>>>>>>>>>>>>>> but
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> that's
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something we will
> > > >>>>>>> need
> > > >>>>>>>>> the
> > > >>>>>>>>>> new
> > > >>>>>>>>>>>>>>> clients
> > > >>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>> cover.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is
> > > >>>>>>> ongoing
> > > >>>>>>>> =
> > > >>>>>>>>>>>> partition
> > > >>>>>>>>>>>>>>> was
> > > >>>>>>>>>>>>>>>>>>>> added to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> via
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn.
> > > >>>>>>> We
> > > >>>>>>>>>> check
> > > >>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>> with
> > > >>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this
> > > >>>>>>> wasn't
> > > >>>>>>>>>>>> sufficiently
> > > >>>>>>>>>>>>>>>>>>>> explained
> > > >>>>>>>>>>>>>>>>>>>>>>> here:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>
> > > >>>>>>
> > > >>>>
> > > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is
> > > >>>>>> that
> > > >>>>>>>> if
> > > >>>>>>>>>> any
> > > >>>>>>>>>>>>>>> messages
> > > >>>>>>>>>>>>>>>>>>>> somehow
> > > >>>>>>>>>>>>>>>>>>>>>>> come
> > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> before
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the
> > > >>>>>>> producer,
> > > >>>>>>>>> they
> > > >>>>>>>>>>>> will be
> > > >>>>>>>>>>>>>>>>>> fenced.
> > > >>>>>>>>>>>>>>>>>>>>>>> However,
> > > >>>>>>>>>>>>>>>>>>>>>>>>> if
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it
> > > >>>>>> can
> > > >>>>>>> be
> > > >>>>>>>>>>>> discussed
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 50)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It should be
> > > >>>>>>>> synchronous
> > > >>>>>>>>>>>> because
> > > >>>>>>>>>>>>>>> if we
> > > >>>>>>>>>>>>>>>>>> have
> > > >>>>>>>>>>>>>>>>>>>> an
> > > >>>>>>>>>>>>>>>>>>>>>>> event
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> (ie,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> an
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes us to need
> > > >>>>>> to
> > > >>>>>>>>> abort
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>> transaction,
> > > >>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>> need
> > > >>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions to send
> > > >>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>> markers
> > > >>>>>>>>>>>>>>>>> to.
> > > >>>>>>>>>>>>>>>>>>>> We
> > > >>>>>>>>>>>>>>>>>>>>>>> know
> > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we added them to
> > > >>>>>> the
> > > >>>>>>>>>>>> coordinator
> > > >>>>>>>>>>>>>>> via
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Previously we have
> > > >>>>>>> had
> > > >>>>>>>>>>>> asynchronous
> > > >>>>>>>>>>>>>>>>> calls
> > > >>>>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>> past
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> (ie,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit markers when
> > > >>>>>>> the
> > > >>>>>>>>>>>>>>> transaction is
> > > >>>>>>>>>>>>>>>>>>>>> completed)
> > > >>>>>>>>>>>>>>>>>>>>>>>> but
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> often
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes confusion as
> > > >>>>>>> we
> > > >>>>>>>>>> need to
> > > >>>>>>>>>>>> wait
> > > >>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>> some
> > > >>>>>>>>>>>>>>>>>>>>>>>>> operations
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complete.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing commit
> > > >>>>>>> markers
> > > >>>>>>>>>> case,
> > > >>>>>>>>>>>>>>> clients
> > > >>>>>>>>>>>>>>>>> often
> > > >>>>>>>>>>>>>>>>>>>> see
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>> CONCURRENT_TRANSACTIONs
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error messages and
> > > >>>>>>> that
> > > >>>>>>>>>> can be
> > > >>>>>>>>>>>>>>>>> confusing.
> > > >>>>>>>>>>>>>>>>>>>> For
> > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> reason,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> may
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simpler to just
> > > >>>>>> have
> > > >>>>>>>>>>>> synchronous
> > > >>>>>>>>>>>>>>>> calls —
> > > >>>>>>>>>>>>>>>>>>>>>>> especially
> > > >>>>>>>>>>>>>>>>>>>>>>>> if
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> need
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> block
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some operation's
> > > >>>>>>>>> completion
> > > >>>>>>>>>>>> anyway
> > > >>>>>>>>>>>>>>>>> before
> > > >>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>> can
> > > >>>>>>>>>>>>>>>>>>>>>>>> start
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> next
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. And
> > > >>>>>>> yes, I
> > > >>>>>>>>>> meant
> > > >>>>>>>>>>>>>>>>>> coordinator. I
> > > >>>>>>>>>>>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>>>>>>>>>>> fix
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> that.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are
> > > >>>>>> checking
> > > >>>>>>> if
> > > >>>>>>>>> the
> > > >>>>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>> ongoing,
> > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> need
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the
> > > >>>>>> leader
> > > >>>>>>>>>> partition
> > > >>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for
> > > >>>>>>> this
> > > >>>>>>>>>>>> message to
> > > >>>>>>>>>>>>>>>> come
> > > >>>>>>>>>>>>>>>>>>>> back,
> > > >>>>>>>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>>>>>>> theory
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
> > > >>>>>>>> that
> > > >>>>>>>>>> would
> > > >>>>>>>>>>>> make
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>> original
> > > >>>>>>>>>>>>>>>>>>>>>>>>> result
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why
> > > >>>>>> we
> > > >>>>>>>> can
> > > >>>>>>>>>> check
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>> leader
> > > >>>>>>>>>>>>>>>>>>>> state
> > > >>>>>>>>>>>>>>>>>>>>>>>> before
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm happy to update
> > > >>>>>>> the
> > > >>>>>>>>>> KIP if
> > > >>>>>>>>>>>>>>> some of
> > > >>>>>>>>>>>>>>>>>> these
> > > >>>>>>>>>>>>>>>>>>>>>>> things
> > > >>>>>>>>>>>>>>>>>>>>>>>>> were
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Nov 21,
> > > >>>>>> 2022
> > > >>>>>>> at
> > > >>>>>>>>>> 7:11 PM
> > > >>>>>>>>>>>>>>>> Matthias
> > > >>>>>>>>>>>>>>>>>> J.
> > > >>>>>>>>>>>>>>>>>>>>> Sax <
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the
> > > >>>>>> KIP.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Couple of
> > > >>>>>>>> clarification
> > > >>>>>>>>>>>> questions
> > > >>>>>>>>>>>>>>> (I
> > > >>>>>>>>>>>>>>>> am
> > > >>>>>>>>>>>>>>>>>>>> not a
> > > >>>>>>>>>>>>>>>>>>>>>>>> broker
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> expert
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> do
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some question are
> > > >>>>>>>>> obvious
> > > >>>>>>>>>> for
> > > >>>>>>>>>>>>>>> others,
> > > >>>>>>>>>>>>>>>>> but
> > > >>>>>>>>>>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>>>>>>>> me
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> my
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lack
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker knowledge).
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (10)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What happens if
> > > >>>>>> the
> > > >>>>>>>>>> message
> > > >>>>>>>>>>>> come
> > > >>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>> before
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> next
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request? It seems
> > > >>>>>>> the
> > > >>>>>>>>>> broker
> > > >>>>>>>>>>>>>>> hosting
> > > >>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> data
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anything about it
> > > >>>>>>> and
> > > >>>>>>>>>> append
> > > >>>>>>>>>>>> it to
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> partition,
> > > >>>>>>>>>>>>>>>>>>>>>>>>> too?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> What
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> difference between
> > > >>>>>>>> both
> > > >>>>>>>>>> cases?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, it seems a
> > > >>>>>> TX
> > > >>>>>>>>> would
> > > >>>>>>>>>> only
> > > >>>>>>>>>>>>>>> hang,
> > > >>>>>>>>>>>>>>>> if
> > > >>>>>>>>>>>>>>>>>>>> there
> > > >>>>>>>>>>>>>>>>>>>>>>> is no
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> following
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> either committer
> > > >>>>>> or
> > > >>>>>>>>>> aborted?
> > > >>>>>>>>>>>> Thus,
> > > >>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>> case
> > > >>>>>>>>>>>>>>>>>>>>>>>>> above,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> TX
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> might
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually not hang
> > > >>>>>>> (of
> > > >>>>>>>>>> course,
> > > >>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>> might
> > > >>>>>>>>>>>>>>>>>> get
> > > >>>>>>>>>>>>>>>>>>>> an
> > > >>>>>>>>>>>>>>>>>>>>>>> EOS
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> violation
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX was aborted and
> > > >>>>>>> the
> > > >>>>>>>>>> second
> > > >>>>>>>>>>>>>>>>> committed,
> > > >>>>>>>>>>>>>>>>>> or
> > > >>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>> other
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> way
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around).
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (20)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Of course, 1 and
> > > >>>>>> 2
> > > >>>>>>>>>> require
> > > >>>>>>>>>>>>>>>> client-side
> > > >>>>>>>>>>>>>>>>>>>>>>> changes, so
> > > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> those approaches
> > > >>>>>>> won’t
> > > >>>>>>>>>> apply.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For (1) I
> > > >>>>>> understand
> > > >>>>>>>>> why a
> > > >>>>>>>>>>>> client
> > > >>>>>>>>>>>>>>>>> change
> > > >>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>>>> necessary,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we need a client
> > > >>>>>>>> change
> > > >>>>>>>>>> for
> > > >>>>>>>>>>>> (2).
> > > >>>>>>>>>>>>>>> Can
> > > >>>>>>>>>>>>>>>>> you
> > > >>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > > >>>>>>>>>>>>>>>>>>>>>>>>> --
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Later
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> explain
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that we should
> > > >>>>>> send
> > > >>>>>>> a
> > > >>>>>>>>>>>>>>>>>>>>>>> DescribeTransactionRequest,
> > > >>>>>>>>>>>>>>>>>>>>>>>>> but I
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> am
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Can't we not just
> > > >>>>>> do
> > > >>>>>>>> an
> > > >>>>>>>>>>>> implicit
> > > >>>>>>>>>>>>>>>>>>>>>>> AddPartiitonToTx,
> > > >>>>>>>>>>>>>>>>>>>>>>>>> too?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> If
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> old
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer correctly
> > > >>>>>>>>>> registered
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>> partition
> > > >>>>>>>>>>>>>>>>>>>>>>>> already,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can just ignore it
> > > >>>>>>> as
> > > >>>>>>>>>> it's an
> > > >>>>>>>>>>>>>>>>> idempotent
> > > >>>>>>>>>>>>>>>>>>>>>>> operation?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (30)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To cover older
> > > >>>>>>>> clients,
> > > >>>>>>>>>> we
> > > >>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>>>> ensure a
> > > >>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ongoing
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> before
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we write to a
> > > >>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Not sure what you
> > > >>>>>>> mean
> > > >>>>>>>>> by
> > > >>>>>>>>>>>> this?
> > > >>>>>>>>>>>>>>> Can
> > > >>>>>>>>>>>>>>>> you
> > > >>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (40)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [the
> > > >>>>>>> TX-coordinator]
> > > >>>>>>>>> will
> > > >>>>>>>>>>>> write
> > > >>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> prepare
> > > >>>>>>>>>>>>>>>>>>>>>>> commit
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch and send
> > > >>>>>>>>>>>>>>> WriteTxnMarkerRequests
> > > >>>>>>>>>>>>>>>>>> with
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>> bumped
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why do we use the
> > > >>>>>>>> bumped
> > > >>>>>>>>>>>> epoch for
> > > >>>>>>>>>>>>>>>>> both?
> > > >>>>>>>>>>>>>>>>>> It
> > > >>>>>>>>>>>>>>>>>>>>>>> seems
> > > >>>>>>>>>>>>>>>>>>>>>>>>> more
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> intuitive
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the current epoch,
> > > >>>>>>> and
> > > >>>>>>>>>> only
> > > >>>>>>>>>>>> return
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> bumped
> > > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (50) "Implicit
> > > >>>>>>>>>>>>>>>>> AddPartitionToTransaction"
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why does the
> > > >>>>>>>> implicitly
> > > >>>>>>>>>> sent
> > > >>>>>>>>>>>>>>> request
> > > >>>>>>>>>>>>>>>>> need
> > > >>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> synchronous?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also says
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in case we need
> > > >>>>>> to
> > > >>>>>>>>> abort
> > > >>>>>>>>>> and
> > > >>>>>>>>>>>>>>> need to
> > > >>>>>>>>>>>>>>>>>> know
> > > >>>>>>>>>>>>>>>>>>>>> which
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What do you mean
> > > >>>>>> by
> > > >>>>>>>>> this?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we don’t want to
> > > >>>>>>>> write
> > > >>>>>>>>>> to it
> > > >>>>>>>>>>>>>>> before
> > > >>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>> store
> > > >>>>>>>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> manager
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Do you mean
> > > >>>>>>>>> TX-coordinator
> > > >>>>>>>>>>>>>>> instead of
> > > >>>>>>>>>>>>>>>>>>>>> "manager"?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (60)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For older clients
> > > >>>>>>> and
> > > >>>>>>>>>> ensuring
> > > >>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>> TX
> > > >>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>>>> ongoing,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> describe a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> race condition. I
> > > >>>>>> am
> > > >>>>>>>> not
> > > >>>>>>>>>> sure
> > > >>>>>>>>>>>> if I
> > > >>>>>>>>>>>>>>>> can
> > > >>>>>>>>>>>>>>>>>>>> follow
> > > >>>>>>>>>>>>>>>>>>>>>>> here.
> > > >>>>>>>>>>>>>>>>>>>>>>>>> Can
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/18/22 1:21
> > > >>>>>> PM,
> > > >>>>>>>>>> Justine
> > > >>>>>>>>>>>>>>> Olshan
> > > >>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey all!
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to
> > > >>>>>> start a
> > > >>>>>>>>>>>> discussion
> > > >>>>>>>>>>>>>>> on my
> > > >>>>>>>>>>>>>>>>>>>> proposal
> > > >>>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>> add
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> some
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checks on
> > > >>>>>>>> transactions
> > > >>>>>>>>> to
> > > >>>>>>>>>>>> avoid
> > > >>>>>>>>>>>>>>>>> hanging
> > > >>>>>>>>>>>>>>>>>>>>>>>>> transactions.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> I
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> been
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an issue for some
> > > >>>>>>>> time,
> > > >>>>>>>>>> so I
> > > >>>>>>>>>>>>>>> really
> > > >>>>>>>>>>>>>>>>> hope
> > > >>>>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>> KIP
> > > >>>>>>>>>>>>>>>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> helpful
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users of EOS.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The KIP includes
> > > >>>>>>>>> changes
> > > >>>>>>>>>> that
> > > >>>>>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>>>>> compatible
> > > >>>>>>>>>>>>>>>>>>>>>>>>> with
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> old
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> changes to
> > > >>>>>> improve
> > > >>>>>>>>>>>> performance
> > > >>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>> correctness
> > > >>>>>>>>>>>>>>>>>>>>>>> on
> > > >>>>>>>>>>>>>>>>>>>>>>>>> new
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Please take a
> > > >>>>>> look
> > > >>>>>>>> and
> > > >>>>>>>>>> leave
> > > >>>>>>>>>>>> any
> > > >>>>>>>>>>>>>>>>>> comments
> > > >>>>>>>>>>>>>>>>>>>> you
> > > >>>>>>>>>>>>>>>>>>>>>>> may
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> have!
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>
> > > >>>>>>
> > > >>>>
> > > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> JIRA:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-14402
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>>>
> > > >>>
> > > >>
> > > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
Thanks for clarifying Guozhang,

I like the framing:
* “if we did abort the txn and continue, could the app ever has a chance to
recover and not have any side effects violating EOS guarantees”.*

With this principle in mind, if we could convince ourselves that we won't
have side effects, then we could move forward with a non-fatal error. I
appreciate laying out the cases we've discussed. I think I also want to
take a look in very close detail and convince myself of the right option.
I'll get back to this thread with the final conclusion as well as update
the KIP.

Thanks for also bringing up KIP-691. I should take a look there as well. :)

Justine

On Thu, Jan 26, 2023 at 5:28 AM Guozhang Wang <gu...@gmail.com>
wrote:

> Hello Justine,
>
> Regarding which errors should be fatal v.s. abortable, I think the
> principle would be “if we did abort the txn and continue, could the
> app ever has a chance to recover and not have any side effects
> violating EOS guarantees”.Of course we can discuss whether that’s a
> good principle  e.g. one can argue that maybe it’s okay to violate EOS
> some times rather than killing the whole app, whereas my rationale is
> that since we want to make EOS the default config, it’s crucial that
> we try to guarantee it as much as possible and whenever we detected if
> there’s a chance that it maybe broken, we should let users know. And
> if we just log an error in app and try to abort and continue, it’s
> very likely that users would not be notified and probably only realize
> that after a while, which may cause an even harder scenario to
> debug.If people agree on that principle, we can go back and check this
> KIP’s scenario: there are several scenarios causing a partition leader
> detects an out of date epoch from a produce requests:
>
> * The old produce requests arrive late after a network partition, and
> the old producer is already gone.
> * The old produce requests arrive late after a network partition, but
> the old producer is still around.
> * A buggy producer client that did not follow the protocol.
>
> For the third case, we should try to let user know asap, and as clear
> as possible, and hence it’s best to just stop the client app; for the
> first case, it does not matter since the producer is already gone;
> only for the second, probably the least likely case, we need to
> ponder, and there my rationale again is that at that time, we may have
> already violated the EOS guarantees since there are some partial txn
> records that should be rejected while the txn itself has been
> committed. In this case I think it’s better to let clients/users know
> as soon as possible as well.
>
> Regarding the past scenarios where a fatal error killing the whole
> apps, I believe part of that reason is that we were doing an
> sub-optimal job on clients side handling various error cases and
> that’s what KIP-691 is trying to resolve, and hence personally I would
> suggest we do not weight in too much on that if we can trust that
> KIP-691 will be successfully eliminate those not-necessary-hard-fail
> scenarios.
>
>
> Guozhang
>
> On Wed, Jan 25, 2023 at 5:51 PM Matthias J. Sax <mj...@apache.org> wrote:
> >
> > So the timestamp would be set when the write happens and thus no
> > out-of-order data (base in time) can be introduced with "append_time"
> > config even if a request sits in purgatory first while we check the TX
> > status.
> >
> > That does make sense. Thanks for confirming, that there is no
> > out-of-order issue for this case.
> >
> >
> > -Matthias
> >
> > On 1/25/23 5:04 PM, Justine Olshan wrote:
> > > Hey Matthias,
> > > Let me put it this way, if a producer is checking if a transaction is
> > > ongoing, then no writes to the partition from the producer will go
> through
> > > until the transaction is confirmed ongoing.
> > >  From then, I think I can apply the writes in the order they came in.
> Does
> > > that make sense?
> > >
> > > Let me know if I'm missing something.
> > > Justine
> > >
> > > On Wed, Jan 25, 2023 at 4:57 PM Matthias J. Sax <mj...@apache.org>
> wrote:
> > >
> > >>> would it build an offset map with just the latest timestamp for a
> key?
> > >>
> > >> Cannot remember the details without reading the KIP, but yes,
> something
> > >> like this (I believe it actually needs to track both, offset and
> > >> timestamp per key).
> > >>
> > >>> I wonder if ordering assumptions are baked in there, why not use
> > >> offset-based compaction.
> > >>
> > >> The use case is a compacted topic that does contain out-of-order data.
> > >> If you set key k1=v1 @ 5 offset 100 and later key1 = v0 @ 3 at offset
> > >> 200 we want to cleanup v0 with higher offset because it's out-of-order
> > >> based on time, but keep v1 what is the actual latest version of k1.
> > >>
> > >>
> > >>> I was also not aware of this "guarantee" with regards to broker side
> > >> time.
> > >>
> > >> As already said: I am not sure if it's a public contract, but based on
> > >> my experience, people might reply on it as "implicit contract". --
> Maybe
> > >> somebody else knows if it's public or not, and if it would be ok to
> > >> "break" it.
> > >>
> > >>> Let me know if you have any concerns here.
> > >>
> > >> My understanding is: While we cannot make an offset-order guarantee
> for
> > >> interleaved writes of different producer, if the topic is configures
> > >> with "append_time", we "guarantee" (cf. my comment above") timestamp
> > >> order... If that's the case, it would be an issue if we break this
> > >> "guarantee".
> > >>
> > >> I am not sure when the broker sets the timestamp for "append_time"
> > >> config? If we do it before putting the request into purgatory, we
> have a
> > >> problem. However, if we set the timestamp when we actually process the
> > >> request and do the actual append, it seems there is no issue, as the
> > >> request that was waiting in purgatory get the "newest" timestamp and
> > >> thus cannot introduce out-of-order data.
> > >>
> > >>
> > >> -Matthias
> > >>
> > >>
> > >> On 1/24/23 10:44 AM, Justine Olshan wrote:
> > >>> Hey Matthias,
> > >>>
> > >>> I have actually never heard of KIP-280 so thanks for bringing it up.
> That
> > >>> seems interesting. I wonder how it would work though -- would it
> build an
> > >>> offset map with just the latest timestamp for a key? I wonder if
> ordering
> > >>> assumptions are baked in there, why not use offset-based compaction.
> > >>>
> > >>> I was also not aware of this "guarantee" with regards to broker side
> > >> time.
> > >>> I think that we can do in order handling for a given producer, but
> not
> > >>> across all producers. However, we can't guarantee that anyway.
> > >>>
> > >>> Let me know if you have any concerns here.
> > >>>
> > >>> Thanks,
> > >>> Justine
> > >>>
> > >>> On Mon, Jan 23, 2023 at 6:33 PM Matthias J. Sax <mj...@apache.org>
> > >> wrote:
> > >>>
> > >>>> Just a side note about Guozhang comments about timestamps.
> > >>>>
> > >>>> If the producer sets the timestamp, putting the record into
> purgatory
> > >>>> seems not to be an issue (as already said: for this case we don't
> > >>>> guarantee timestamp order between writes of different producers
> anyway).
> > >>>> However, if the broker sets the timestamp, the expectation is that
> there
> > >>>> is no out-of-order data in the partition ever; if we would introduce
> > >>>> out-of-order data for this case (for interleaved writes of different
> > >>>> producers), it seems we would violate the current contract? (To be
> fair:
> > >>>> I don't know if that's an official contract, but I assume people
> rely on
> > >>>> this behavior -- and it "advertised" in many public talks...)
> > >>>>
> > >>>> About compaction: there is actually KIP-280 that adds timestamp
> based
> > >>>> compaction what is a very useful feature for Kafka Streams with
> regard
> > >>>> to out-of-order data handling. So the impact if we introduce
> > >>>> out-of-order data could be larger scoped.
> > >>>>
> > >>>>
> > >>>> -Matthias
> > >>>>
> > >>>>
> > >>>> On 1/20/23 4:48 PM, Justine Olshan wrote:
> > >>>>> Hey Artem,
> > >>>>>
> > >>>>> I see there is a check for transactional producers. I'm wondering
> if we
> > >>>>> don't handle the epoch overflow case. I'm also not sure it will be
> a
> > >> huge
> > >>>>> issue to extend to transactional producers, but maybe I'm missing
> > >>>> something.
> > >>>>>
> > >>>>> As for the recovery path -- I think Guozhang's point was if we
> have a
> > >> bad
> > >>>>> client that repeatedly tries to produce without adding to the
> > >> transaction
> > >>>>> we would do the following:
> > >>>>> a) if not fatal, we just fail the produce request over and over
> > >>>>> b) if fatal, we fence the producer
> > >>>>>
> > >>>>> Here with B, the issue with the client would be made clear more
> > >> quickly.
> > >>>> I
> > >>>>> suppose there are some intermediate cases where the issue only
> occurs
> > >>>>> sometimes, but I wonder if we should consider how to recover with
> > >> clients
> > >>>>> who don't behave as expected anyway.
> > >>>>>
> > >>>>> I think there is a place for the abortable error that we are
> adding --
> > >>>> just
> > >>>>> abort and try again. But I think there are also some cases where
> trying
> > >>>> to
> > >>>>> recover overcomplicates some logic. Especially if we are
> considering
> > >>>> older
> > >>>>> clients -- there I'm not sure if there's a ton we can do besides
> fail
> > >> the
> > >>>>> batch or fence the producer. With newer clients, we can consider
> more
> > >>>>> options for what can just be recovered after aborting. But epochs
> might
> > >>>> be
> > >>>>> a hard one unless we also want to reset producer ID.
> > >>>>>
> > >>>>> Thanks,
> > >>>>> Justine
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> On Fri, Jan 20, 2023 at 3:59 PM Artem Livshits
> > >>>>> <al...@confluent.io.invalid> wrote:
> > >>>>>
> > >>>>>>>     besides the poorly written client case
> > >>>>>>
> > >>>>>> A poorly written client could create a lot of grief to people who
> run
> > >>>> Kafka
> > >>>>>> brokers :-), so when deciding to make an error fatal I would see
> if
> > >>>> there
> > >>>>>> is a reasonable recovery path rather than how often it could
> happen.
> > >>>> If we
> > >>>>>> have solid implementation of transactions (which I hope we'll do
> as a
> > >>>>>> result of this KIP), it would help to recover from a large class
> of
> > >>>> errors
> > >>>>>> by just aborting a transaction, even if the cause of error is a
> race
> > >>>>>> condition or etc.
> > >>>>>>
> > >>>>>> -Artem
> > >>>>>>
> > >>>>>> On Fri, Jan 20, 2023 at 3:26 PM Justine Olshan
> > >>>>>> <jo...@confluent.io.invalid>
> > >>>>>> wrote:
> > >>>>>>
> > >>>>>>> Artem --
> > >>>>>>> I guess the discussion path we were going down is when we expect
> to
> > >> see
> > >>>>>>> this error. I mentioned that it was hard to come up with cases
> for
> > >> when
> > >>>>>> the
> > >>>>>>> producer would still be around to receive the error besides the
> > >> poorly
> > >>>>>>> written client case.
> > >>>>>>> If we don't expect to have a producer to receive the response, it
> > >> sort
> > >>>> of
> > >>>>>>> makes sense for it to be fatal.
> > >>>>>>>
> > >>>>>>> I had some discussion with Jason offline about the epoch being
> off
> > >>>> cases
> > >>>>>>> and I'm not sure we could find a ton (outside of produce
> requests)
> > >>>> where
> > >>>>>> we
> > >>>>>>> could/should recover. I'd be happy to hear some examples though,
> > >> maybe
> > >>>>>> I'm
> > >>>>>>> missing something.
> > >>>>>>>
> > >>>>>>> Thanks,
> > >>>>>>> Justine
> > >>>>>>>
> > >>>>>>> On Fri, Jan 20, 2023 at 3:19 PM Artem Livshits
> > >>>>>>> <al...@confluent.io.invalid> wrote:
> > >>>>>>>
> > >>>>>>>> In general, I'd like to avoid fatal errors as much as possible,
> in
> > >>>> some
> > >>>>>>>> sense fatal errors just push out recovery logic to the
> application
> > >>>>>> which
> > >>>>>>>> either complicates the application or leads to disruption (we've
> > >> seen
> > >>>>>>> cases
> > >>>>>>>> when a transient broker error could lead to work stoppage when
> > >>>>>>> applications
> > >>>>>>>> need to be manually restarted).  I think we should strive to
> define
> > >>>>>>>> recovery logic for most errors (and/or encapsulate it in the
> Kafka
> > >>>>>> client
> > >>>>>>>> as much as possible).
> > >>>>>>>>
> > >>>>>>>> One benefit of transactions is that they simplify recovery from
> > >>>> errors,
> > >>>>>>>> pretty much any error (that's not handled transparently by
> retries
> > >> in
> > >>>>>>> Kafka
> > >>>>>>>> client) can be handled by the application via aborting the
> > >> transaction
> > >>>>>>> and
> > >>>>>>>> repeating the transactional logic again.  One tricky error is an
> > >> error
> > >>>>>>>> during commit, because we don't know the outcome.  For commit
> > >> errors,
> > >>>>>> the
> > >>>>>>>> recommendation should be to retry the commit until it returns
> the
> > >>>>>>> specific
> > >>>>>>>> result (committed or aborted).
> > >>>>>>>>
> > >>>>>>>> -Artem
> > >>>>>>>>
> > >>>>>>>> On Fri, Jan 20, 2023 at 2:52 PM Justine Olshan
> > >>>>>>>> <jo...@confluent.io.invalid>
> > >>>>>>>> wrote:
> > >>>>>>>>
> > >>>>>>>>> That's a fair point about other clients.
> > >>>>>>>>>
> > >>>>>>>>> I think the abortable error case is interesting because I'm
> curious
> > >>>>>> how
> > >>>>>>>>> other clients would handle this. I assume they would need to
> > >>>>>> implement
> > >>>>>>>>> handling for the error code unless they did something like "any
> > >>>>>> unknown
> > >>>>>>>>> error codes/any codes that aren't x,y,z are retriable." I would
> > >> hope
> > >>>>>>> that
> > >>>>>>>>> unknown error codes were fatal, and if the code was
> implemented it
> > >>>>>>> would
> > >>>>>>>>> abort the transaction. But I will think on this too.
> > >>>>>>>>>
> > >>>>>>>>> As for InvalidRecord -- you mentioned it was not fatal, but I'm
> > >>>>>> taking
> > >>>>>>> a
> > >>>>>>>>> look through the code. We would see this on handling the
> produce
> > >>>>>>>> response.
> > >>>>>>>>> If I recall correctly, we check if errors are retriable. I
> think
> > >> this
> > >>>>>>>> error
> > >>>>>>>>> would not be retriable. But I guess the concern here is that
> it is
> > >>>>>> not
> > >>>>>>>>> enough for just that batch to fail. I guess I hadn't considered
> > >> fully
> > >>>>>>>>> fencing the old producer but there are valid arguments here
> why we
> > >>>>>>> would
> > >>>>>>>>> want to.
> > >>>>>>>>>
> > >>>>>>>>> Thanks,
> > >>>>>>>>> Justine
> > >>>>>>>>>
> > >>>>>>>>> On Fri, Jan 20, 2023 at 2:35 PM Guozhang Wang <
> > >>>>>>>> guozhang.wang.us@gmail.com>
> > >>>>>>>>> wrote:
> > >>>>>>>>>
> > >>>>>>>>>> Thanks Justine for the replies! I agree with most of your
> > >> thoughts.
> > >>>>>>>>>>
> > >>>>>>>>>> Just for 3/7), though I agree for our own AK producer, since
> we do
> > >>>>>>>>>> "nextRequest(boolean hasIncompleteBatches)", we guarantee the
> > >>>>>> end-txn
> > >>>>>>>>>> would not be sent until we've effectively flushed, but I was
> > >>>>>>> referring
> > >>>>>>>>>> to any future bugs or other buggy clients that the same
> client may
> > >>>>>>> get
> > >>>>>>>>>> into this situation, in which case we should give the client a
> > >>>>>> clear
> > >>>>>>>>>> msg that "you did something wrong, and hence now you should
> > >> fatally
> > >>>>>>>>>> close yourself". What I'm concerned about is that, by seeing
> an
> > >>>>>>>>>> "abortable error" or in some rare cases an "invalid record",
> the
> > >>>>>>>>>> client could not realize "something that's really bad
> happened".
> > >> So
> > >>>>>>>>>> it's not about adding a new error, it's mainly about those
> real
> > >>>>>> buggy
> > >>>>>>>>>> situations causing such "should never happen" cases, the
> errors
> > >>>>>>> return
> > >>>>>>>>>> would not be informative enough.
> > >>>>>>>>>>
> > >>>>>>>>>> Thinking in other ways, if we believe that for most cases such
> > >>>>>> error
> > >>>>>>>>>> codes would not reach the original clients since they would be
> > >>>>>>>>>> disconnected or even gone by that time, and only in some rare
> > >> cases
> > >>>>>>>>>> they would still be seen by the sending clients, then why not
> make
> > >>>>>>>>>> them more fatal and more specific than generic.
> > >>>>>>>>>>
> > >>>>>>>>>> Guozhang
> > >>>>>>>>>>
> > >>>>>>>>>> On Fri, Jan 20, 2023 at 1:59 PM Justine Olshan
> > >>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > >>>>>>>>>>>
> > >>>>>>>>>>> Hey Guozhang. Thanks for taking a look and for the detailed
> > >>>>>>> comments!
> > >>>>>>>>>> I'll
> > >>>>>>>>>>> do my best to address below.
> > >>>>>>>>>>>
> > >>>>>>>>>>> 1. I see what you are saying here, but I think I need to look
> > >>>>>>> through
> > >>>>>>>>> the
> > >>>>>>>>>>> sequence of events you mention. Typically we've seen this
> issue
> > >>>>>> in
> > >>>>>>> a
> > >>>>>>>>> few
> > >>>>>>>>>>> cases.
> > >>>>>>>>>>>
> > >>>>>>>>>>>     One is when we have a producer disconnect when trying to
> > >>>>>> produce.
> > >>>>>>>>>>> Typically in these cases, we abort the transaction. We've
> seen
> > >>>>>> that
> > >>>>>>>>> after
> > >>>>>>>>>>> the markers are written, the disconnection can sometimes
> cause
> > >>>>>> the
> > >>>>>>>>>> request
> > >>>>>>>>>>> to get flushed to the broker. In this case, we don't need
> client
> > >>>>>>>>> handling
> > >>>>>>>>>>> because the producer we are responding to is gone. We just
> needed
> > >>>>>>> to
> > >>>>>>>>> make
> > >>>>>>>>>>> sure we didn't write to the log on the broker side. I'm
> trying to
> > >>>>>>>> think
> > >>>>>>>>>> of
> > >>>>>>>>>>> a case where we do have the client to return to. I'd think
> the
> > >>>>>> same
> > >>>>>>>>>> client
> > >>>>>>>>>>> couldn't progress to committing the transaction unless the
> > >>>>>> produce
> > >>>>>>>>>> request
> > >>>>>>>>>>> returned right? Of course, there is the incorrectly written
> > >>>>>> clients
> > >>>>>>>>> case.
> > >>>>>>>>>>> I'll think on this a bit more and let you know if I come up
> with
> > >>>>>>>>> another
> > >>>>>>>>>>> scenario when we would return to an active client when the
> > >>>>>>>> transaction
> > >>>>>>>>> is
> > >>>>>>>>>>> no longer ongoing.
> > >>>>>>>>>>>
> > >>>>>>>>>>> I was not aware that we checked the result of a send after we
> > >>>>>>> commit
> > >>>>>>>>>>> though. I'll need to look into that a bit more.
> > >>>>>>>>>>>
> > >>>>>>>>>>> 2. There were some questions about this in the discussion.
> The
> > >>>>>> plan
> > >>>>>>>> is
> > >>>>>>>>> to
> > >>>>>>>>>>> handle overflow with the mechanism we currently have in the
> > >>>>>>> producer.
> > >>>>>>>>> If
> > >>>>>>>>>> we
> > >>>>>>>>>>> try to bump and the epoch will overflow, we actually
> allocate a
> > >>>>>> new
> > >>>>>>>>>>> producer ID. I need to confirm the fencing logic on the last
> > >>>>>> epoch
> > >>>>>>>> (ie,
> > >>>>>>>>>> we
> > >>>>>>>>>>> probably shouldn't allow any records to be produced with the
> > >>>>>> final
> > >>>>>>>>> epoch
> > >>>>>>>>>>> since we can never properly fence that one).
> > >>>>>>>>>>>
> > >>>>>>>>>>> 3. I can agree with you that the current error handling is
> > >>>>>> messy. I
> > >>>>>>>>>> recall
> > >>>>>>>>>>> taking a look at your KIP a while back, but I think I mostly
> saw
> > >>>>>>> the
> > >>>>>>>>>>> section about how the errors were wrapped. Maybe I need to
> take
> > >>>>>>>> another
> > >>>>>>>>>>> look. As for abortable error, the idea was that the handling
> > >>>>>> would
> > >>>>>>> be
> > >>>>>>>>>>> simple -- if this error is seen, the transaction should be
> > >>>>>> aborted
> > >>>>>>> --
> > >>>>>>>>> no
> > >>>>>>>>>>> other logic about previous state or requests necessary. Is
> your
> > >>>>>>>> concern
> > >>>>>>>>>>> simply about adding new errors? We were hoping to have an
> error
> > >>>>>>> that
> > >>>>>>>>>> would
> > >>>>>>>>>>> have one meaning and many of the current errors have a
> history of
> > >>>>>>>>> meaning
> > >>>>>>>>>>> different things on different client versions. That was the
> main
> > >>>>>>>>>> motivation
> > >>>>>>>>>>> for adding a new error.
> > >>>>>>>>>>>
> > >>>>>>>>>>> 4. This is a good point about record timestamp reordering.
> > >>>>>>> Timestamps
> > >>>>>>>>>> don't
> > >>>>>>>>>>> affect compaction, but they do affect retention deletion. For
> > >>>>>> that,
> > >>>>>>>>> kafka
> > >>>>>>>>>>> considers the largest timestamp in the segment, so I think a
> > >>>>>> small
> > >>>>>>>>> amount
> > >>>>>>>>>>> of reordering (hopefully on the order of milliseconds or even
> > >>>>>>>> seconds)
> > >>>>>>>>>> will
> > >>>>>>>>>>> be ok. We take timestamps from clients so there is already a
> > >>>>>>>>> possibility
> > >>>>>>>>>>> for some drift and non-monotonically increasing timestamps.
> > >>>>>>>>>>>
> > >>>>>>>>>>> 5. Thanks for catching. The error is there, but it's actually
> > >>>>>> that
> > >>>>>>>>> those
> > >>>>>>>>>>> fields should be 4+! Due to how the message generator works,
> I
> > >>>>>>>> actually
> > >>>>>>>>>>> have to redefine those fields inside the
> > >>>>>>>>> `"AddPartitionsToTxnTransaction`
> > >>>>>>>>>>> block for it to build correctly. I'll fix it to be correct.
> > >>>>>>>>>>>
> > >>>>>>>>>>> 6. Correct -- we will only add the request to purgatory if
> the
> > >>>>>>> cache
> > >>>>>>>>> has
> > >>>>>>>>>> no
> > >>>>>>>>>>> ongoing transaction. I can change the wording to make that
> > >>>>>> clearer
> > >>>>>>>> that
> > >>>>>>>>>> we
> > >>>>>>>>>>> only place the request in purgatory if we need to contact the
> > >>>>>>>>> transaction
> > >>>>>>>>>>> coordinator.
> > >>>>>>>>>>>
> > >>>>>>>>>>> 7. We did take a look at some of the errors and it was hard
> to
> > >>>>>> come
> > >>>>>>>> up
> > >>>>>>>>>> with
> > >>>>>>>>>>> a good one. I agree that InvalidTxnStateException is ideal
> except
> > >>>>>>> for
> > >>>>>>>>> the
> > >>>>>>>>>>> fact that it hasn't been returned on Produce requests
> before. The
> > >>>>>>>> error
> > >>>>>>>>>>> handling for clients is a bit vague (which is why I opened
> > >>>>>>>> KAFKA-14439
> > >>>>>>>>>>> <https://issues.apache.org/jira/browse/KAFKA-14439>), but
> the
> > >>>>>>>> decision
> > >>>>>>>>>> we
> > >>>>>>>>>>> made here was to only return errors that have been previously
> > >>>>>>>> returned
> > >>>>>>>>> to
> > >>>>>>>>>>> producers. As for not being fatal, I think part of the
> theory was
> > >>>>>>>> that
> > >>>>>>>>> in
> > >>>>>>>>>>> many cases, the producer would be disconnected. (See point
> 1) and
> > >>>>>>>> this
> > >>>>>>>>>>> would just be an error to return from the server. I did plan
> to
> > >>>>>>> think
> > >>>>>>>>>> about
> > >>>>>>>>>>> other cases, so let me know if you think of any as well!
> > >>>>>>>>>>>
> > >>>>>>>>>>> Lots to say! Let me know if you have further thoughts!
> > >>>>>>>>>>> Justine
> > >>>>>>>>>>>
> > >>>>>>>>>>> On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang <
> > >>>>>>>>>> guozhang.wang.us@gmail.com>
> > >>>>>>>>>>> wrote:
> > >>>>>>>>>>>
> > >>>>>>>>>>>> Hello Justine,
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> Thanks for the great write-up! I made a quick pass through
> it
> > >>>>>> and
> > >>>>>>>>> here
> > >>>>>>>>>>>> are some thoughts (I have not been able to read through this
> > >>>>>>> thread
> > >>>>>>>>> so
> > >>>>>>>>>>>> pardon me if they have overlapped or subsumed by previous
> > >>>>>>>> comments):
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> First are some meta ones:
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> 1. I think we need to also improve the client's experience
> once
> > >>>>>>> we
> > >>>>>>>>>>>> have this defence in place. More concretely, say a user's
> > >>>>>>> producer
> > >>>>>>>>>>>> code is like following:
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> future = producer.send();
> > >>>>>>>>>>>> // producer.flush();
> > >>>>>>>>>>>> producer.commitTransaction();
> > >>>>>>>>>>>> future.get();
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> Which resulted in the order of a) produce-request sent by
> > >>>>>>> producer,
> > >>>>>>>>> b)
> > >>>>>>>>>>>> end-txn-request sent by producer, c) end-txn-response sent
> > >>>>>> back,
> > >>>>>>> d)
> > >>>>>>>>>>>> txn-marker-request sent from coordinator to partition
> leader,
> > >>>>>> e)
> > >>>>>>>>>>>> produce-request finally received by the partition leader,
> > >>>>>> before
> > >>>>>>>> this
> > >>>>>>>>>>>> KIP e) step would be accepted causing a dangling txn; now it
> > >>>>>>> would
> > >>>>>>>> be
> > >>>>>>>>>>>> rejected in step e) which is good. But from the client's
> point
> > >>>>>> of
> > >>>>>>>>> view
> > >>>>>>>>>>>> now it becomes confusing since the `commitTransaction()`
> > >>>>>> returns
> > >>>>>>>>>>>> successfully, but the "future" throws an invalid-epoch
> error,
> > >>>>>> and
> > >>>>>>>>> they
> > >>>>>>>>>>>> are not sure if the transaction did succeed or not. In
> fact, it
> > >>>>>>>>>>>> "partially succeeded" with some msgs being rejected but
> others
> > >>>>>>>>>>>> committed successfully.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> Of course the easy way to avoid this is, always call
> > >>>>>>>>>>>> "producer.flush()" before commitTxn and that's what we do
> > >>>>>>>> ourselves,
> > >>>>>>>>>>>> and what we recommend users do. But I suspect not everyone
> does
> > >>>>>>> it.
> > >>>>>>>>> In
> > >>>>>>>>>>>> fact I just checked the javadoc in KafkaProducer and our
> code
> > >>>>>>>> snippet
> > >>>>>>>>>>>> does not include a `flush()` call. So I'm thinking maybe we
> can
> > >>>>>>> in
> > >>>>>>>>>>>> side the `commitTxn` code to enforce flushing before sending
> > >>>>>> the
> > >>>>>>>>>>>> end-txn request.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> 2. I'd like to clarify a bit details on "just add
> partitions to
> > >>>>>>> the
> > >>>>>>>>>>>> transaction on the first produce request during a
> transaction".
> > >>>>>>> My
> > >>>>>>>>>>>> understanding is that the partition leader's cache has the
> > >>>>>>> producer
> > >>>>>>>>> id
> > >>>>>>>>>>>> / sequence / epoch for the latest txn, either on-going or is
> > >>>>>>>>> completed
> > >>>>>>>>>>>> (upon receiving the marker request from coordinator). When a
> > >>>>>>>> produce
> > >>>>>>>>>>>> request is received, if
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> * producer's epoch < cached epoch, or producer's epoch ==
> > >>>>>> cached
> > >>>>>>>>> epoch
> > >>>>>>>>>>>> but the latest txn is completed, leader directly reject with
> > >>>>>>>>>>>> invalid-epoch.
> > >>>>>>>>>>>> * producer's epoch > cached epoch, park the the request and
> > >>>>>> send
> > >>>>>>>>>>>> add-partitions request to coordinator.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> In order to do it, does the coordinator need to bump the
> > >>>>>> sequence
> > >>>>>>>> and
> > >>>>>>>>>>>> reset epoch to 0 when the next epoch is going to overflow?
> If
> > >>>>>> no
> > >>>>>>>> need
> > >>>>>>>>>>>> to do so, then how we handle the (admittedly rare, but still
> > >>>>>> may
> > >>>>>>>>>>>> happen) epoch overflow situation?
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> 3. I'm a bit concerned about adding a generic
> "ABORTABLE_ERROR"
> > >>>>>>>> given
> > >>>>>>>>>>>> we already have a pretty messy error classification and
> error
> > >>>>>>>>> handling
> > >>>>>>>>>>>> on the producer clients side --- I have a summary about the
> > >>>>>>> issues
> > >>>>>>>>> and
> > >>>>>>>>>>>> a proposal to address this in
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>
> > >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
> > >>>>>>>>>>>> -- I understand we do not want to use "UNKNOWN_PRODUCER_ID"
> > >>>>>>> anymore
> > >>>>>>>>>>>> and in fact we intend to deprecate it in KIP-360 and
> eventually
> > >>>>>>>>> remove
> > >>>>>>>>>>>> it; but I'm wondering can we still use specific error codes.
> > >>>>>> E.g.
> > >>>>>>>>> what
> > >>>>>>>>>>>> about "InvalidProducerEpochException" since for new clients,
> > >>>>>> the
> > >>>>>>>>>>>> actual reason this would actually be rejected is indeed
> because
> > >>>>>>> the
> > >>>>>>>>>>>> epoch on the coordinator caused the add-partitions-request
> from
> > >>>>>>> the
> > >>>>>>>>>>>> brokers to be rejected anyways?
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> 4. It seems we put the producer request into purgatory
> before
> > >>>>>> we
> > >>>>>>>> ever
> > >>>>>>>>>>>> append the records, while other producer's records may
> still be
> > >>>>>>>>>>>> appended during the time; and that potentially may result in
> > >>>>>> some
> > >>>>>>>>>>>> re-ordering compared with reception order. I'm not super
> > >>>>>>> concerned
> > >>>>>>>>>>>> about it since Kafka does not guarantee reception ordering
> > >>>>>> across
> > >>>>>>>>>>>> producers anyways, but it may make the timestamps of records
> > >>>>>>>> inside a
> > >>>>>>>>>>>> partition to be more out-of-ordered. Are we aware of any
> > >>>>>>> scenarios
> > >>>>>>>>>>>> such as future enhancements on log compactions that may be
> > >>>>>>> affected
> > >>>>>>>>> by
> > >>>>>>>>>>>> this effect?
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> Below are just minor comments:
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> 5. In "AddPartitionsToTxnTransaction" field of
> > >>>>>>>>>>>> "AddPartitionsToTxnRequest" RPC, the versions of those inner
> > >>>>>>> fields
> > >>>>>>>>>>>> are "0-3" while I thought they should be "0+" still?
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> 6. Regarding "we can place the request in a purgatory of
> sorts
> > >>>>>>> and
> > >>>>>>>>>>>> check if there is any state for the transaction on the
> > >>>>>> broker": i
> > >>>>>>>>>>>> think at this time when we just do the checks against the
> > >>>>>> cached
> > >>>>>>>>>>>> state, we do not need to put the request to purgatory yet?
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> 7. This is related to 3) above. I feel using
> > >>>>>>>> "InvalidRecordException"
> > >>>>>>>>>>>> for older clients may also be a bit confusing, and also it
> is
> > >>>>>> not
> > >>>>>>>>>>>> fatal -- for old clients, it better to be fatal since this
> > >>>>>>>> indicates
> > >>>>>>>>>>>> the clients is doing something wrong and hence it should be
> > >>>>>>> closed.
> > >>>>>>>>>>>> And in general I'd prefer to use slightly more specific
> meaning
> > >>>>>>>> error
> > >>>>>>>>>>>> codes for clients. That being said, I also feel
> > >>>>>>>>>>>> "InvalidProducerEpochException" is not suitable for old
> > >>>>>> versioned
> > >>>>>>>>>>>> clients, and we'd have to pick one that old clients
> recognize.
> > >>>>>>> I'd
> > >>>>>>>>>>>> prefer "InvalidTxnStateException" but that one is supposed
> to
> > >>>>>> be
> > >>>>>>>>>>>> returned from txn coordinators only today. I'd suggest we
> do a
> > >>>>>>>> quick
> > >>>>>>>>>>>> check in the current client's code path and see if that one
> > >>>>>> would
> > >>>>>>>> be
> > >>>>>>>>>>>> handled if it's from a produce-response, and if yes, use
> this
> > >>>>>>> one;
> > >>>>>>>>>>>> otherwise, use "ProducerFencedException" which is much less
> > >>>>>>>>> meaningful
> > >>>>>>>>>>>> but it's still a fatal error.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>> Guozhang
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
> > >>>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Yeah -- looks like we already have code to handle bumping
> the
> > >>>>>>>> epoch
> > >>>>>>>>>> and
> > >>>>>>>>>>>>> when the epoch is Short.MAX_VALUE, we get a new producer
> ID.
> > >>>>>>>> Since
> > >>>>>>>>>> this
> > >>>>>>>>>>>> is
> > >>>>>>>>>>>>> already the behavior, do we want to change it further?
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Justine
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <
> > >>>>>>>>> jolshan@confluent.io
> > >>>>>>>>>>>
> > >>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Hey all, just wanted to quickly update and say I've
> > >>>>>> modified
> > >>>>>>>> the
> > >>>>>>>>>> KIP to
> > >>>>>>>>>>>>>> explicitly mention that AddOffsetCommitsToTxnRequest will
> > >>>>>> be
> > >>>>>>>>>> replaced
> > >>>>>>>>>>>> by
> > >>>>>>>>>>>>>> a coordinator-side (inter-broker) AddPartitionsToTxn
> > >>>>>> implicit
> > >>>>>>>>>> request.
> > >>>>>>>>>>>> This
> > >>>>>>>>>>>>>> mirrors the user partitions and will implicitly add offset
> > >>>>>>>>>> partitions
> > >>>>>>>>>>>> to
> > >>>>>>>>>>>>>> transactions when we commit offsets on them. We will
> > >>>>>>> deprecate
> > >>>>>>>>>>>> AddOffsetCommitsToTxnRequest
> > >>>>>>>>>>>>>> for new clients.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Also to address Artem's comments --
> > >>>>>>>>>>>>>> I'm a bit unsure if the changes here will change the
> > >>>>>> previous
> > >>>>>>>>>> behavior
> > >>>>>>>>>>>> for
> > >>>>>>>>>>>>>> fencing producers. In the case you mention in the first
> > >>>>>>>>> paragraph,
> > >>>>>>>>>> are
> > >>>>>>>>>>>> you
> > >>>>>>>>>>>>>> saying we bump the epoch before we try to abort the
> > >>>>>>>> transaction?
> > >>>>>>>>> I
> > >>>>>>>>>>>> think I
> > >>>>>>>>>>>>>> need to understand the scenarios you mention a bit better.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> As for the second part -- I think it makes sense to have
> > >>>>>> some
> > >>>>>>>>> sort
> > >>>>>>>>>> of
> > >>>>>>>>>>>>>> "sentinel" epoch to signal epoch is about to overflow (I
> > >>>>>>> think
> > >>>>>>>> we
> > >>>>>>>>>> sort
> > >>>>>>>>>>>> of
> > >>>>>>>>>>>>>> have this value in place in some ways) so we can codify it
> > >>>>>> in
> > >>>>>>>> the
> > >>>>>>>>>> KIP.
> > >>>>>>>>>>>> I'll
> > >>>>>>>>>>>>>> look into that and try to update soon.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>> Justine.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
> > >>>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> It's good to know that KIP-588 addressed some of the
> > >>>>>> issues.
> > >>>>>>>>>> Looking
> > >>>>>>>>>>>> at
> > >>>>>>>>>>>>>>> the code, it still looks like there are some cases that
> > >>>>>>> would
> > >>>>>>>>>> result
> > >>>>>>>>>>>> in
> > >>>>>>>>>>>>>>> fatal error, e.g. PRODUCER_FENCED is issued by the
> > >>>>>>> transaction
> > >>>>>>>>>>>> coordinator
> > >>>>>>>>>>>>>>> if epoch doesn't match, and the client treats it as a
> > >>>>>> fatal
> > >>>>>>>>> error
> > >>>>>>>>>>>> (code in
> > >>>>>>>>>>>>>>> TransactionManager request handling).  If we consider,
> for
> > >>>>>>>>>> example,
> > >>>>>>>>>>>>>>> committing a transaction that returns a timeout, but
> > >>>>>>> actually
> > >>>>>>>>>>>> succeeds,
> > >>>>>>>>>>>>>>> trying to abort it or re-commit may result in
> > >>>>>>> PRODUCER_FENCED
> > >>>>>>>>>> error
> > >>>>>>>>>>>>>>> (because of epoch bump).
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> For failed commits, specifically, we need to know the
> > >>>>>> actual
> > >>>>>>>>>> outcome,
> > >>>>>>>>>>>>>>> because if we return an error the application may think
> > >>>>>> that
> > >>>>>>>> the
> > >>>>>>>>>>>>>>> transaction is aborted and redo the work, leading to
> > >>>>>>>> duplicates.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Re: overflowing epoch.  We could either do it on the TC
> > >>>>>> and
> > >>>>>>>>> return
> > >>>>>>>>>>>> both
> > >>>>>>>>>>>>>>> producer id and epoch (e.g. change the protocol), or
> > >>>>>> signal
> > >>>>>>>> the
> > >>>>>>>>>> client
> > >>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>> it needs to get a new producer id.  Checking for max
> epoch
> > >>>>>>>> could
> > >>>>>>>>>> be a
> > >>>>>>>>>>>>>>> reasonable signal, the value to check should probably be
> > >>>>>>>> present
> > >>>>>>>>>> in
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>> KIP
> > >>>>>>>>>>>>>>> as this is effectively a part of the contract.  Also, the
> > >>>>>> TC
> > >>>>>>>>>> should
> > >>>>>>>>>>>>>>> probably return an error if the client didn't change
> > >>>>>>> producer
> > >>>>>>>> id
> > >>>>>>>>>> after
> > >>>>>>>>>>>>>>> hitting max epoch.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> -Artem
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
> > >>>>>>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Thanks for the discussion Artem.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> With respect to the handling of fenced producers, we
> > >>>>>> have
> > >>>>>>>> some
> > >>>>>>>>>>>> behavior
> > >>>>>>>>>>>>>>>> already in place. As of KIP-588:
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>
> > >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> > >>>>>>>>>>>>>>>> ,
> > >>>>>>>>>>>>>>>> we handle timeouts more gracefully. The producer can
> > >>>>>>>> recover.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Produce requests can also recover from epoch fencing by
> > >>>>>>>>>> aborting the
> > >>>>>>>>>>>>>>>> transaction and starting over.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> What other cases were you considering that would cause
> > >>>>>> us
> > >>>>>>> to
> > >>>>>>>>>> have a
> > >>>>>>>>>>>>>>> fenced
> > >>>>>>>>>>>>>>>> epoch but we'd want to recover?
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> The first point about handling epoch overflows is fair.
> > >>>>>> I
> > >>>>>>>>> think
> > >>>>>>>>>>>> there is
> > >>>>>>>>>>>>>>>> some logic we'd need to consider. (ie, if we are one
> > >>>>>> away
> > >>>>>>>> from
> > >>>>>>>>>> the
> > >>>>>>>>>>>> max
> > >>>>>>>>>>>>>>>> epoch, we need to reset the producer ID.) I'm still
> > >>>>>>>> wondering
> > >>>>>>>>> if
> > >>>>>>>>>>>> there
> > >>>>>>>>>>>>>>> is a
> > >>>>>>>>>>>>>>>> way to direct this from the response, or if everything
> > >>>>>>>> should
> > >>>>>>>>> be
> > >>>>>>>>>>>> done on
> > >>>>>>>>>>>>>>>> the client side. Let me know if you have any thoughts
> > >>>>>>> here.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>> Justine
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
> > >>>>>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> There are some workflows in the client that are
> > >>>>>> implied
> > >>>>>>> by
> > >>>>>>>>>>>> protocol
> > >>>>>>>>>>>>>>>>> changes, e.g.:
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> - for new clients, epoch changes with every
> > >>>>>> transaction
> > >>>>>>>> and
> > >>>>>>>>>> can
> > >>>>>>>>>>>>>>> overflow,
> > >>>>>>>>>>>>>>>>> in old clients this condition was handled
> > >>>>>> transparently,
> > >>>>>>>>>> because
> > >>>>>>>>>>>> epoch
> > >>>>>>>>>>>>>>>> was
> > >>>>>>>>>>>>>>>>> bumped in InitProducerId and it would return a new
> > >>>>>>>> producer
> > >>>>>>>>>> id if
> > >>>>>>>>>>>>>>> epoch
> > >>>>>>>>>>>>>>>>> overflows, the new clients would need to implement
> > >>>>>> some
> > >>>>>>>>>> workflow
> > >>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>> refresh
> > >>>>>>>>>>>>>>>>> producer id
> > >>>>>>>>>>>>>>>>> - how to handle fenced producers, for new clients
> > >>>>>> epoch
> > >>>>>>>>>> changes
> > >>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>> every
> > >>>>>>>>>>>>>>>>> transaction, so in presence of failures during
> > >>>>>> commits /
> > >>>>>>>>>> aborts,
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>> producer could get easily fenced, old clients would
> > >>>>>>> pretty
> > >>>>>>>>>> much
> > >>>>>>>>>>>> would
> > >>>>>>>>>>>>>>> get
> > >>>>>>>>>>>>>>>>> fenced when a new incarnation of the producer was
> > >>>>>>>>> initialized
> > >>>>>>>>>> with
> > >>>>>>>>>>>>>>>>> InitProducerId so it's ok to treat as a fatal error,
> > >>>>>> the
> > >>>>>>>> new
> > >>>>>>>>>>>> clients
> > >>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>> need to implement some workflow to handle that error,
> > >>>>>>>>>> otherwise
> > >>>>>>>>>>>> they
> > >>>>>>>>>>>>>>>> could
> > >>>>>>>>>>>>>>>>> get fenced by themselves
> > >>>>>>>>>>>>>>>>> - in particular (as a subset of the previous issue),
> > >>>>>>> what
> > >>>>>>>>>> would
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>> client
> > >>>>>>>>>>>>>>>>> do if it got a timeout during commit?  commit could've
> > >>>>>>>>>> succeeded
> > >>>>>>>>>>>> or
> > >>>>>>>>>>>>>>>> failed
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Not sure if this has to be defined in the KIP as
> > >>>>>>>>> implementing
> > >>>>>>>>>>>> those
> > >>>>>>>>>>>>>>>>> probably wouldn't require protocol changes, but we
> > >>>>>> have
> > >>>>>>>>>> multiple
> > >>>>>>>>>>>>>>>>> implementations of Kafka clients, so probably would be
> > >>>>>>>> good
> > >>>>>>>>> to
> > >>>>>>>>>>>> have
> > >>>>>>>>>>>>>>> some
> > >>>>>>>>>>>>>>>>> client implementation guidance.  Could also be done
> > >>>>>> as a
> > >>>>>>>>>> separate
> > >>>>>>>>>>>> doc.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> -Artem
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
> > >>>>>>>>>>>>>>>> <jolshan@confluent.io.invalid
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Hey all, I've updated the KIP to incorporate Jason's
> > >>>>>>>>>>>> suggestions.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>
> > >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> 1. Use AddPartitionsToTxn + verify flag to check on
> > >>>>>>> old
> > >>>>>>>>>> clients
> > >>>>>>>>>>>>>>>>>> 2. Updated AddPartitionsToTxn API to support
> > >>>>>>> transaction
> > >>>>>>>>>>>> batching
> > >>>>>>>>>>>>>>>>>> 3. Mention IBP bump
> > >>>>>>>>>>>>>>>>>> 4. Mention auth change on new AddPartitionsToTxn
> > >>>>>>>> version.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> I'm planning on opening a vote soon.
> > >>>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>>> Justine
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <
> > >>>>>>>>>>>> jolshan@confluent.io
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Thanks Jason. Those changes make sense to me. I
> > >>>>>> will
> > >>>>>>>>>> update
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>> KIP.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
> > >>>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
> > >>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Hey Justine,
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> I was wondering about compatibility here. When
> > >>>>>> we
> > >>>>>>>>> send
> > >>>>>>>>>>>> requests
> > >>>>>>>>>>>>>>>>>>>> between brokers, we want to ensure that the
> > >>>>>>> receiving
> > >>>>>>>>>> broker
> > >>>>>>>>>>>>>>>>> understands
> > >>>>>>>>>>>>>>>>>>>> the request (specifically the new fields).
> > >>>>>>> Typically
> > >>>>>>>>>> this is
> > >>>>>>>>>>>> done
> > >>>>>>>>>>>>>>>> via
> > >>>>>>>>>>>>>>>>>>>> IBP/metadata version.
> > >>>>>>>>>>>>>>>>>>>> I'm trying to think if there is a way around it
> > >>>>>> but
> > >>>>>>>> I'm
> > >>>>>>>>>> not
> > >>>>>>>>>>>> sure
> > >>>>>>>>>>>>>>>> there
> > >>>>>>>>>>>>>>>>>> is.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Yes. I think we would gate usage of this behind
> > >>>>>> an
> > >>>>>>>> IBP
> > >>>>>>>>>> bump.
> > >>>>>>>>>>>> Does
> > >>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>> seem
> > >>>>>>>>>>>>>>>>>>>> reasonable?
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> As for the improvements -- can you clarify how
> > >>>>>>> the
> > >>>>>>>>>> multiple
> > >>>>>>>>>>>>>>>>>>>> transactional
> > >>>>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking of a case
> > >>>>>>>> where
> > >>>>>>>>> we
> > >>>>>>>>>>>>>>> wait/batch
> > >>>>>>>>>>>>>>>>>>>> multiple produce requests together? My
> > >>>>>>> understanding
> > >>>>>>>>> for
> > >>>>>>>>>> now
> > >>>>>>>>>>>> was
> > >>>>>>>>>>>>>>> 1
> > >>>>>>>>>>>>>>>>>>>> transactional ID and one validation per 1 produce
> > >>>>>>>>>> request.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Each call to `AddPartitionsToTxn` is essentially
> > >>>>>> a
> > >>>>>>>>> write
> > >>>>>>>>>> to
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>> log and must block on replication. The more we
> > >>>>>> can
> > >>>>>>>> fit
> > >>>>>>>>>> into a
> > >>>>>>>>>>>>>>> single
> > >>>>>>>>>>>>>>>>>>>> request, the more writes we can do in parallel.
> > >>>>>> The
> > >>>>>>>>>>>> alternative
> > >>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>> make
> > >>>>>>>>>>>>>>>>>>>> use of more connections, but usually we prefer
> > >>>>>>>> batching
> > >>>>>>>>>>>> since the
> > >>>>>>>>>>>>>>>>>> network
> > >>>>>>>>>>>>>>>>>>>> stack is not really optimized for high
> > >>>>>>>>> connection/request
> > >>>>>>>>>>>> loads.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> Finally with respect to the authorizations, I
> > >>>>>>> think
> > >>>>>>>>> it
> > >>>>>>>>>>>> makes
> > >>>>>>>>>>>>>>> sense
> > >>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>> skip
> > >>>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit confused by
> > >>>>>> the
> > >>>>>>>>>> "leader
> > >>>>>>>>>>>> ID"
> > >>>>>>>>>>>>>>>> field.
> > >>>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the request as
> > >>>>>> from a
> > >>>>>>>>>> broker
> > >>>>>>>>>>>> (does
> > >>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>> matter
> > >>>>>>>>>>>>>>>>>>>> which one?).
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> We could also make it version-based. For the next
> > >>>>>>>>>> version, we
> > >>>>>>>>>>>>>>> could
> > >>>>>>>>>>>>>>>>>>>> require
> > >>>>>>>>>>>>>>>>>>>> CLUSTER auth. So clients would not be able to use
> > >>>>>>> the
> > >>>>>>>>> API
> > >>>>>>>>>>>>>>> anymore,
> > >>>>>>>>>>>>>>>>> which
> > >>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>> probably what we want.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> -Jason
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> > >>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > >>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> As a follow up, I was just thinking about the
> > >>>>>>>>> batching
> > >>>>>>>>>> a
> > >>>>>>>>>>>> bit
> > >>>>>>>>>>>>>>> more.
> > >>>>>>>>>>>>>>>>>>>>> I suppose if we have one request in flight and
> > >>>>>> we
> > >>>>>>>>>> queue up
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>> other
> > >>>>>>>>>>>>>>>>>>>>> produce requests in some sort of purgatory, we
> > >>>>>>>> could
> > >>>>>>>>>> send
> > >>>>>>>>>>>>>>>>> information
> > >>>>>>>>>>>>>>>>>>>> out
> > >>>>>>>>>>>>>>>>>>>>> for all of them rather than one by one. So that
> > >>>>>>>> would
> > >>>>>>>>>> be a
> > >>>>>>>>>>>>>>> benefit
> > >>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>> batching partitions to add per transaction.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> I'll need to think a bit more on the design of
> > >>>>>>> this
> > >>>>>>>>>> part
> > >>>>>>>>>>>> of the
> > >>>>>>>>>>>>>>>> KIP,
> > >>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>> will update the KIP in the next few days.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>>>>>> Justine
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan
> > >>>>>> <
> > >>>>>>>>>>>>>>>>> jolshan@confluent.io>
> > >>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> Hey Jason -- thanks for the input -- I was
> > >>>>>> just
> > >>>>>>>>>> digging
> > >>>>>>>>>>>> a bit
> > >>>>>>>>>>>>>>>>> deeper
> > >>>>>>>>>>>>>>>>>>>> into
> > >>>>>>>>>>>>>>>>>>>>>> the design + implementation of the validation
> > >>>>>>>> calls
> > >>>>>>>>>> here
> > >>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>> what
> > >>>>>>>>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>>>> say
> > >>>>>>>>>>>>>>>>>>>>>> makes sense.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> I was wondering about compatibility here.
> > >>>>>> When
> > >>>>>>> we
> > >>>>>>>>>> send
> > >>>>>>>>>>>>>>> requests
> > >>>>>>>>>>>>>>>>>>>>>> between brokers, we want to ensure that the
> > >>>>>>>>> receiving
> > >>>>>>>>>>>> broker
> > >>>>>>>>>>>>>>>>>>>> understands
> > >>>>>>>>>>>>>>>>>>>>>> the request (specifically the new fields).
> > >>>>>>>>> Typically
> > >>>>>>>>>>>> this is
> > >>>>>>>>>>>>>>>> done
> > >>>>>>>>>>>>>>>>>> via
> > >>>>>>>>>>>>>>>>>>>>>> IBP/metadata version.
> > >>>>>>>>>>>>>>>>>>>>>> I'm trying to think if there is a way around
> > >>>>>> it
> > >>>>>>>> but
> > >>>>>>>>>> I'm
> > >>>>>>>>>>>> not
> > >>>>>>>>>>>>>>> sure
> > >>>>>>>>>>>>>>>>>> there
> > >>>>>>>>>>>>>>>>>>>>> is.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> As for the improvements -- can you clarify
> > >>>>>> how
> > >>>>>>>> the
> > >>>>>>>>>>>> multiple
> > >>>>>>>>>>>>>>>>>>>> transactional
> > >>>>>>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking of a
> > >>>>>>> case
> > >>>>>>>>>> where we
> > >>>>>>>>>>>>>>>>> wait/batch
> > >>>>>>>>>>>>>>>>>>>>>> multiple produce requests together? My
> > >>>>>>>>> understanding
> > >>>>>>>>>> for
> > >>>>>>>>>>>> now
> > >>>>>>>>>>>>>>>> was 1
> > >>>>>>>>>>>>>>>>>>>>>> transactional ID and one validation per 1
> > >>>>>>> produce
> > >>>>>>>>>>>> request.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> Finally with respect to the authorizations, I
> > >>>>>>>> think
> > >>>>>>>>>> it
> > >>>>>>>>>>>> makes
> > >>>>>>>>>>>>>>>> sense
> > >>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>> skip
> > >>>>>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit confused
> > >>>>>> by
> > >>>>>>>> the
> > >>>>>>>>>>>> "leader
> > >>>>>>>>>>>>>>> ID"
> > >>>>>>>>>>>>>>>>>> field.
> > >>>>>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the request as
> > >>>>>>>> from a
> > >>>>>>>>>>>> broker
> > >>>>>>>>>>>>>>> (does
> > >>>>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>>>>> matter
> > >>>>>>>>>>>>>>>>>>>>>> which one?).
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> I think I want to adopt these suggestions,
> > >>>>>> just
> > >>>>>>>> had
> > >>>>>>>>>> a few
> > >>>>>>>>>>>>>>>>> questions
> > >>>>>>>>>>>>>>>>>> on
> > >>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>> details.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>>>>>>> Justine
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> On Thu, Jan 5, 2023 at 5:05 PM Jason
> > >>>>>> Gustafson
> > >>>>>>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
> > >>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Thanks for the proposal.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> I was thinking about the implementation a
> > >>>>>>> little
> > >>>>>>>>>> bit.
> > >>>>>>>>>>>> In the
> > >>>>>>>>>>>>>>>>>> current
> > >>>>>>>>>>>>>>>>>>>>>>> proposal, the behavior depends on whether we
> > >>>>>>>> have
> > >>>>>>>>> an
> > >>>>>>>>>>>> old or
> > >>>>>>>>>>>>>>> new
> > >>>>>>>>>>>>>>>>>>>> client.
> > >>>>>>>>>>>>>>>>>>>>>>> For
> > >>>>>>>>>>>>>>>>>>>>>>> old clients, we send `DescribeTransactions`
> > >>>>>>> and
> > >>>>>>>>>> verify
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>> result
> > >>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>> new clients, we send `AddPartitionsToTxn`.
> > >>>>>> We
> > >>>>>>>>> might
> > >>>>>>>>>> be
> > >>>>>>>>>>>> able
> > >>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>> simplify
> > >>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> implementation if we can use the same
> > >>>>>> request
> > >>>>>>>>> type.
> > >>>>>>>>>> For
> > >>>>>>>>>>>>>>>> example,
> > >>>>>>>>>>>>>>>>>>>> what if
> > >>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>> bump the protocol version for
> > >>>>>>>> `AddPartitionsToTxn`
> > >>>>>>>>>> and
> > >>>>>>>>>>>> add a
> > >>>>>>>>>>>>>>>>>>>>>>> `validateOnly`
> > >>>>>>>>>>>>>>>>>>>>>>> flag? For older versions, we can set
> > >>>>>>>>>>>> `validateOnly=true` so
> > >>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> request only returns successfully if the
> > >>>>>>>> partition
> > >>>>>>>>>> had
> > >>>>>>>>>>>>>>> already
> > >>>>>>>>>>>>>>>>> been
> > >>>>>>>>>>>>>>>>>>>>> added.
> > >>>>>>>>>>>>>>>>>>>>>>> For new versions, we can set
> > >>>>>>>> `validateOnly=false`
> > >>>>>>>>>> and
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>> partition
> > >>>>>>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>> added to the transaction. The other slightly
> > >>>>>>>>>> annoying
> > >>>>>>>>>>>> thing
> > >>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>>>>> get around is the need to collect the
> > >>>>>>>> transaction
> > >>>>>>>>>> state
> > >>>>>>>>>>>> for
> > >>>>>>>>>>>>>>> all
> > >>>>>>>>>>>>>>>>>>>>> partitions
> > >>>>>>>>>>>>>>>>>>>>>>> even when we only care about a subset.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Some additional improvements to consider:
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> - We can give `AddPartitionsToTxn` better
> > >>>>>>> batch
> > >>>>>>>>>> support
> > >>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>> inter-broker
> > >>>>>>>>>>>>>>>>>>>>>>> usage. Currently we only allow one
> > >>>>>>>>>> `TransactionalId` to
> > >>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>> specified,
> > >>>>>>>>>>>>>>>>>>>>> but
> > >>>>>>>>>>>>>>>>>>>>>>> the broker may get some benefit being able
> > >>>>>> to
> > >>>>>>>>> batch
> > >>>>>>>>>>>> across
> > >>>>>>>>>>>>>>>>> multiple
> > >>>>>>>>>>>>>>>>>>>>>>> transactions.
> > >>>>>>>>>>>>>>>>>>>>>>> - Another small improvement is skipping
> > >>>>>> topic
> > >>>>>>>>>>>> authorization
> > >>>>>>>>>>>>>>>>> checks
> > >>>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>> `AddPartitionsToTxn` when the request is
> > >>>>>> from
> > >>>>>>> a
> > >>>>>>>>>> broker.
> > >>>>>>>>>>>>>>> Perhaps
> > >>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>> can
> > >>>>>>>>>>>>>>>>>>>>> add
> > >>>>>>>>>>>>>>>>>>>>>>> a field for the `LeaderId` or something like
> > >>>>>>>> that
> > >>>>>>>>>> and
> > >>>>>>>>>>>>>>> require
> > >>>>>>>>>>>>>>>>>> CLUSTER
> > >>>>>>>>>>>>>>>>>>>>>>> permission when set.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>>>>>>>>>> Jason
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
> > >>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation. It makes sense
> > >>>>>>> to
> > >>>>>>>> me
> > >>>>>>>>>> now.
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> Jun
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 1:42 PM Justine
> > >>>>>>> Olshan
> > >>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>> My understanding of the mechanism is
> > >>>>>> that
> > >>>>>>>> when
> > >>>>>>>>>> we
> > >>>>>>>>>>>> get to
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>> last
> > >>>>>>>>>>>>>>>>>>>>>>> epoch,
> > >>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>> increment to the fencing/last epoch and
> > >>>>>> if
> > >>>>>>>> any
> > >>>>>>>>>>>> further
> > >>>>>>>>>>>>>>>>> requests
> > >>>>>>>>>>>>>>>>>>>> come
> > >>>>>>>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>>> this producer ID they are fenced. Then
> > >>>>>> the
> > >>>>>>>>>> producer
> > >>>>>>>>>>>>>>> gets a
> > >>>>>>>>>>>>>>>>> new
> > >>>>>>>>>>>>>>>>>> ID
> > >>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>> restarts with epoch/sequence 0. The
> > >>>>>> fenced
> > >>>>>>>>> epoch
> > >>>>>>>>>>>> sticks
> > >>>>>>>>>>>>>>>>> around
> > >>>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>> duration of producer.id.expiration.ms
> > >>>>>> and
> > >>>>>>>>>> blocks
> > >>>>>>>>>>>> any
> > >>>>>>>>>>>>>>> late
> > >>>>>>>>>>>>>>>>>>>> messages
> > >>>>>>>>>>>>>>>>>>>>>>>> there.
> > >>>>>>>>>>>>>>>>>>>>>>>>> The new ID will get to take advantage of
> > >>>>>>> the
> > >>>>>>>>>>>> improved
> > >>>>>>>>>>>>>>>>> semantics
> > >>>>>>>>>>>>>>>>>>>>> around
> > >>>>>>>>>>>>>>>>>>>>>>>>> non-zero start sequences. So I think we
> > >>>>>>> are
> > >>>>>>>>>> covered.
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>> The only potential issue is overloading
> > >>>>>>> the
> > >>>>>>>>>> cache,
> > >>>>>>>>>>>> but
> > >>>>>>>>>>>>>>>>>> hopefully
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>> improvements (lowered
> > >>>>>>>>> producer.id.expiration.ms
> > >>>>>>>>>> )
> > >>>>>>>>>>>> will
> > >>>>>>>>>>>>>>> help
> > >>>>>>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>>>>>> that.
> > >>>>>>>>>>>>>>>>>>>>>>>> Let
> > >>>>>>>>>>>>>>>>>>>>>>>>> me know if you still have concerns.
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
> > >>>>>>>>>>>>>>>>>>>> <ju...@confluent.io.invalid>
> > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> 70. The proposed fencing logic doesn't
> > >>>>>>>> apply
> > >>>>>>>>>> when
> > >>>>>>>>>>>> pid
> > >>>>>>>>>>>>>>>>>> changes,
> > >>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>>>>>> right? If so, I am not sure how
> > >>>>>> complete
> > >>>>>>>> we
> > >>>>>>>>>> are
> > >>>>>>>>>>>>>>>> addressing
> > >>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>> issue
> > >>>>>>>>>>>>>>>>>>>>>>>> if
> > >>>>>>>>>>>>>>>>>>>>>>>>>> the pid changes more frequently.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 16, 2022 at 9:16 AM
> > >>>>>> Justine
> > >>>>>>>>> Olshan
> > >>>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for replying!
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> 70.We already do the overflow
> > >>>>>>> mechanism,
> > >>>>>>>>> so
> > >>>>>>>>>> my
> > >>>>>>>>>>>>>>> change
> > >>>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>> just
> > >>>>>>>>>>>>>>>>>>>>>>> make
> > >>>>>>>>>>>>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> happen more often.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> I was also not suggesting a new
> > >>>>>> field
> > >>>>>>> in
> > >>>>>>>>> the
> > >>>>>>>>>>>> log,
> > >>>>>>>>>>>>>>> but
> > >>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> response,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> which would be gated by the client
> > >>>>>>>>> version.
> > >>>>>>>>>>>> Sorry if
> > >>>>>>>>>>>>>>>>>>>> something
> > >>>>>>>>>>>>>>>>>>>>>>> there
> > >>>>>>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> unclear. I think we are starting to
> > >>>>>>>>> diverge.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> The goal of this KIP is to not
> > >>>>>> change
> > >>>>>>> to
> > >>>>>>>>> the
> > >>>>>>>>>>>> marker
> > >>>>>>>>>>>>>>>>> format
> > >>>>>>>>>>>>>>>>>> at
> > >>>>>>>>>>>>>>>>>>>>> all.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> 71. Yes, I guess I was going under
> > >>>>>> the
> > >>>>>>>>>>>> assumption
> > >>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> log
> > >>>>>>>>>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>>>>>>> just
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> look at its last epoch and treat it
> > >>>>>> as
> > >>>>>>>> the
> > >>>>>>>>>>>> current
> > >>>>>>>>>>>>>>>>> epoch. I
> > >>>>>>>>>>>>>>>>>>>>>>> suppose
> > >>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>> can
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> have some special logic that if the
> > >>>>>>> last
> > >>>>>>>>>> epoch
> > >>>>>>>>>>>> was
> > >>>>>>>>>>>>>>> on a
> > >>>>>>>>>>>>>>>>>>>> marker
> > >>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>> actually
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> expect the next epoch or something
> > >>>>>>> like
> > >>>>>>>>>> that. We
> > >>>>>>>>>>>>>>> just
> > >>>>>>>>>>>>>>>>> need
> > >>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>> distinguish
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> based on whether we had a
> > >>>>>> commit/abort
> > >>>>>>>>>> marker.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> 72.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> if the producer epoch hasn't been
> > >>>>>>>> bumped
> > >>>>>>>>>> on
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
> > >>>>>>>> message
> > >>>>>>>>>> will
> > >>>>>>>>>>>> fail
> > >>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> sequence
> > >>>>>>>>>>>>>>>>>>>>>>>>>> validation
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the producer
> > >>>>>>>> epoch
> > >>>>>>>>>> has
> > >>>>>>>>>>>> been
> > >>>>>>>>>>>>>>>>> bumped,
> > >>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>> ignore
> > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck message
> > >>>>>>>> could
> > >>>>>>>>>> be
> > >>>>>>>>>>>>>>> appended
> > >>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>> log.
> > >>>>>>>>>>>>>>>>>>>>>>>> So,
> > >>>>>>>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
> > >>>>>> guard?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I follow that "the
> > >>>>>>> message
> > >>>>>>>>> will
> > >>>>>>>>>>>> fail
> > >>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> sequence
> > >>>>>>>>>>>>>>>>>>>>>>>>>> validation".
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> In some of these cases, we had an
> > >>>>>>> abort
> > >>>>>>>>>> marker
> > >>>>>>>>>>>> (due
> > >>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>> an
> > >>>>>>>>>>>>>>>>>>>> error)
> > >>>>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>> then
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> the late message comes in with the
> > >>>>>>>> correct
> > >>>>>>>>>>>> sequence
> > >>>>>>>>>>>>>>>>> number.
> > >>>>>>>>>>>>>>>>>>>> This
> > >>>>>>>>>>>>>>>>>>>>>>> is a
> > >>>>>>>>>>>>>>>>>>>>>>>>>> case
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> covered by the KIP.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> The latter case is actually not
> > >>>>>>>> something
> > >>>>>>>>>> we've
> > >>>>>>>>>>>>>>>>> considered
> > >>>>>>>>>>>>>>>>>>>>> here. I
> > >>>>>>>>>>>>>>>>>>>>>>>>> think
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> generally when we bump the epoch, we
> > >>>>>>> are
> > >>>>>>>>>>>> accepting
> > >>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> sequence
> > >>>>>>>>>>>>>>>>>>>>>>>>> does
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> not need to be checked anymore. My
> > >>>>>>>>>>>> understanding is
> > >>>>>>>>>>>>>>>> also
> > >>>>>>>>>>>>>>>>>>>> that we
> > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> typically bump epoch mid transaction
> > >>>>>>>>> (based
> > >>>>>>>>>> on a
> > >>>>>>>>>>>>>>> quick
> > >>>>>>>>>>>>>>>>> look
> > >>>>>>>>>>>>>>>>>>>> at
> > >>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>> code)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> but let me know if that is the case.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 15, 2022 at 12:23 PM Jun
> > >>>>>>> Rao
> > >>>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the reply.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Assigning a new pid on int
> > >>>>>>>> overflow
> > >>>>>>>>>> seems
> > >>>>>>>>>>>> a
> > >>>>>>>>>>>>>>> bit
> > >>>>>>>>>>>>>>>>>> hacky.
> > >>>>>>>>>>>>>>>>>>>> If
> > >>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>> need a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> level id, it will be better to
> > >>>>>> model
> > >>>>>>>>> this
> > >>>>>>>>>>>>>>> explicitly.
> > >>>>>>>>>>>>>>>>>>>> Adding a
> > >>>>>>>>>>>>>>>>>>>>>>> new
> > >>>>>>>>>>>>>>>>>>>>>>>>>> field
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> would require a bit more work
> > >>>>>> since
> > >>>>>>> it
> > >>>>>>>>>>>> requires a
> > >>>>>>>>>>>>>>> new
> > >>>>>>>>>>>>>>>>> txn
> > >>>>>>>>>>>>>>>>>>>>> marker
> > >>>>>>>>>>>>>>>>>>>>>>>>> format
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the log. So, we probably need to
> > >>>>>>> guard
> > >>>>>>>>> it
> > >>>>>>>>>>>> with an
> > >>>>>>>>>>>>>>> IBP
> > >>>>>>>>>>>>>>>>> or
> > >>>>>>>>>>>>>>>>>>>>>>> metadata
> > >>>>>>>>>>>>>>>>>>>>>>>>>> version
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and document the impact on
> > >>>>>> downgrade
> > >>>>>>>>> once
> > >>>>>>>>>> the
> > >>>>>>>>>>>> new
> > >>>>>>>>>>>>>>>>> format
> > >>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>> written
> > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 71. Hmm, once the marker is
> > >>>>>> written,
> > >>>>>>>> the
> > >>>>>>>>>>>> partition
> > >>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>>>> expect
> > >>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>> next
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> append to be on the next epoch.
> > >>>>>> Does
> > >>>>>>>>> that
> > >>>>>>>>>>>> cover
> > >>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>> case
> > >>>>>>>>>>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>>>>>>>>> mentioned?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> 72. Also, just to be clear on the
> > >>>>>>>>> stucked
> > >>>>>>>>>>>> message
> > >>>>>>>>>>>>>>>> issue
> > >>>>>>>>>>>>>>>>>>>>>>> described
> > >>>>>>>>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> motivation. With EoS, we also
> > >>>>>>> validate
> > >>>>>>>>> the
> > >>>>>>>>>>>>>>> sequence
> > >>>>>>>>>>>>>>>> id
> > >>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>>> idempotency.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> So,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> with the current logic, if the
> > >>>>>>>> producer
> > >>>>>>>>>> epoch
> > >>>>>>>>>>>>>>> hasn't
> > >>>>>>>>>>>>>>>>> been
> > >>>>>>>>>>>>>>>>>>>>>>> bumped on
> > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
> > >>>>>>>>> message
> > >>>>>>>>>> will
> > >>>>>>>>>>>>>>> fail
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>> sequence
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> validation
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the
> > >>>>>> producer
> > >>>>>>>>>> epoch has
> > >>>>>>>>>>>>>>> been
> > >>>>>>>>>>>>>>>>>>>> bumped, we
> > >>>>>>>>>>>>>>>>>>>>>>>> ignore
> > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck
> > >>>>>> message
> > >>>>>>>>>> could be
> > >>>>>>>>>>>>>>>> appended
> > >>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> log.
> > >>>>>>>>>>>>>>>>>>>>>>>>> So,
> > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
> > >>>>>>> guard?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 10:44 AM
> > >>>>>>>> Justine
> > >>>>>>>>>>>> Olshan
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> > >>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias — thanks again for
> > >>>>>> taking
> > >>>>>>>>> time
> > >>>>>>>>>> to
> > >>>>>>>>>>>> look
> > >>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>> this.
> > >>>>>>>>>>>>>>>>>>>> You
> > >>>>>>>>>>>>>>>>>>>>>>> said:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My proposal was only focusing
> > >>>>>> to
> > >>>>>>>>> avoid
> > >>>>>>>>>>>>>>> dangling
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
> > >>>>>> added
> > >>>>>>>>>> without
> > >>>>>>>>>>>>>>>> registered
> > >>>>>>>>>>>>>>>>>>>>>>> partition.
> > >>>>>>>>>>>>>>>>>>>>>>>> --
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more details
> > >>>>>> to
> > >>>>>>>> the
> > >>>>>>>>>> KIP
> > >>>>>>>>>>>> about
> > >>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>> scenario
> > >>>>>>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> better
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I understand what
> > >>>>>> you
> > >>>>>>>>> mean
> > >>>>>>>>>>>> here.
> > >>>>>>>>>>>>>>> The
> > >>>>>>>>>>>>>>>>>>>> motivation
> > >>>>>>>>>>>>>>>>>>>>>>>>> section
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> describes two scenarios about
> > >>>>>> how
> > >>>>>>>> the
> > >>>>>>>>>> record
> > >>>>>>>>>>>>>>> can be
> > >>>>>>>>>>>>>>>>>> added
> > >>>>>>>>>>>>>>>>>>>>>>>> without a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> registered partition:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Another way hanging
> > >>>>>> transactions
> > >>>>>>>> can
> > >>>>>>>>>>>> occur is
> > >>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>> client
> > >>>>>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>>> buggy
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> may somehow try to write to a
> > >>>>>>>>> partition
> > >>>>>>>>>>>> before
> > >>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>> adds
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>> partition
> > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> For the first example of this
> > >>>>>>> would
> > >>>>>>>> it
> > >>>>>>>>>> be
> > >>>>>>>>>>>>>>> helpful
> > >>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>> say
> > >>>>>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> comes in after the abort, but
> > >>>>>>> before
> > >>>>>>>>> the
> > >>>>>>>>>>>>>>> partition
> > >>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>> added
> > >>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>> next
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction so it becomes
> > >>>>>>> "hanging."
> > >>>>>>>>>>>> Perhaps the
> > >>>>>>>>>>>>>>>> next
> > >>>>>>>>>>>>>>>>>>>>> sentence
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> describing
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the message becoming part of the
> > >>>>>>>> next
> > >>>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>> (a
> > >>>>>>>>>>>>>>>>>>>>> different
> > >>>>>>>>>>>>>>>>>>>>>>>>> case)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> was
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not properly differentiated.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun — thanks for reading the
> > >>>>>> KIP.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. The int typing was a
> > >>>>>> concern.
> > >>>>>>>>>> Currently
> > >>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>> have a
> > >>>>>>>>>>>>>>>>>>>>>>> mechanism
> > >>>>>>>>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> place
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> fence the final epoch when the
> > >>>>>>> epoch
> > >>>>>>>>> is
> > >>>>>>>>>>>> about to
> > >>>>>>>>>>>>>>>>>> overflow
> > >>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>> assign
> > >>>>>>>>>>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer ID with epoch 0. Of
> > >>>>>>> course,
> > >>>>>>>>>> this
> > >>>>>>>>>>>> is a
> > >>>>>>>>>>>>>>> bit
> > >>>>>>>>>>>>>>>>>> tricky
> > >>>>>>>>>>>>>>>>>>>>>>> when it
> > >>>>>>>>>>>>>>>>>>>>>>>>>> comes
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response back to the client.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Making this a long could be
> > >>>>>>> another
> > >>>>>>>>>> option,
> > >>>>>>>>>>>> but
> > >>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>> wonder
> > >>>>>>>>>>>>>>>>>>>> are
> > >>>>>>>>>>>>>>>>>>>>>>>> there
> > >>>>>>>>>>>>>>>>>>>>>>>>>> any
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> implications on changing this
> > >>>>>>> field
> > >>>>>>>> if
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>>> epoch is
> > >>>>>>>>>>>>>>>>>>>>> persisted
> > >>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>> disk?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to check the usages.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 71.This was something Matthias
> > >>>>>>> asked
> > >>>>>>>>>> about
> > >>>>>>>>>>>> as
> > >>>>>>>>>>>>>>>> well. I
> > >>>>>>>>>>>>>>>>>> was
> > >>>>>>>>>>>>>>>>>>>>>>>>>> considering a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible edge case where a
> > >>>>>> produce
> > >>>>>>>>>> request
> > >>>>>>>>>>>> from
> > >>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>> new
> > >>>>>>>>>>>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> somehow
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> gets sent right after the marker
> > >>>>>>> is
> > >>>>>>>>>>>> written, but
> > >>>>>>>>>>>>>>>>> before
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>> producer
> > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> alerted of the newly bumped
> > >>>>>> epoch.
> > >>>>>>>> In
> > >>>>>>>>>> this
> > >>>>>>>>>>>>>>> case, we
> > >>>>>>>>>>>>>>>>> may
> > >>>>>>>>>>>>>>>>>>>>>>> include
> > >>>>>>>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> record
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we don't want to. I suppose
> > >>>>>>> we
> > >>>>>>>>>> could
> > >>>>>>>>>>>> try
> > >>>>>>>>>>>>>>> to do
> > >>>>>>>>>>>>>>>>>>>>> something
> > >>>>>>>>>>>>>>>>>>>>>>>>> client
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> side
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to bump the epoch after sending
> > >>>>>> an
> > >>>>>>>>>> endTxn as
> > >>>>>>>>>>>>>>> well
> > >>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>>> scenario
> > >>>>>>>>>>>>>>>>>>>>>>>>> —
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wonder how it would work when
> > >>>>>> the
> > >>>>>>>>>> server is
> > >>>>>>>>>>>>>>>> aborting
> > >>>>>>>>>>>>>>>>>>>> based
> > >>>>>>>>>>>>>>>>>>>>> on
> > >>>>>>>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> error. I could also be missing
> > >>>>>>>>>> something and
> > >>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>> scenario
> > >>>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>>> actually
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again to everyone reading
> > >>>>>>> and
> > >>>>>>>>>>>> commenting.
> > >>>>>>>>>>>>>>>> Let
> > >>>>>>>>>>>>>>>>> me
> > >>>>>>>>>>>>>>>>>>>> know
> > >>>>>>>>>>>>>>>>>>>>>>>> about
> > >>>>>>>>>>>>>>>>>>>>>>>>>> any
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> further questions or comments.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 9:41 AM
> > >>>>>>> Jun
> > >>>>>>>>> Rao
> > >>>>>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP. A couple
> > >>>>>> of
> > >>>>>>>>>> comments.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Currently, the producer
> > >>>>>>> epoch
> > >>>>>>>> is
> > >>>>>>>>>> an
> > >>>>>>>>>>>> int.
> > >>>>>>>>>>>>>>> I am
> > >>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>> sure
> > >>>>>>>>>>>>>>>>>>>>>>> if
> > >>>>>>>>>>>>>>>>>>>>>>>>> it's
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> enough
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to accommodate all
> > >>>>>> transactions
> > >>>>>>> in
> > >>>>>>>>> the
> > >>>>>>>>>>>>>>> lifetime
> > >>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>>>> producer.
> > >>>>>>>>>>>>>>>>>>>>>>>>>> Should
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> change that to a long or add a
> > >>>>>>> new
> > >>>>>>>>>> long
> > >>>>>>>>>>>> field
> > >>>>>>>>>>>>>>>> like
> > >>>>>>>>>>>>>>>>>>>> txnId?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 71. "it will write the prepare
> > >>>>>>>>> commit
> > >>>>>>>>>>>> message
> > >>>>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>> bumped
> > >>>>>>>>>>>>>>>>>>>>>>>> epoch
> > >>>>>>>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> send
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WriteTxnMarkerRequests with
> > >>>>>> the
> > >>>>>>>>> bumped
> > >>>>>>>>>>>> epoch."
> > >>>>>>>>>>>>>>>> Hmm,
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > >>>>>>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> associated
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the current txn right?
> > >>>>>> So,
> > >>>>>>> it
> > >>>>>>>>>> seems
> > >>>>>>>>>>>>>>> weird to
> > >>>>>>>>>>>>>>>>>>>> write a
> > >>>>>>>>>>>>>>>>>>>>>>>> commit
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with a bumped epoch. Should we
> > >>>>>>>> only
> > >>>>>>>>>> bump
> > >>>>>>>>>>>> up
> > >>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>> epoch
> > >>>>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> EndTxnResponse
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rename the field to sth like
> > >>>>>>>>>>>>>>> nextProducerEpoch?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 12, 2022 at 8:54
> > >>>>>> PM
> > >>>>>>>>>> Matthias
> > >>>>>>>>>>>> J.
> > >>>>>>>>>>>>>>> Sax <
> > >>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the background.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30: SGTM. My proposal was
> > >>>>>>>> only
> > >>>>>>>>>>>> focusing
> > >>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>> avoid
> > >>>>>>>>>>>>>>>>>>>>>>> dangling
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
> > >>>>>>>> added
> > >>>>>>>>>>>> without
> > >>>>>>>>>>>>>>>>>> registered
> > >>>>>>>>>>>>>>>>>>>>>>>> partition.
> > >>>>>>>>>>>>>>>>>>>>>>>>>> --
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more
> > >>>>>> details
> > >>>>>>>> to
> > >>>>>>>>>> the
> > >>>>>>>>>>>> KIP
> > >>>>>>>>>>>>>>> about
> > >>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>> scenario
> > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40: I think you hit a fair
> > >>>>>>> point
> > >>>>>>>>>> about
> > >>>>>>>>>>>> race
> > >>>>>>>>>>>>>>>>>>>> conditions
> > >>>>>>>>>>>>>>>>>>>>> or
> > >>>>>>>>>>>>>>>>>>>>>>>>> client
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> bugs
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (incorrectly not bumping the
> > >>>>>>>>>> epoch). The
> > >>>>>>>>>>>>>>>>>>>>>>> complexity/confusion
> > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> using
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the bumped epoch I see, is
> > >>>>>>>> mainly
> > >>>>>>>>>> for
> > >>>>>>>>>>>>>>> internal
> > >>>>>>>>>>>>>>>>>>>>> debugging,
> > >>>>>>>>>>>>>>>>>>>>>>> ie,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> inspecting
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log segment dumps -- it
> > >>>>>> seems
> > >>>>>>>>>> harder to
> > >>>>>>>>>>>>>>> reason
> > >>>>>>>>>>>>>>>>>> about
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> system
> > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> us
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> humans. But if we get better
> > >>>>>>>>>>>> guarantees, it
> > >>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>> worth to
> > >>>>>>>>>>>>>>>>>>>>>>>>> use
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped epoch.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60: as I mentioned already,
> > >>>>>> I
> > >>>>>>>>> don't
> > >>>>>>>>>>>> know the
> > >>>>>>>>>>>>>>>>> broker
> > >>>>>>>>>>>>>>>>>>>>>>> internals
> > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> provide
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more input. So if nobody
> > >>>>>> else
> > >>>>>>>>> chimes
> > >>>>>>>>>>>> in, we
> > >>>>>>>>>>>>>>>>> should
> > >>>>>>>>>>>>>>>>>>>> just
> > >>>>>>>>>>>>>>>>>>>>>>> move
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> forward
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with your proposal.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 12/6/22 4:22 PM, Justine
> > >>>>>>>> Olshan
> > >>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi all,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> After Artem's questions
> > >>>>>>> about
> > >>>>>>>>>> error
> > >>>>>>>>>>>>>>> behavior,
> > >>>>>>>>>>>>>>>>>> I've
> > >>>>>>>>>>>>>>>>>>>>>>>>> re-evaluated
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unknown producer ID
> > >>>>>>> exception
> > >>>>>>>>> and
> > >>>>>>>>>> had
> > >>>>>>>>>>>> some
> > >>>>>>>>>>>>>>>>>>>> discussions
> > >>>>>>>>>>>>>>>>>>>>>>>>> offline.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think generally it makes
> > >>>>>>>> sense
> > >>>>>>>>>> to
> > >>>>>>>>>>>>>>> simplify
> > >>>>>>>>>>>>>>>>>> error
> > >>>>>>>>>>>>>>>>>>>>>>> handling
> > >>>>>>>>>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> cases
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this and the
> > >>>>>>>> UNKNOWN_PRODUCER_ID
> > >>>>>>>>>> error
> > >>>>>>>>>>>>>>> has a
> > >>>>>>>>>>>>>>>>>> pretty
> > >>>>>>>>>>>>>>>>>>>>> long
> > >>>>>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complicated
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> history. Because of this,
> > >>>>>> I
> > >>>>>>>>>> propose
> > >>>>>>>>>>>>>>> adding a
> > >>>>>>>>>>>>>>>>> new
> > >>>>>>>>>>>>>>>>>>>> error
> > >>>>>>>>>>>>>>>>>>>>>>> code
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ABORTABLE_ERROR
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that when encountered by
> > >>>>>> new
> > >>>>>>>>>> clients
> > >>>>>>>>>>>>>>> (gated
> > >>>>>>>>>>>>>>>> by
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> produce
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will simply abort the
> > >>>>>>>>> transaction.
> > >>>>>>>>>>>> This
> > >>>>>>>>>>>>>>>> allows
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> server
> > >>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>> have
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> some
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> say
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in whether the client
> > >>>>>> aborts
> > >>>>>>>> and
> > >>>>>>>>>> makes
> > >>>>>>>>>>>>>>>> handling
> > >>>>>>>>>>>>>>>>>>>> much
> > >>>>>>>>>>>>>>>>>>>>>>>> simpler.
> > >>>>>>>>>>>>>>>>>>>>>>>>>> In
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> future, we can also use
> > >>>>>> this
> > >>>>>>>>>> error in
> > >>>>>>>>>>>>>>> other
> > >>>>>>>>>>>>>>>>>>>> situations
> > >>>>>>>>>>>>>>>>>>>>>>>> where
> > >>>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> want
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> abort the transactions. We
> > >>>>>>> can
> > >>>>>>>>>> even
> > >>>>>>>>>>>> use on
> > >>>>>>>>>>>>>>>>> other
> > >>>>>>>>>>>>>>>>>>>> apis.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've added this to the
> > >>>>>> KIP.
> > >>>>>>>> Let
> > >>>>>>>>> me
> > >>>>>>>>>>>> know if
> > >>>>>>>>>>>>>>>>> there
> > >>>>>>>>>>>>>>>>>>>> are
> > >>>>>>>>>>>>>>>>>>>>> any
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> questions
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> or
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 2, 2022 at
> > >>>>>> 10:22
> > >>>>>>>> AM
> > >>>>>>>>>>>> Justine
> > >>>>>>>>>>>>>>>> Olshan
> > >>>>>>>>>>>>>>>>> <
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> jolshan@confluent.io
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey Matthias,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30 — Maybe I also
> > >>>>>> didn't
> > >>>>>>>>>> express
> > >>>>>>>>>>>>>>> myself
> > >>>>>>>>>>>>>>>>>>>> clearly.
> > >>>>>>>>>>>>>>>>>>>>> For
> > >>>>>>>>>>>>>>>>>>>>>>>>> older
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't have a way to
> > >>>>>>>> distinguish
> > >>>>>>>>>>>> between a
> > >>>>>>>>>>>>>>>>>> previous
> > >>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> current
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction since we
> > >>>>>> don't
> > >>>>>>>> have
> > >>>>>>>>>> the
> > >>>>>>>>>>>> epoch
> > >>>>>>>>>>>>>>>>> bump.
> > >>>>>>>>>>>>>>>>>>>> This
> > >>>>>>>>>>>>>>>>>>>>>>> means
> > >>>>>>>>>>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message from the previous
> > >>>>>>>>>> transaction
> > >>>>>>>>>>>>>>> may be
> > >>>>>>>>>>>>>>>>>>>> added to
> > >>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>> new
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> one.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> With
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older clients — we can't
> > >>>>>>>>>> guarantee
> > >>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>> won't
> > >>>>>>>>>>>>>>>>>>>> happen
> > >>>>>>>>>>>>>>>>>>>>>>> if we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> already
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call
> > >>>>>>> (why
> > >>>>>>>> we
> > >>>>>>>>>> make
> > >>>>>>>>>>>>>>> changes
> > >>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> newer
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> client)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can at least gate some by
> > >>>>>>>>>> ensuring
> > >>>>>>>>>>>> that
> > >>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> partition
> > >>>>>>>>>>>>>>>>>>>>>>> has
> > >>>>>>>>>>>>>>>>>>>>>>>>> been
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> added
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. The
> > >>>>>> rationale
> > >>>>>>>> here
> > >>>>>>>>>> is
> > >>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>> there
> > >>>>>>>>>>>>>>>>>> are
> > >>>>>>>>>>>>>>>>>>>>>>> likely
> > >>>>>>>>>>>>>>>>>>>>>>>>> LESS
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrivals
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as time goes on, so
> > >>>>>>> hopefully
> > >>>>>>>>>> most
> > >>>>>>>>>>>> late
> > >>>>>>>>>>>>>>>>> arrivals
> > >>>>>>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>>>>>>> come
> > >>>>>>>>>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> BEFORE
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call.
> > >>>>>>>> Those
> > >>>>>>>>>> that
> > >>>>>>>>>>>>>>> arrive
> > >>>>>>>>>>>>>>>>>> before
> > >>>>>>>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> properly
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> gated
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> > >>>>>>> describeTransactions
> > >>>>>>>>>>>> approach.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we take the approach
> > >>>>>> you
> > >>>>>>>>>>>> suggested,
> > >>>>>>>>>>>>>>> ANY
> > >>>>>>>>>>>>>>>>> late
> > >>>>>>>>>>>>>>>>>>>>> arrival
> > >>>>>>>>>>>>>>>>>>>>>>>>> from a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> previous
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction will be
> > >>>>>> added.
> > >>>>>>>> And
> > >>>>>>>>> we
> > >>>>>>>>>>>> don't
> > >>>>>>>>>>>>>>> want
> > >>>>>>>>>>>>>>>>>>>> that. I
> > >>>>>>>>>>>>>>>>>>>>>>> also
> > >>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> see
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> benefit in sending
> > >>>>>>>>>> addPartitionsToTxn
> > >>>>>>>>>>>>>>> over
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> describeTxns
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> They
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> both be one extra RPC to
> > >>>>>>> the
> > >>>>>>>>> Txn
> > >>>>>>>>>>>>>>>> coordinator.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To be clear — newer
> > >>>>>> clients
> > >>>>>>>>> will
> > >>>>>>>>>> use
> > >>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > >>>>>>>>>>>>>>>>>>>>>>>>>> instead
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTxns.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that if we
> > >>>>>>> have
> > >>>>>>>>>> some
> > >>>>>>>>>>>> delay
> > >>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>> client
> > >>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>> bump
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it could continue to send
> > >>>>>>>> epoch
> > >>>>>>>>>> 73
> > >>>>>>>>>>>> and
> > >>>>>>>>>>>>>>> those
> > >>>>>>>>>>>>>>>>>>>> records
> > >>>>>>>>>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fenced.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Perhaps this is not an
> > >>>>>>> issue
> > >>>>>>>> if
> > >>>>>>>>>> we
> > >>>>>>>>>>>> don't
> > >>>>>>>>>>>>>>>> allow
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>> next
> > >>>>>>>>>>>>>>>>>>>>>>>>>> produce
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> go
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through before the EndTxn
> > >>>>>>>>> request
> > >>>>>>>>>>>>>>> returns.
> > >>>>>>>>>>>>>>>> I'm
> > >>>>>>>>>>>>>>>>>>>> also
> > >>>>>>>>>>>>>>>>>>>>>>>> thinking
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> about
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cases of
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure. I will need to
> > >>>>>>> think
> > >>>>>>>>> on
> > >>>>>>>>>>>> this a
> > >>>>>>>>>>>>>>> bit.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I wasn't sure if it was
> > >>>>>>> that
> > >>>>>>>>>>>> confusing.
> > >>>>>>>>>>>>>>> But
> > >>>>>>>>>>>>>>>> if
> > >>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>> think it
> > >>>>>>>>>>>>>>>>>>>>>>>>> is,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> investigate other ways.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure these are
> > >>>>>> the
> > >>>>>>>> same
> > >>>>>>>>>>>>>>> purgatories
> > >>>>>>>>>>>>>>>>>> since
> > >>>>>>>>>>>>>>>>>>>> one
> > >>>>>>>>>>>>>>>>>>>>>>> is a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> produce
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory (I was planning
> > >>>>>>> on
> > >>>>>>>>>> using a
> > >>>>>>>>>>>>>>>> callback
> > >>>>>>>>>>>>>>>>>>>> rather
> > >>>>>>>>>>>>>>>>>>>>>>> than
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the other is simply a
> > >>>>>>> request
> > >>>>>>>>> to
> > >>>>>>>>>>>> append
> > >>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> log.
> > >>>>>>>>>>>>>>>>>>>>> Not
> > >>>>>>>>>>>>>>>>>>>>>>>> sure
> > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> structure here for
> > >>>>>>> ordering,
> > >>>>>>>>> but
> > >>>>>>>>>> my
> > >>>>>>>>>>>>>>>>>> understanding
> > >>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handle the write request
> > >>>>>>>> before
> > >>>>>>>>>> it
> > >>>>>>>>>>>> hears
> > >>>>>>>>>>>>>>>> back
> > >>>>>>>>>>>>>>>>>> from
> > >>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> Txn
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Coordinator.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if I
> > >>>>>>>> misunderstood
> > >>>>>>>>>>>> something
> > >>>>>>>>>>>>>>> or
> > >>>>>>>>>>>>>>>>>>>> something
> > >>>>>>>>>>>>>>>>>>>>>>> was
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> unclear.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 1, 2022 at
> > >>>>>>> 12:15
> > >>>>>>>> PM
> > >>>>>>>>>>>> Matthias
> > >>>>>>>>>>>>>>> J.
> > >>>>>>>>>>>>>>>>> Sax
> > >>>>>>>>>>>>>>>>>> <
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the details
> > >>>>>>>>> Justine!
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side change
> > >>>>>>> for
> > >>>>>>>> 2
> > >>>>>>>>> is
> > >>>>>>>>>>>>>>> removing
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> addPartitions
> > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need to
> > >>>>>>> make
> > >>>>>>>>>> this
> > >>>>>>>>>>>> from
> > >>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> producer
> > >>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think I did not
> > >>>>>> express
> > >>>>>>>>> myself
> > >>>>>>>>>>>>>>> clearly. I
> > >>>>>>>>>>>>>>>>>>>>> understand
> > >>>>>>>>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should) change the
> > >>>>>>> producer
> > >>>>>>>> to
> > >>>>>>>>>> not
> > >>>>>>>>>>>> send
> > >>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> longer. But I don't
> > >>>>>> thinks
> > >>>>>>>>> it's
> > >>>>>>>>>>>>>>> requirement
> > >>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>> change
> > >>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> broker?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What I am trying to say
> > >>>>>>> is:
> > >>>>>>>>> as a
> > >>>>>>>>>>>>>>> safe-guard
> > >>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>> improvement
> > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producers, the partition
> > >>>>>>>>> leader
> > >>>>>>>>>> can
> > >>>>>>>>>>>> just
> > >>>>>>>>>>>>>>>> send
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request to the
> > >>>>>>>> TX-coordinator
> > >>>>>>>>>> in any
> > >>>>>>>>>>>>>>> case
> > >>>>>>>>>>>>>>>> --
> > >>>>>>>>>>>>>>>>> if
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> old
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> correctly did send the
> > >>>>>>>>>>>> `addPartition`
> > >>>>>>>>>>>>>>>> request
> > >>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> already, the
> > >>>>>>> TX-coordinator
> > >>>>>>>>> can
> > >>>>>>>>>> just
> > >>>>>>>>>>>>>>>> "ignore"
> > >>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>> as
> > >>>>>>>>>>>>>>>>>>>>>>>>>> idempotent.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if the old producer has
> > >>>>>> a
> > >>>>>>>> bug
> > >>>>>>>>>> and
> > >>>>>>>>>>>> did
> > >>>>>>>>>>>>>>>> forget
> > >>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>> sent
> > >>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `addPartition`
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request, we would now
> > >>>>>>> ensure
> > >>>>>>>>>> that
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>> partition
> > >>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>> indeed
> > >>>>>>>>>>>>>>>>>>>>>>>>>> added
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX and thus fix a
> > >>>>>>> potential
> > >>>>>>>>>>>> producer bug
> > >>>>>>>>>>>>>>>>> (even
> > >>>>>>>>>>>>>>>>>>>> if we
> > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > >>>>>>>>>>>>>>>>>>>>>>>>>> get
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fencing via the bump
> > >>>>>>> epoch).
> > >>>>>>>>> --
> > >>>>>>>>>> It
> > >>>>>>>>>>>>>>> seems to
> > >>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>> good
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Or
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there a reason to not do
> > >>>>>>>> this?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is ongoing
> > >>>>>> =
> > >>>>>>>>>> partition
> > >>>>>>>>>>>> was
> > >>>>>>>>>>>>>>>> added
> > >>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> via
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn. We
> > >>>>>>>> check
> > >>>>>>>>>> this
> > >>>>>>>>>>>> with
> > >>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this wasn't
> > >>>>>>>>>> sufficiently
> > >>>>>>>>>>>>>>>>> explained
> > >>>>>>>>>>>>>>>>>>>> here:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do what I propose
> > >>>>>> in
> > >>>>>>>>>> (20), we
> > >>>>>>>>>>>>>>> don't
> > >>>>>>>>>>>>>>>>>> really
> > >>>>>>>>>>>>>>>>>>>>> need
> > >>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>> make
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `DescribeTransaction`
> > >>>>>>> call,
> > >>>>>>>> as
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>>>> partition
> > >>>>>>>>>>>>>>>>>>>> leader
> > >>>>>>>>>>>>>>>>>>>>>>> adds
> > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for older clients and we
> > >>>>>>> get
> > >>>>>>>>>> this
> > >>>>>>>>>>>> check
> > >>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>> free.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is that
> > >>>>>> if
> > >>>>>>>> any
> > >>>>>>>>>>>> messages
> > >>>>>>>>>>>>>>>>> somehow
> > >>>>>>>>>>>>>>>>>>>> come
> > >>>>>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>>>>>> before
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the producer,
> > >>>>>>> they
> > >>>>>>>>>> will be
> > >>>>>>>>>>>>>>>> fenced.
> > >>>>>>>>>>>>>>>>>>>>> However,
> > >>>>>>>>>>>>>>>>>>>>>>> if
> > >>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it can be
> > >>>>>>>>>> discussed
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree that we should
> > >>>>>>> have
> > >>>>>>>>>> epoch
> > >>>>>>>>>>>>>>> fencing.
> > >>>>>>>>>>>>>>>> My
> > >>>>>>>>>>>>>>>>>>>>>>> question is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> different:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Assume we are at epoch
> > >>>>>> 73,
> > >>>>>>>> and
> > >>>>>>>>>> we
> > >>>>>>>>>>>> have
> > >>>>>>>>>>>>>>> an
> > >>>>>>>>>>>>>>>>>> ongoing
> > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed. It seems
> > >>>>>>> natural
> > >>>>>>>> to
> > >>>>>>>>>>>> write the
> > >>>>>>>>>>>>>>>>>> "prepare
> > >>>>>>>>>>>>>>>>>>>>>>> commit"
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> marker
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
> > >>>>>>> both
> > >>>>>>>>> with
> > >>>>>>>>>>>> epoch
> > >>>>>>>>>>>>>>> 73,
> > >>>>>>>>>>>>>>>>> too,
> > >>>>>>>>>>>>>>>>>>>> as
> > >>>>>>>>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>>>>>>>>> belongs
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current transaction. Of
> > >>>>>>>>> course,
> > >>>>>>>>>> we
> > >>>>>>>>>>>> now
> > >>>>>>>>>>>>>>> also
> > >>>>>>>>>>>>>>>>>> bump
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> epoch
> > >>>>>>>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> expect
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the next requests to
> > >>>>>> have
> > >>>>>>>>> epoch
> > >>>>>>>>>> 74,
> > >>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>> reject
> > >>>>>>>>>>>>>>>>>>>>>>> an
> > >>>>>>>>>>>>>>>>>>>>>>>>>> request
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch 73, as the
> > >>>>>>>> corresponding
> > >>>>>>>>>> TX
> > >>>>>>>>>>>> for
> > >>>>>>>>>>>>>>> epoch
> > >>>>>>>>>>>>>>>>> 73
> > >>>>>>>>>>>>>>>>>>>> was
> > >>>>>>>>>>>>>>>>>>>>>>>> already
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems you propose to
> > >>>>>>>> write
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>>> "prepare
> > >>>>>>>>>>>>>>>>>> commit
> > >>>>>>>>>>>>>>>>>>>>>>> marker"
> > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
> > >>>>>>> with
> > >>>>>>>>>> epoch 74
> > >>>>>>>>>>>>>>>> though,
> > >>>>>>>>>>>>>>>>>> what
> > >>>>>>>>>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>>>>>>> work,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems confusing. Is
> > >>>>>> there
> > >>>>>>> a
> > >>>>>>>>>> reason
> > >>>>>>>>>>>> why
> > >>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>> use
> > >>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>> bumped
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 74
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of the current
> > >>>>>>> epoch
> > >>>>>>>>> 73?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are checking if
> > >>>>>>> the
> > >>>>>>>>>>>>>>> transaction is
> > >>>>>>>>>>>>>>>>>>>> ongoing,
> > >>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>> need
> > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the leader
> > >>>>>>>>> partition
> > >>>>>>>>>> to
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for this
> > >>>>>>>>>> message to
> > >>>>>>>>>>>> come
> > >>>>>>>>>>>>>>>>> back,
> > >>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>>>> theory
> > >>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
> > >>>>>> that
> > >>>>>>>>> would
> > >>>>>>>>>>>> make the
> > >>>>>>>>>>>>>>>>>> original
> > >>>>>>>>>>>>>>>>>>>>>>> result
> > >>>>>>>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out of
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why we
> > >>>>>> can
> > >>>>>>>>> check
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>>> leader
> > >>>>>>>>>>>>>>>>>> state
> > >>>>>>>>>>>>>>>>>>>>>>> before
> > >>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks. Got it.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, is this really
> > >>>>>> an
> > >>>>>>>>>> issue?
> > >>>>>>>>>>>> We put
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> produce
> > >>>>>>>>>>>>>>>>>>>>>>>>> request
> > >>>>>>>>>>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory, so how could
> > >>>>>> we
> > >>>>>>>>>> process
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> first?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Don't we need to put the
> > >>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
> > >>>>>>>>>>>>>>>>>>>>> into
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> too,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for this case, and
> > >>>>>> process
> > >>>>>>>>> both
> > >>>>>>>>>>>> request
> > >>>>>>>>>>>>>>>>>> in-order?
> > >>>>>>>>>>>>>>>>>>>>>>> (Again,
> > >>>>>>>>>>>>>>>>>>>>>>>>> my
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> knowledge is limited and
> > >>>>>>>> maybe
> > >>>>>>>>>> we
> > >>>>>>>>>>>> don't
> > >>>>>>>>>>>>>>>>>> maintain
> > >>>>>>>>>>>>>>>>>>>>>>> request
> > >>>>>>>>>>>>>>>>>>>>>>>>>> order
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case, what seems to be
> > >>>>>> an
> > >>>>>>>>> issue
> > >>>>>>>>>>>> IMHO,
> > >>>>>>>>>>>>>>> and I
> > >>>>>>>>>>>>>>>>> am
> > >>>>>>>>>>>>>>>>>>>>>>> wondering
> > >>>>>>>>>>>>>>>>>>>>>>>> if
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> changing
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request handling to
> > >>>>>>> preserve
> > >>>>>>>>>> order
> > >>>>>>>>>>>> for
> > >>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>> case
> > >>>>>>>>>>>>>>>>>>>>>>> might be
> > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> cleaner
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution?)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/30/22 3:28 PM,
> > >>>>>> Artem
> > >>>>>>>>>> Livshits
> > >>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think the interesting
> > >>>>>>>> part
> > >>>>>>>>> is
> > >>>>>>>>>>>> not in
> > >>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>> logic
> > >>>>>>>>>>>>>>>>>>>>>>>> (because
> > >>>>>>>>>>>>>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> tries
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> figure out when
> > >>>>>>>>>>>> UNKNOWN_PRODUCER_ID is
> > >>>>>>>>>>>>>>>>>> retriable
> > >>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>> if
> > >>>>>>>>>>>>>>>>>>>>>>>>> it's
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retryable,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's definitely not
> > >>>>>>> fatal),
> > >>>>>>>>> but
> > >>>>>>>>>>>> what
> > >>>>>>>>>>>>>>>> happens
> > >>>>>>>>>>>>>>>>>>>> when
> > >>>>>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>>>> logic
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'true' and falls
> > >>>>>> through.
> > >>>>>>>> In
> > >>>>>>>>>> the
> > >>>>>>>>>>>> old
> > >>>>>>>>>>>>>>>>> clients
> > >>>>>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>>>>>>> seems
> > >>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> keep the behavior in
> > >>>>>> the
> > >>>>>>>> new
> > >>>>>>>>>>>> clients,
> > >>>>>>>>>>>>>>> I'd
> > >>>>>>>>>>>>>>>>>>>> expect it
> > >>>>>>>>>>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> well.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 29, 2022 at
> > >>>>>>>> 11:57
> > >>>>>>>>>> AM
> > >>>>>>>>>>>>>>> Justine
> > >>>>>>>>>>>>>>>>>> Olshan
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>> <jolshan@confluent.io.invalid
> > >>>>>>>>>>
> > >>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Artem and Jeff,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
> > >>>>>> look
> > >>>>>>>> and
> > >>>>>>>>>>>> sorry for
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>> slow
> > >>>>>>>>>>>>>>>>>>>>>>>> response.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You both mentioned the
> > >>>>>>>>> change
> > >>>>>>>>>> to
> > >>>>>>>>>>>>>>> handle
> > >>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear — this error
> > >>>>>> code
> > >>>>>>>> will
> > >>>>>>>>>> only
> > >>>>>>>>>>>> be
> > >>>>>>>>>>>>>>> sent
> > >>>>>>>>>>>>>>>>>> again
> > >>>>>>>>>>>>>>>>>>>>> when
> > >>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> client's
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version is high enough
> > >>>>>>> to
> > >>>>>>>>>> ensure
> > >>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>> handle
> > >>>>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>>>>>>>> correctly.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The current (Java)
> > >>>>>>> client
> > >>>>>>>>>> handles
> > >>>>>>>>>>>>>>> this by
> > >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> following
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> (somewhat
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> long)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code snippet:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // An
> > >>>>>>> UNKNOWN_PRODUCER_ID
> > >>>>>>>>>> means
> > >>>>>>>>>>>> that
> > >>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>> have
> > >>>>>>>>>>>>>>>>>>>> lost
> > >>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker. Depending on
> > >>>>>> the
> > >>>>>>>> log
> > >>>>>>>>>> start
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // offset, we may want
> > >>>>>>> to
> > >>>>>>>>>> retry
> > >>>>>>>>>>>>>>> these, as
> > >>>>>>>>>>>>>>>>>>>>> described
> > >>>>>>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>>>> each
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> case
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below. If
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> none of those apply,
> > >>>>>>> then
> > >>>>>>>>> for
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // idempotent
> > >>>>>> producer,
> > >>>>>>> we
> > >>>>>>>>>> will
> > >>>>>>>>>>>>>>> locally
> > >>>>>>>>>>>>>>>>> bump
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> reset
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence numbers of
> > >>>>>>>>> in-flight
> > >>>>>>>>>>>> batches
> > >>>>>>>>>>>>>>>> from
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // sequence 0, then
> > >>>>>>> retry
> > >>>>>>>>> the
> > >>>>>>>>>>>> failed
> > >>>>>>>>>>>>>>>> batch,
> > >>>>>>>>>>>>>>>>>>>> which
> > >>>>>>>>>>>>>>>>>>>>>>>> should
> > >>>>>>>>>>>>>>>>>>>>>>>>>> now
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> succeed.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the transactional
> > >>>>>>>> producer,
> > >>>>>>>>>> allow
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // batch to fail. When
> > >>>>>>>>>> processing
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>> failed
> > >>>>>>>>>>>>>>>>>>>>> batch,
> > >>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transition
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an abortable error and
> > >>>>>>>> set a
> > >>>>>>>>>> flag
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // indicating that we
> > >>>>>>> need
> > >>>>>>>>> to
> > >>>>>>>>>>>> bump the
> > >>>>>>>>>>>>>>>>> epoch
> > >>>>>>>>>>>>>>>>>>>> (if
> > >>>>>>>>>>>>>>>>>>>>>>>>> supported
> > >>>>>>>>>>>>>>>>>>>>>>>>>> by
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker).
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if (error ==
> > >>>>>>>>>>>>>>>> Errors.*UNKNOWN_PRODUCER_ID*)
> > >>>>>>>>>>>>>>>>> {
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > >>>>>>>>>> (response.logStartOffset
> > >>>>>>>>>>>> ==
> > >>>>>>>>>>>>>>> -1)
> > >>>>>>>>>>>>>>>> {
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // We don't
> > >>>>>>> know
> > >>>>>>>>>> the log
> > >>>>>>>>>>>>>>> start
> > >>>>>>>>>>>>>>>>>> offset
> > >>>>>>>>>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> response.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the request
> > >>>>>>>> until
> > >>>>>>>>>> we
> > >>>>>>>>>>>> get
> > >>>>>>>>>>>>>>> it.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // The
> > >>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > >>>>>>>>>>>>>>>> error
> > >>>>>>>>>>>>>>>>>> code
> > >>>>>>>>>>>>>>>>>>>>> was
> > >>>>>>>>>>>>>>>>>>>>>>>> added
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> along
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProduceResponse which
> > >>>>>>>>>> includes the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              //
> > >>>>>>>> logStartOffset.
> > >>>>>>>>>> So
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>> '-1'
> > >>>>>>>>>>>>>>>>>>>> sentinel
> > >>>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> backward
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatibility.
> > >>>>>> Instead,
> > >>>>>>> it
> > >>>>>>>>> is
> > >>>>>>>>>>>> possible
> > >>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // a broker
> > >>>>>> to
> > >>>>>>>> not
> > >>>>>>>>>> know
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> logStartOffset at
> > >>>>>>>>>>>>>>>>>>>>>>>>>> when
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returning
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response because
> > >>>>>> the
> > >>>>>>>>>> partition
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // may have
> > >>>>>>>> moved
> > >>>>>>>>>> away
> > >>>>>>>>>>>> from
> > >>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> broker
> > >>>>>>>>>>>>>>>>>>>>>>> from
> > >>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error was
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initially raised to
> > >>>>>> the
> > >>>>>>>> time
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // response
> > >>>>>>> was
> > >>>>>>>>>> being
> > >>>>>>>>>>>>>>>>> constructed.
> > >>>>>>>>>>>>>>>>>> In
> > >>>>>>>>>>>>>>>>>>>>>>> these
> > >>>>>>>>>>>>>>>>>>>>>>>>>> cases,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retry the request: we
> > >>>>>>> are
> > >>>>>>>>>>>> guaranteed
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // to
> > >>>>>>> eventually
> > >>>>>>>>>> get a
> > >>>>>>>>>>>>>>>>>> logStartOffset
> > >>>>>>>>>>>>>>>>>>>>> once
> > >>>>>>>>>>>>>>>>>>>>>>>>> things
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> settle
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> down.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > >>>>>>>>>>>>>>> (batch.sequenceHasBeenReset()) {
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // When the
> > >>>>>>>> first
> > >>>>>>>>>>>> inflight
> > >>>>>>>>>>>>>>>> batch
> > >>>>>>>>>>>>>>>>>>>> fails
> > >>>>>>>>>>>>>>>>>>>>>>> due to
> > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> truncation
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> then the sequences of
> > >>>>>>> all
> > >>>>>>>>> the
> > >>>>>>>>>>>> other
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // in flight
> > >>>>>>>>> batches
> > >>>>>>>>>>>> would
> > >>>>>>>>>>>>>>> have
> > >>>>>>>>>>>>>>>>>> been
> > >>>>>>>>>>>>>>>>>>>>>>>> restarted
> > >>>>>>>>>>>>>>>>>>>>>>>>>> from
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> beginning.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, when those
> > >>>>>>>>> responses
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // come back
> > >>>>>>>> from
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>>> broker,
> > >>>>>>>>>>>>>>>>> they
> > >>>>>>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>>>>> also
> > >>>>>>>>>>>>>>>>>>>>>>>>>> come
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > >>>>>>> error.
> > >>>>>>>>> In
> > >>>>>>>>>> this
> > >>>>>>>>>>>>>>> case,
> > >>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>> should
> > >>>>>>>>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // reset the
> > >>>>>>>>>> sequence
> > >>>>>>>>>>>>>>> numbers
> > >>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> beginning.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          } else if
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>> (lastAckedOffset(batch.topicPartition).orElse(
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > >>>>>>>>>>>>>>>>>>>>>>>>>> response.logStartOffset) {
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // The head
> > >>>>>> of
> > >>>>>>>> the
> > >>>>>>>>>> log
> > >>>>>>>>>>>> has
> > >>>>>>>>>>>>>>> been
> > >>>>>>>>>>>>>>>>>>>> removed,
> > >>>>>>>>>>>>>>>>>>>>>>>>> probably
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> due
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retention time
> > >>>>>> elapsing.
> > >>>>>>>> In
> > >>>>>>>>>> this
> > >>>>>>>>>>>> case,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // we expect
> > >>>>>>> to
> > >>>>>>>>>> lose the
> > >>>>>>>>>>>>>>>> producer
> > >>>>>>>>>>>>>>>>>>>> state.
> > >>>>>>>>>>>>>>>>>>>>>>> For
> > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer, reset the
> > >>>>>>>>> sequences
> > >>>>>>>>>> of
> > >>>>>>>>>>>> all
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // inflight
> > >>>>>>>>> batches
> > >>>>>>>>>> to
> > >>>>>>>>>>>> be
> > >>>>>>>>>>>>>>> from
> > >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> beginning
> > >>>>>>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> retry
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the transaction
> > >>>>>>> does
> > >>>>>>>>> not
> > >>>>>>>>>>>> need to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // be
> > >>>>>> aborted.
> > >>>>>>>> For
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>>>> idempotent
> > >>>>>>>>>>>>>>>>>>>>>>> producer,
> > >>>>>>>>>>>>>>>>>>>>>>>>> bump
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> avoid
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reusing (sequence,
> > >>>>>>> epoch)
> > >>>>>>>>>> pairs
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              if
> > >>>>>>>>>> (isTransactional()) {
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>
> txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>> this.producerIdAndEpoch);
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              } else {
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> requestEpochBumpForPartition(batch.topicPartition);
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              }
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> > >>>>>>>>> (!isTransactional())
> > >>>>>>>>>> {
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // For the
> > >>>>>>>>>> idempotent
> > >>>>>>>>>>>>>>> producer,
> > >>>>>>>>>>>>>>>>>>>> always
> > >>>>>>>>>>>>>>>>>>>>>>> retry
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors. If the batch
> > >>>>>> has
> > >>>>>>>> the
> > >>>>>>>>>>>> current
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // producer
> > >>>>>> ID
> > >>>>>>>> and
> > >>>>>>>>>>>> epoch,
> > >>>>>>>>>>>>>>>>> request a
> > >>>>>>>>>>>>>>>>>>>> bump
> > >>>>>>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Otherwise
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the
> > >>>>>> produce.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>> requestEpochBumpForPartition(batch.topicPartition);
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I was considering
> > >>>>>>> keeping
> > >>>>>>>>> this
> > >>>>>>>>>>>>>>> behavior —
> > >>>>>>>>>>>>>>>>> but
> > >>>>>>>>>>>>>>>>>>>> am
> > >>>>>>>>>>>>>>>>>>>>>>> open
> > >>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simplifying
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We are leaving changes
> > >>>>>>> to
> > >>>>>>>>>> older
> > >>>>>>>>>>>>>>> clients
> > >>>>>>>>>>>>>>>> off
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> table
> > >>>>>>>>>>>>>>>>>>>>>>>>> here
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> since
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> caused
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many issues for
> > >>>>>> clients
> > >>>>>>> in
> > >>>>>>>>> the
> > >>>>>>>>>>>> past.
> > >>>>>>>>>>>>>>>>>> Previously
> > >>>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>>> was
> > >>>>>>>>>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we didn't have the
> > >>>>>>>>> mechanisms
> > >>>>>>>>>> in
> > >>>>>>>>>>>>>>> place to
> > >>>>>>>>>>>>>>>>>>>> detect
> > >>>>>>>>>>>>>>>>>>>>>>> when
> > >>>>>>>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> was
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> legitimate
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case vs some bug or
> > >>>>>> gap
> > >>>>>>> in
> > >>>>>>>>> the
> > >>>>>>>>>>>>>>> protocol.
> > >>>>>>>>>>>>>>>>>>>> Ensuring
> > >>>>>>>>>>>>>>>>>>>>>>> each
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> its
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> own epoch should close
> > >>>>>>>> this
> > >>>>>>>>>> gap.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> And to address Jeff's
> > >>>>>>>> second
> > >>>>>>>>>>>> point:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *does the typical
> > >>>>>>> produce
> > >>>>>>>>>> request
> > >>>>>>>>>>>> path
> > >>>>>>>>>>>>>>>>> append
> > >>>>>>>>>>>>>>>>>>>>>>> records
> > >>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> local
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> log
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along*
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *with the
> > >>>>>>>>>> currentTxnFirstOffset
> > >>>>>>>>>>>>>>>>> information?
> > >>>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>>>>>> like
> > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand*
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *when the field is
> > >>>>>>> written
> > >>>>>>>>> to
> > >>>>>>>>>>>> disk.*
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, the first produce
> > >>>>>>>>> request
> > >>>>>>>>>>>>>>> populates
> > >>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>> field
> > >>>>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> writes
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> offset
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as part of the record
> > >>>>>>>> batch
> > >>>>>>>>>> and
> > >>>>>>>>>>>> also
> > >>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>> producer
> > >>>>>>>>>>>>>>>>>>>>>>>>> state
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> snapshot.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we reload the records
> > >>>>>> on
> > >>>>>>>>>> restart
> > >>>>>>>>>>>>>>> and/or
> > >>>>>>>>>>>>>>>>>>>>>>> reassignment,
> > >>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> repopulate
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> field with the
> > >>>>>> snapshot
> > >>>>>>>> from
> > >>>>>>>>>> disk
> > >>>>>>>>>>>>>>> along
> > >>>>>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> rest
> > >>>>>>>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if there
> > >>>>>> are
> > >>>>>>>>>> further
> > >>>>>>>>>>>>>>> comments
> > >>>>>>>>>>>>>>>>>>>> and/or
> > >>>>>>>>>>>>>>>>>>>>>>>>> questions.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > >>>>>> at
> > >>>>>>>> 9:00
> > >>>>>>>>>> PM
> > >>>>>>>>>>>> Jeff
> > >>>>>>>>>>>>>>> Kim
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>> <jeff.kim@confluent.io.invalid
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP! I
> > >>>>>>>> have
> > >>>>>>>>>> two
> > >>>>>>>>>>>>>>>> questions:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1) For new clients,
> > >>>>>> we
> > >>>>>>>> can
> > >>>>>>>>>> once
> > >>>>>>>>>>>> again
> > >>>>>>>>>>>>>>>>> return
> > >>>>>>>>>>>>>>>>>>>> an
> > >>>>>>>>>>>>>>>>>>>>>>> error
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for sequences
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that are non-zero
> > >>>>>> when
> > >>>>>>>>> there
> > >>>>>>>>>> is
> > >>>>>>>>>>>> no
> > >>>>>>>>>>>>>>>>> producer
> > >>>>>>>>>>>>>>>>>>>> state
> > >>>>>>>>>>>>>>>>>>>>>>>>> present
> > >>>>>>>>>>>>>>>>>>>>>>>>>> on
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This will indicate we
> > >>>>>>>>> missed
> > >>>>>>>>>> the
> > >>>>>>>>>>>> 0
> > >>>>>>>>>>>>>>>>> sequence
> > >>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>> don't
> > >>>>>>>>>>>>>>>>>>>>>>>>>> yet
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> want
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the log.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would like to
> > >>>>>>>> understand
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>>> current
> > >>>>>>>>>>>>>>>>>>>> behavior
> > >>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>> handle
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and if there are any
> > >>>>>>>>> changes
> > >>>>>>>>>> we
> > >>>>>>>>>>>> are
> > >>>>>>>>>>>>>>>>> making.
> > >>>>>>>>>>>>>>>>>>>> Maybe
> > >>>>>>>>>>>>>>>>>>>>>>> I'm
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> missing
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but we would want to
> > >>>>>>>>> identify
> > >>>>>>>>>>>>>>> whether we
> > >>>>>>>>>>>>>>>>>>>> missed
> > >>>>>>>>>>>>>>>>>>>>>>> the 0
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> sequence
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients, no?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2) Upon returning
> > >>>>>> from
> > >>>>>>>> the
> > >>>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>>>>> coordinator, we
> > >>>>>>>>>>>>>>>>>>>>>>>>> can
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> set
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as ongoing on the
> > >>>>>>> leader
> > >>>>>>>> by
> > >>>>>>>>>>>>>>> populating
> > >>>>>>>>>>>>>>>>>>>>>>>>>> currentTxnFirstOffset
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through the typical
> > >>>>>>>> produce
> > >>>>>>>>>>>> request
> > >>>>>>>>>>>>>>>>>> handling.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does the typical
> > >>>>>>> produce
> > >>>>>>>>>> request
> > >>>>>>>>>>>> path
> > >>>>>>>>>>>>>>>>> append
> > >>>>>>>>>>>>>>>>>>>>>>> records
> > >>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> local
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> log
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> > >>>>>>>>>> currentTxnFirstOffset
> > >>>>>>>>>>>>>>>>> information?
> > >>>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>>>>>> like
> > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when the field is
> > >>>>>>> written
> > >>>>>>>>> to
> > >>>>>>>>>>>> disk.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeff
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > >>>>>> at
> > >>>>>>>>> 4:44
> > >>>>>>>>>> PM
> > >>>>>>>>>>>> Artem
> > >>>>>>>>>>>>>>>>>> Livshits
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
> > >>>>>>> alivshits@confluent.io
> > >>>>>>>>>> .invalid>
> > >>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for the
> > >>>>>> KIP.
> > >>>>>>>> I
> > >>>>>>>>>> have
> > >>>>>>>>>>>> one
> > >>>>>>>>>>>>>>>>>> question.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 5) For new clients,
> > >>>>>> we
> > >>>>>>>> can
> > >>>>>>>>>> once
> > >>>>>>>>>>>>>>> again
> > >>>>>>>>>>>>>>>>>> return
> > >>>>>>>>>>>>>>>>>>>> an
> > >>>>>>>>>>>>>>>>>>>>>>> error
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe we had
> > >>>>>>>> problems
> > >>>>>>>>>> in the
> > >>>>>>>>>>>>>>> past
> > >>>>>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>>>>>>>> returning
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because it was
> > >>>>>>>> considered
> > >>>>>>>>>> fatal
> > >>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>> required
> > >>>>>>>>>>>>>>>>>>>>>>> client
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> restart.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> It
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good to spell out
> > >>>>>> the
> > >>>>>>>> new
> > >>>>>>>>>> client
> > >>>>>>>>>>>>>>>> behavior
> > >>>>>>>>>>>>>>>>>>>> when
> > >>>>>>>>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>>>>>>>>>> receives
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> > >>>>>>> at
> > >>>>>>>>>> 10:00 AM
> > >>>>>>>>>>>>>>>> Justine
> > >>>>>>>>>>>>>>>>>>>> Olshan
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>> <jo...@confluent.io.invalid>
> > >>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
> > >>>>>>>> look
> > >>>>>>>>>>>> Matthias.
> > >>>>>>>>>>>>>>>> I've
> > >>>>>>>>>>>>>>>>>>>> tried
> > >>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>> answer
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> your
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> questions
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 10)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Right — so the
> > >>>>>>> hanging
> > >>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>> only
> > >>>>>>>>>>>>>>>>>>>> occurs
> > >>>>>>>>>>>>>>>>>>>>>>> when
> > >>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> come in and the
> > >>>>>>>> partition
> > >>>>>>>>>> is
> > >>>>>>>>>>>> never
> > >>>>>>>>>>>>>>>> added
> > >>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> again.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never add the
> > >>>>>>> partition
> > >>>>>>>>> to
> > >>>>>>>>>> a
> > >>>>>>>>>>>>>>>>> transaction,
> > >>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>>>>>>>>> never
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> marker
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never advance the
> > >>>>>>> LSO.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do end up
> > >>>>>>> adding
> > >>>>>>>>> the
> > >>>>>>>>>>>>>>> partition
> > >>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>>>>>>>> (I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suppose
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happen before or
> > >>>>>>> after
> > >>>>>>>>> the
> > >>>>>>>>>> late
> > >>>>>>>>>>>>>>>> message
> > >>>>>>>>>>>>>>>>>>>> comes
> > >>>>>>>>>>>>>>>>>>>>> in)
> > >>>>>>>>>>>>>>>>>>>>>>>> then
> > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> include
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late message in the
> > >>>>>>>> next
> > >>>>>>>>>>>>>>> (incorrect)
> > >>>>>>>>>>>>>>>>>>>>> transaction.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> So perhaps it is
> > >>>>>>>> clearer
> > >>>>>>>>> to
> > >>>>>>>>>>>> make
> > >>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>> distinction
> > >>>>>>>>>>>>>>>>>>>>>>>>> between
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> eventually get
> > >>>>>> added
> > >>>>>>> to
> > >>>>>>>>> the
> > >>>>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>> (but
> > >>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> wrong
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> one)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> or
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that never get
> > >>>>>> added
> > >>>>>>>> and
> > >>>>>>>>>> become
> > >>>>>>>>>>>>>>>> hanging.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side
> > >>>>>>> change
> > >>>>>>>>> for
> > >>>>>>>>>> 2 is
> > >>>>>>>>>>>>>>>> removing
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>> addPartitions
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need
> > >>>>>>> to
> > >>>>>>>>> make
> > >>>>>>>>>>>> this
> > >>>>>>>>>>>>>>> from
> > >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> producer
> > >>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> txn
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In my opinion, the
> > >>>>>>>> issue
> > >>>>>>>>>> with
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > >>>>>>>>>>>>>>>>>>>>>>>>>> call
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is that we don't
> > >>>>>> have
> > >>>>>>>> the
> > >>>>>>>>>> epoch
> > >>>>>>>>>>>>>>> bump,
> > >>>>>>>>>>>>>>>> so
> > >>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>> don't
> > >>>>>>>>>>>>>>>>>>>>>>>> know
> > >>>>>>>>>>>>>>>>>>>>>>>>>> if
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> belongs to the
> > >>>>>>> previous
> > >>>>>>>>>>>>>>> transaction or
> > >>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>> one.
> > >>>>>>>>>>>>>>>>>>>>>>> We
> > >>>>>>>>>>>>>>>>>>>>>>>>> need
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition has been
> > >>>>>>>> added
> > >>>>>>>>> to
> > >>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>> transaction.
> > >>>>>>>>>>>>>>>>>>>>> Of
> > >>>>>>>>>>>>>>>>>>>>>>>>> course,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> means
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't completely
> > >>>>>>> cover
> > >>>>>>>>> the
> > >>>>>>>>>> case
> > >>>>>>>>>>>>>>> where
> > >>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>> have a
> > >>>>>>>>>>>>>>>>>>>>>>>> really
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> late
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have added the
> > >>>>>>>> partition
> > >>>>>>>>> to
> > >>>>>>>>>>>> the new
> > >>>>>>>>>>>>>>>>>>>>> transaction,
> > >>>>>>>>>>>>>>>>>>>>>>> but
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> that's
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something we will
> > >>>>>>> need
> > >>>>>>>>> the
> > >>>>>>>>>> new
> > >>>>>>>>>>>>>>> clients
> > >>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>> cover.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is
> > >>>>>>> ongoing
> > >>>>>>>> =
> > >>>>>>>>>>>> partition
> > >>>>>>>>>>>>>>> was
> > >>>>>>>>>>>>>>>>>>>> added to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> via
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn.
> > >>>>>>> We
> > >>>>>>>>>> check
> > >>>>>>>>>>>> this
> > >>>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this
> > >>>>>>> wasn't
> > >>>>>>>>>>>> sufficiently
> > >>>>>>>>>>>>>>>>>>>> explained
> > >>>>>>>>>>>>>>>>>>>>>>> here:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>
> > >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is
> > >>>>>> that
> > >>>>>>>> if
> > >>>>>>>>>> any
> > >>>>>>>>>>>>>>> messages
> > >>>>>>>>>>>>>>>>>>>> somehow
> > >>>>>>>>>>>>>>>>>>>>>>> come
> > >>>>>>>>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> before
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the
> > >>>>>>> producer,
> > >>>>>>>>> they
> > >>>>>>>>>>>> will be
> > >>>>>>>>>>>>>>>>>> fenced.
> > >>>>>>>>>>>>>>>>>>>>>>> However,
> > >>>>>>>>>>>>>>>>>>>>>>>>> if
> > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it
> > >>>>>> can
> > >>>>>>> be
> > >>>>>>>>>>>> discussed
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 50)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It should be
> > >>>>>>>> synchronous
> > >>>>>>>>>>>> because
> > >>>>>>>>>>>>>>> if we
> > >>>>>>>>>>>>>>>>>> have
> > >>>>>>>>>>>>>>>>>>>> an
> > >>>>>>>>>>>>>>>>>>>>>>> event
> > >>>>>>>>>>>>>>>>>>>>>>>>>> (ie,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> an
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes us to need
> > >>>>>> to
> > >>>>>>>>> abort
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>> transaction,
> > >>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>> need
> > >>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions to send
> > >>>>>>>>>> transaction
> > >>>>>>>>>>>>>>> markers
> > >>>>>>>>>>>>>>>>> to.
> > >>>>>>>>>>>>>>>>>>>> We
> > >>>>>>>>>>>>>>>>>>>>>>> know
> > >>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we added them to
> > >>>>>> the
> > >>>>>>>>>>>> coordinator
> > >>>>>>>>>>>>>>> via
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Previously we have
> > >>>>>>> had
> > >>>>>>>>>>>> asynchronous
> > >>>>>>>>>>>>>>>>> calls
> > >>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> past
> > >>>>>>>>>>>>>>>>>>>>>>>>>> (ie,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit markers when
> > >>>>>>> the
> > >>>>>>>>>>>>>>> transaction is
> > >>>>>>>>>>>>>>>>>>>>> completed)
> > >>>>>>>>>>>>>>>>>>>>>>>> but
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> often
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes confusion as
> > >>>>>>> we
> > >>>>>>>>>> need to
> > >>>>>>>>>>>> wait
> > >>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>> some
> > >>>>>>>>>>>>>>>>>>>>>>>>> operations
> > >>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complete.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing commit
> > >>>>>>> markers
> > >>>>>>>>>> case,
> > >>>>>>>>>>>>>>> clients
> > >>>>>>>>>>>>>>>>> often
> > >>>>>>>>>>>>>>>>>>>> see
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>> CONCURRENT_TRANSACTIONs
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error messages and
> > >>>>>>> that
> > >>>>>>>>>> can be
> > >>>>>>>>>>>>>>>>> confusing.
> > >>>>>>>>>>>>>>>>>>>> For
> > >>>>>>>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>>>>>> reason,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> may
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simpler to just
> > >>>>>> have
> > >>>>>>>>>>>> synchronous
> > >>>>>>>>>>>>>>>> calls —
> > >>>>>>>>>>>>>>>>>>>>>>> especially
> > >>>>>>>>>>>>>>>>>>>>>>>> if
> > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> need
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> block
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some operation's
> > >>>>>>>>> completion
> > >>>>>>>>>>>> anyway
> > >>>>>>>>>>>>>>>>> before
> > >>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>> can
> > >>>>>>>>>>>>>>>>>>>>>>>> start
> > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> next
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. And
> > >>>>>>> yes, I
> > >>>>>>>>>> meant
> > >>>>>>>>>>>>>>>>>> coordinator. I
> > >>>>>>>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>>>>>>>> fix
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> that.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are
> > >>>>>> checking
> > >>>>>>> if
> > >>>>>>>>> the
> > >>>>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>> ongoing,
> > >>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> need
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the
> > >>>>>> leader
> > >>>>>>>>>> partition
> > >>>>>>>>>>>> to
> > >>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for
> > >>>>>>> this
> > >>>>>>>>>>>> message to
> > >>>>>>>>>>>>>>>> come
> > >>>>>>>>>>>>>>>>>>>> back,
> > >>>>>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>>>>>> theory
> > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> could
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
> > >>>>>>>> that
> > >>>>>>>>>> would
> > >>>>>>>>>>>> make
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>> original
> > >>>>>>>>>>>>>>>>>>>>>>>>> result
> > >>>>>>>>>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why
> > >>>>>> we
> > >>>>>>>> can
> > >>>>>>>>>> check
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>> leader
> > >>>>>>>>>>>>>>>>>>>> state
> > >>>>>>>>>>>>>>>>>>>>>>>> before
> > >>>>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm happy to update
> > >>>>>>> the
> > >>>>>>>>>> KIP if
> > >>>>>>>>>>>>>>> some of
> > >>>>>>>>>>>>>>>>>> these
> > >>>>>>>>>>>>>>>>>>>>>>> things
> > >>>>>>>>>>>>>>>>>>>>>>>>> were
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Nov 21,
> > >>>>>> 2022
> > >>>>>>> at
> > >>>>>>>>>> 7:11 PM
> > >>>>>>>>>>>>>>>> Matthias
> > >>>>>>>>>>>>>>>>>> J.
> > >>>>>>>>>>>>>>>>>>>>> Sax <
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the
> > >>>>>> KIP.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Couple of
> > >>>>>>>> clarification
> > >>>>>>>>>>>> questions
> > >>>>>>>>>>>>>>> (I
> > >>>>>>>>>>>>>>>> am
> > >>>>>>>>>>>>>>>>>>>> not a
> > >>>>>>>>>>>>>>>>>>>>>>>> broker
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> expert
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> do
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some question are
> > >>>>>>>>> obvious
> > >>>>>>>>>> for
> > >>>>>>>>>>>>>>> others,
> > >>>>>>>>>>>>>>>>> but
> > >>>>>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>> me
> > >>>>>>>>>>>>>>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> my
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lack
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker knowledge).
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (10)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What happens if
> > >>>>>> the
> > >>>>>>>>>> message
> > >>>>>>>>>>>> come
> > >>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>> before
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> next
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request? It seems
> > >>>>>>> the
> > >>>>>>>>>> broker
> > >>>>>>>>>>>>>>> hosting
> > >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> data
> > >>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anything about it
> > >>>>>>> and
> > >>>>>>>>>> append
> > >>>>>>>>>>>> it to
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> partition,
> > >>>>>>>>>>>>>>>>>>>>>>>>> too?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> What
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> difference between
> > >>>>>>>> both
> > >>>>>>>>>> cases?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, it seems a
> > >>>>>> TX
> > >>>>>>>>> would
> > >>>>>>>>>> only
> > >>>>>>>>>>>>>>> hang,
> > >>>>>>>>>>>>>>>> if
> > >>>>>>>>>>>>>>>>>>>> there
> > >>>>>>>>>>>>>>>>>>>>>>> is no
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> following
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> either committer
> > >>>>>> or
> > >>>>>>>>>> aborted?
> > >>>>>>>>>>>> Thus,
> > >>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>> case
> > >>>>>>>>>>>>>>>>>>>>>>>>> above,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> TX
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> might
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually not hang
> > >>>>>>> (of
> > >>>>>>>>>> course,
> > >>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>> might
> > >>>>>>>>>>>>>>>>>> get
> > >>>>>>>>>>>>>>>>>>>> an
> > >>>>>>>>>>>>>>>>>>>>>>> EOS
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> violation
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX was aborted and
> > >>>>>>> the
> > >>>>>>>>>> second
> > >>>>>>>>>>>>>>>>> committed,
> > >>>>>>>>>>>>>>>>>> or
> > >>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> other
> > >>>>>>>>>>>>>>>>>>>>>>>>>> way
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around).
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (20)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Of course, 1 and
> > >>>>>> 2
> > >>>>>>>>>> require
> > >>>>>>>>>>>>>>>> client-side
> > >>>>>>>>>>>>>>>>>>>>>>> changes, so
> > >>>>>>>>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> those approaches
> > >>>>>>> won’t
> > >>>>>>>>>> apply.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For (1) I
> > >>>>>> understand
> > >>>>>>>>> why a
> > >>>>>>>>>>>> client
> > >>>>>>>>>>>>>>>>> change
> > >>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>> necessary,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we need a client
> > >>>>>>>> change
> > >>>>>>>>>> for
> > >>>>>>>>>>>> (2).
> > >>>>>>>>>>>>>>> Can
> > >>>>>>>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > >>>>>>>>>>>>>>>>>>>>>>>>> --
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Later
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> explain
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that we should
> > >>>>>> send
> > >>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>>>> DescribeTransactionRequest,
> > >>>>>>>>>>>>>>>>>>>>>>>>> but I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> am
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Can't we not just
> > >>>>>> do
> > >>>>>>>> an
> > >>>>>>>>>>>> implicit
> > >>>>>>>>>>>>>>>>>>>>>>> AddPartiitonToTx,
> > >>>>>>>>>>>>>>>>>>>>>>>>> too?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> If
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> old
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer correctly
> > >>>>>>>>>> registered
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>> partition
> > >>>>>>>>>>>>>>>>>>>>>>>> already,
> > >>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can just ignore it
> > >>>>>>> as
> > >>>>>>>>>> it's an
> > >>>>>>>>>>>>>>>>> idempotent
> > >>>>>>>>>>>>>>>>>>>>>>> operation?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (30)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To cover older
> > >>>>>>>> clients,
> > >>>>>>>>>> we
> > >>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>> ensure a
> > >>>>>>>>>>>>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ongoing
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> before
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we write to a
> > >>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Not sure what you
> > >>>>>>> mean
> > >>>>>>>>> by
> > >>>>>>>>>>>> this?
> > >>>>>>>>>>>>>>> Can
> > >>>>>>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (40)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [the
> > >>>>>>> TX-coordinator]
> > >>>>>>>>> will
> > >>>>>>>>>>>> write
> > >>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> prepare
> > >>>>>>>>>>>>>>>>>>>>>>> commit
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> message
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch and send
> > >>>>>>>>>>>>>>> WriteTxnMarkerRequests
> > >>>>>>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> bumped
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why do we use the
> > >>>>>>>> bumped
> > >>>>>>>>>>>> epoch for
> > >>>>>>>>>>>>>>>>> both?
> > >>>>>>>>>>>>>>>>>> It
> > >>>>>>>>>>>>>>>>>>>>>>> seems
> > >>>>>>>>>>>>>>>>>>>>>>>>> more
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> intuitive
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the current epoch,
> > >>>>>>> and
> > >>>>>>>>>> only
> > >>>>>>>>>>>> return
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> bumped
> > >>>>>>>>>>>>>>>>>>>>>>> epoch
> > >>>>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (50) "Implicit
> > >>>>>>>>>>>>>>>>> AddPartitionToTransaction"
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why does the
> > >>>>>>>> implicitly
> > >>>>>>>>>> sent
> > >>>>>>>>>>>>>>> request
> > >>>>>>>>>>>>>>>>> need
> > >>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> synchronous?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also says
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in case we need
> > >>>>>> to
> > >>>>>>>>> abort
> > >>>>>>>>>> and
> > >>>>>>>>>>>>>>> need to
> > >>>>>>>>>>>>>>>>>> know
> > >>>>>>>>>>>>>>>>>>>>> which
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What do you mean
> > >>>>>> by
> > >>>>>>>>> this?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we don’t want to
> > >>>>>>>> write
> > >>>>>>>>>> to it
> > >>>>>>>>>>>>>>> before
> > >>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>> store
> > >>>>>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> manager
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Do you mean
> > >>>>>>>>> TX-coordinator
> > >>>>>>>>>>>>>>> instead of
> > >>>>>>>>>>>>>>>>>>>>> "manager"?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (60)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For older clients
> > >>>>>>> and
> > >>>>>>>>>> ensuring
> > >>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>> TX
> > >>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>> ongoing,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> describe a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> race condition. I
> > >>>>>> am
> > >>>>>>>> not
> > >>>>>>>>>> sure
> > >>>>>>>>>>>> if I
> > >>>>>>>>>>>>>>>> can
> > >>>>>>>>>>>>>>>>>>>> follow
> > >>>>>>>>>>>>>>>>>>>>>>> here.
> > >>>>>>>>>>>>>>>>>>>>>>>>> Can
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> elaborate?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/18/22 1:21
> > >>>>>> PM,
> > >>>>>>>>>> Justine
> > >>>>>>>>>>>>>>> Olshan
> > >>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey all!
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to
> > >>>>>> start a
> > >>>>>>>>>>>> discussion
> > >>>>>>>>>>>>>>> on my
> > >>>>>>>>>>>>>>>>>>>> proposal
> > >>>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>> add
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> some
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checks on
> > >>>>>>>> transactions
> > >>>>>>>>> to
> > >>>>>>>>>>>> avoid
> > >>>>>>>>>>>>>>>>> hanging
> > >>>>>>>>>>>>>>>>>>>>>>>>> transactions.
> > >>>>>>>>>>>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> know
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> been
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an issue for some
> > >>>>>>>> time,
> > >>>>>>>>>> so I
> > >>>>>>>>>>>>>>> really
> > >>>>>>>>>>>>>>>>> hope
> > >>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>> KIP
> > >>>>>>>>>>>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> helpful
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users of EOS.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The KIP includes
> > >>>>>>>>> changes
> > >>>>>>>>>> that
> > >>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>> compatible
> > >>>>>>>>>>>>>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> old
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> changes to
> > >>>>>> improve
> > >>>>>>>>>>>> performance
> > >>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>> correctness
> > >>>>>>>>>>>>>>>>>>>>>>> on
> > >>>>>>>>>>>>>>>>>>>>>>>>> new
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Please take a
> > >>>>>> look
> > >>>>>>>> and
> > >>>>>>>>>> leave
> > >>>>>>>>>>>> any
> > >>>>>>>>>>>>>>>>>> comments
> > >>>>>>>>>>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>>>>>>> may
> > >>>>>>>>>>>>>>>>>>>>>>>>>> have!
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>
> > >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> JIRA:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-14402
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>
> > >>>>
> > >>>
> > >>
> > >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Guozhang Wang <gu...@gmail.com>.
Hello Justine,

Regarding which errors should be fatal v.s. abortable, I think the
principle would be “if we did abort the txn and continue, could the
app ever has a chance to recover and not have any side effects
violating EOS guarantees”.Of course we can discuss whether that’s a
good principle  e.g. one can argue that maybe it’s okay to violate EOS
some times rather than killing the whole app, whereas my rationale is
that since we want to make EOS the default config, it’s crucial that
we try to guarantee it as much as possible and whenever we detected if
there’s a chance that it maybe broken, we should let users know. And
if we just log an error in app and try to abort and continue, it’s
very likely that users would not be notified and probably only realize
that after a while, which may cause an even harder scenario to
debug.If people agree on that principle, we can go back and check this
KIP’s scenario: there are several scenarios causing a partition leader
detects an out of date epoch from a produce requests:

* The old produce requests arrive late after a network partition, and
the old producer is already gone.
* The old produce requests arrive late after a network partition, but
the old producer is still around.
* A buggy producer client that did not follow the protocol.

For the third case, we should try to let user know asap, and as clear
as possible, and hence it’s best to just stop the client app; for the
first case, it does not matter since the producer is already gone;
only for the second, probably the least likely case, we need to
ponder, and there my rationale again is that at that time, we may have
already violated the EOS guarantees since there are some partial txn
records that should be rejected while the txn itself has been
committed. In this case I think it’s better to let clients/users know
as soon as possible as well.

Regarding the past scenarios where a fatal error killing the whole
apps, I believe part of that reason is that we were doing an
sub-optimal job on clients side handling various error cases and
that’s what KIP-691 is trying to resolve, and hence personally I would
suggest we do not weight in too much on that if we can trust that
KIP-691 will be successfully eliminate those not-necessary-hard-fail
scenarios.


Guozhang

On Wed, Jan 25, 2023 at 5:51 PM Matthias J. Sax <mj...@apache.org> wrote:
>
> So the timestamp would be set when the write happens and thus no
> out-of-order data (base in time) can be introduced with "append_time"
> config even if a request sits in purgatory first while we check the TX
> status.
>
> That does make sense. Thanks for confirming, that there is no
> out-of-order issue for this case.
>
>
> -Matthias
>
> On 1/25/23 5:04 PM, Justine Olshan wrote:
> > Hey Matthias,
> > Let me put it this way, if a producer is checking if a transaction is
> > ongoing, then no writes to the partition from the producer will go through
> > until the transaction is confirmed ongoing.
> >  From then, I think I can apply the writes in the order they came in. Does
> > that make sense?
> >
> > Let me know if I'm missing something.
> > Justine
> >
> > On Wed, Jan 25, 2023 at 4:57 PM Matthias J. Sax <mj...@apache.org> wrote:
> >
> >>> would it build an offset map with just the latest timestamp for a key?
> >>
> >> Cannot remember the details without reading the KIP, but yes, something
> >> like this (I believe it actually needs to track both, offset and
> >> timestamp per key).
> >>
> >>> I wonder if ordering assumptions are baked in there, why not use
> >> offset-based compaction.
> >>
> >> The use case is a compacted topic that does contain out-of-order data.
> >> If you set key k1=v1 @ 5 offset 100 and later key1 = v0 @ 3 at offset
> >> 200 we want to cleanup v0 with higher offset because it's out-of-order
> >> based on time, but keep v1 what is the actual latest version of k1.
> >>
> >>
> >>> I was also not aware of this "guarantee" with regards to broker side
> >> time.
> >>
> >> As already said: I am not sure if it's a public contract, but based on
> >> my experience, people might reply on it as "implicit contract". -- Maybe
> >> somebody else knows if it's public or not, and if it would be ok to
> >> "break" it.
> >>
> >>> Let me know if you have any concerns here.
> >>
> >> My understanding is: While we cannot make an offset-order guarantee for
> >> interleaved writes of different producer, if the topic is configures
> >> with "append_time", we "guarantee" (cf. my comment above") timestamp
> >> order... If that's the case, it would be an issue if we break this
> >> "guarantee".
> >>
> >> I am not sure when the broker sets the timestamp for "append_time"
> >> config? If we do it before putting the request into purgatory, we have a
> >> problem. However, if we set the timestamp when we actually process the
> >> request and do the actual append, it seems there is no issue, as the
> >> request that was waiting in purgatory get the "newest" timestamp and
> >> thus cannot introduce out-of-order data.
> >>
> >>
> >> -Matthias
> >>
> >>
> >> On 1/24/23 10:44 AM, Justine Olshan wrote:
> >>> Hey Matthias,
> >>>
> >>> I have actually never heard of KIP-280 so thanks for bringing it up. That
> >>> seems interesting. I wonder how it would work though -- would it build an
> >>> offset map with just the latest timestamp for a key? I wonder if ordering
> >>> assumptions are baked in there, why not use offset-based compaction.
> >>>
> >>> I was also not aware of this "guarantee" with regards to broker side
> >> time.
> >>> I think that we can do in order handling for a given producer, but not
> >>> across all producers. However, we can't guarantee that anyway.
> >>>
> >>> Let me know if you have any concerns here.
> >>>
> >>> Thanks,
> >>> Justine
> >>>
> >>> On Mon, Jan 23, 2023 at 6:33 PM Matthias J. Sax <mj...@apache.org>
> >> wrote:
> >>>
> >>>> Just a side note about Guozhang comments about timestamps.
> >>>>
> >>>> If the producer sets the timestamp, putting the record into purgatory
> >>>> seems not to be an issue (as already said: for this case we don't
> >>>> guarantee timestamp order between writes of different producers anyway).
> >>>> However, if the broker sets the timestamp, the expectation is that there
> >>>> is no out-of-order data in the partition ever; if we would introduce
> >>>> out-of-order data for this case (for interleaved writes of different
> >>>> producers), it seems we would violate the current contract? (To be fair:
> >>>> I don't know if that's an official contract, but I assume people rely on
> >>>> this behavior -- and it "advertised" in many public talks...)
> >>>>
> >>>> About compaction: there is actually KIP-280 that adds timestamp based
> >>>> compaction what is a very useful feature for Kafka Streams with regard
> >>>> to out-of-order data handling. So the impact if we introduce
> >>>> out-of-order data could be larger scoped.
> >>>>
> >>>>
> >>>> -Matthias
> >>>>
> >>>>
> >>>> On 1/20/23 4:48 PM, Justine Olshan wrote:
> >>>>> Hey Artem,
> >>>>>
> >>>>> I see there is a check for transactional producers. I'm wondering if we
> >>>>> don't handle the epoch overflow case. I'm also not sure it will be a
> >> huge
> >>>>> issue to extend to transactional producers, but maybe I'm missing
> >>>> something.
> >>>>>
> >>>>> As for the recovery path -- I think Guozhang's point was if we have a
> >> bad
> >>>>> client that repeatedly tries to produce without adding to the
> >> transaction
> >>>>> we would do the following:
> >>>>> a) if not fatal, we just fail the produce request over and over
> >>>>> b) if fatal, we fence the producer
> >>>>>
> >>>>> Here with B, the issue with the client would be made clear more
> >> quickly.
> >>>> I
> >>>>> suppose there are some intermediate cases where the issue only occurs
> >>>>> sometimes, but I wonder if we should consider how to recover with
> >> clients
> >>>>> who don't behave as expected anyway.
> >>>>>
> >>>>> I think there is a place for the abortable error that we are adding --
> >>>> just
> >>>>> abort and try again. But I think there are also some cases where trying
> >>>> to
> >>>>> recover overcomplicates some logic. Especially if we are considering
> >>>> older
> >>>>> clients -- there I'm not sure if there's a ton we can do besides fail
> >> the
> >>>>> batch or fence the producer. With newer clients, we can consider more
> >>>>> options for what can just be recovered after aborting. But epochs might
> >>>> be
> >>>>> a hard one unless we also want to reset producer ID.
> >>>>>
> >>>>> Thanks,
> >>>>> Justine
> >>>>>
> >>>>>
> >>>>>
> >>>>> On Fri, Jan 20, 2023 at 3:59 PM Artem Livshits
> >>>>> <al...@confluent.io.invalid> wrote:
> >>>>>
> >>>>>>>     besides the poorly written client case
> >>>>>>
> >>>>>> A poorly written client could create a lot of grief to people who run
> >>>> Kafka
> >>>>>> brokers :-), so when deciding to make an error fatal I would see if
> >>>> there
> >>>>>> is a reasonable recovery path rather than how often it could happen.
> >>>> If we
> >>>>>> have solid implementation of transactions (which I hope we'll do as a
> >>>>>> result of this KIP), it would help to recover from a large class of
> >>>> errors
> >>>>>> by just aborting a transaction, even if the cause of error is a race
> >>>>>> condition or etc.
> >>>>>>
> >>>>>> -Artem
> >>>>>>
> >>>>>> On Fri, Jan 20, 2023 at 3:26 PM Justine Olshan
> >>>>>> <jo...@confluent.io.invalid>
> >>>>>> wrote:
> >>>>>>
> >>>>>>> Artem --
> >>>>>>> I guess the discussion path we were going down is when we expect to
> >> see
> >>>>>>> this error. I mentioned that it was hard to come up with cases for
> >> when
> >>>>>> the
> >>>>>>> producer would still be around to receive the error besides the
> >> poorly
> >>>>>>> written client case.
> >>>>>>> If we don't expect to have a producer to receive the response, it
> >> sort
> >>>> of
> >>>>>>> makes sense for it to be fatal.
> >>>>>>>
> >>>>>>> I had some discussion with Jason offline about the epoch being off
> >>>> cases
> >>>>>>> and I'm not sure we could find a ton (outside of produce requests)
> >>>> where
> >>>>>> we
> >>>>>>> could/should recover. I'd be happy to hear some examples though,
> >> maybe
> >>>>>> I'm
> >>>>>>> missing something.
> >>>>>>>
> >>>>>>> Thanks,
> >>>>>>> Justine
> >>>>>>>
> >>>>>>> On Fri, Jan 20, 2023 at 3:19 PM Artem Livshits
> >>>>>>> <al...@confluent.io.invalid> wrote:
> >>>>>>>
> >>>>>>>> In general, I'd like to avoid fatal errors as much as possible, in
> >>>> some
> >>>>>>>> sense fatal errors just push out recovery logic to the application
> >>>>>> which
> >>>>>>>> either complicates the application or leads to disruption (we've
> >> seen
> >>>>>>> cases
> >>>>>>>> when a transient broker error could lead to work stoppage when
> >>>>>>> applications
> >>>>>>>> need to be manually restarted).  I think we should strive to define
> >>>>>>>> recovery logic for most errors (and/or encapsulate it in the Kafka
> >>>>>> client
> >>>>>>>> as much as possible).
> >>>>>>>>
> >>>>>>>> One benefit of transactions is that they simplify recovery from
> >>>> errors,
> >>>>>>>> pretty much any error (that's not handled transparently by retries
> >> in
> >>>>>>> Kafka
> >>>>>>>> client) can be handled by the application via aborting the
> >> transaction
> >>>>>>> and
> >>>>>>>> repeating the transactional logic again.  One tricky error is an
> >> error
> >>>>>>>> during commit, because we don't know the outcome.  For commit
> >> errors,
> >>>>>> the
> >>>>>>>> recommendation should be to retry the commit until it returns the
> >>>>>>> specific
> >>>>>>>> result (committed or aborted).
> >>>>>>>>
> >>>>>>>> -Artem
> >>>>>>>>
> >>>>>>>> On Fri, Jan 20, 2023 at 2:52 PM Justine Olshan
> >>>>>>>> <jo...@confluent.io.invalid>
> >>>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> That's a fair point about other clients.
> >>>>>>>>>
> >>>>>>>>> I think the abortable error case is interesting because I'm curious
> >>>>>> how
> >>>>>>>>> other clients would handle this. I assume they would need to
> >>>>>> implement
> >>>>>>>>> handling for the error code unless they did something like "any
> >>>>>> unknown
> >>>>>>>>> error codes/any codes that aren't x,y,z are retriable." I would
> >> hope
> >>>>>>> that
> >>>>>>>>> unknown error codes were fatal, and if the code was implemented it
> >>>>>>> would
> >>>>>>>>> abort the transaction. But I will think on this too.
> >>>>>>>>>
> >>>>>>>>> As for InvalidRecord -- you mentioned it was not fatal, but I'm
> >>>>>> taking
> >>>>>>> a
> >>>>>>>>> look through the code. We would see this on handling the produce
> >>>>>>>> response.
> >>>>>>>>> If I recall correctly, we check if errors are retriable. I think
> >> this
> >>>>>>>> error
> >>>>>>>>> would not be retriable. But I guess the concern here is that it is
> >>>>>> not
> >>>>>>>>> enough for just that batch to fail. I guess I hadn't considered
> >> fully
> >>>>>>>>> fencing the old producer but there are valid arguments here why we
> >>>>>>> would
> >>>>>>>>> want to.
> >>>>>>>>>
> >>>>>>>>> Thanks,
> >>>>>>>>> Justine
> >>>>>>>>>
> >>>>>>>>> On Fri, Jan 20, 2023 at 2:35 PM Guozhang Wang <
> >>>>>>>> guozhang.wang.us@gmail.com>
> >>>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Thanks Justine for the replies! I agree with most of your
> >> thoughts.
> >>>>>>>>>>
> >>>>>>>>>> Just for 3/7), though I agree for our own AK producer, since we do
> >>>>>>>>>> "nextRequest(boolean hasIncompleteBatches)", we guarantee the
> >>>>>> end-txn
> >>>>>>>>>> would not be sent until we've effectively flushed, but I was
> >>>>>>> referring
> >>>>>>>>>> to any future bugs or other buggy clients that the same client may
> >>>>>>> get
> >>>>>>>>>> into this situation, in which case we should give the client a
> >>>>>> clear
> >>>>>>>>>> msg that "you did something wrong, and hence now you should
> >> fatally
> >>>>>>>>>> close yourself". What I'm concerned about is that, by seeing an
> >>>>>>>>>> "abortable error" or in some rare cases an "invalid record", the
> >>>>>>>>>> client could not realize "something that's really bad happened".
> >> So
> >>>>>>>>>> it's not about adding a new error, it's mainly about those real
> >>>>>> buggy
> >>>>>>>>>> situations causing such "should never happen" cases, the errors
> >>>>>>> return
> >>>>>>>>>> would not be informative enough.
> >>>>>>>>>>
> >>>>>>>>>> Thinking in other ways, if we believe that for most cases such
> >>>>>> error
> >>>>>>>>>> codes would not reach the original clients since they would be
> >>>>>>>>>> disconnected or even gone by that time, and only in some rare
> >> cases
> >>>>>>>>>> they would still be seen by the sending clients, then why not make
> >>>>>>>>>> them more fatal and more specific than generic.
> >>>>>>>>>>
> >>>>>>>>>> Guozhang
> >>>>>>>>>>
> >>>>>>>>>> On Fri, Jan 20, 2023 at 1:59 PM Justine Olshan
> >>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>> Hey Guozhang. Thanks for taking a look and for the detailed
> >>>>>>> comments!
> >>>>>>>>>> I'll
> >>>>>>>>>>> do my best to address below.
> >>>>>>>>>>>
> >>>>>>>>>>> 1. I see what you are saying here, but I think I need to look
> >>>>>>> through
> >>>>>>>>> the
> >>>>>>>>>>> sequence of events you mention. Typically we've seen this issue
> >>>>>> in
> >>>>>>> a
> >>>>>>>>> few
> >>>>>>>>>>> cases.
> >>>>>>>>>>>
> >>>>>>>>>>>     One is when we have a producer disconnect when trying to
> >>>>>> produce.
> >>>>>>>>>>> Typically in these cases, we abort the transaction. We've seen
> >>>>>> that
> >>>>>>>>> after
> >>>>>>>>>>> the markers are written, the disconnection can sometimes cause
> >>>>>> the
> >>>>>>>>>> request
> >>>>>>>>>>> to get flushed to the broker. In this case, we don't need client
> >>>>>>>>> handling
> >>>>>>>>>>> because the producer we are responding to is gone. We just needed
> >>>>>>> to
> >>>>>>>>> make
> >>>>>>>>>>> sure we didn't write to the log on the broker side. I'm trying to
> >>>>>>>> think
> >>>>>>>>>> of
> >>>>>>>>>>> a case where we do have the client to return to. I'd think the
> >>>>>> same
> >>>>>>>>>> client
> >>>>>>>>>>> couldn't progress to committing the transaction unless the
> >>>>>> produce
> >>>>>>>>>> request
> >>>>>>>>>>> returned right? Of course, there is the incorrectly written
> >>>>>> clients
> >>>>>>>>> case.
> >>>>>>>>>>> I'll think on this a bit more and let you know if I come up with
> >>>>>>>>> another
> >>>>>>>>>>> scenario when we would return to an active client when the
> >>>>>>>> transaction
> >>>>>>>>> is
> >>>>>>>>>>> no longer ongoing.
> >>>>>>>>>>>
> >>>>>>>>>>> I was not aware that we checked the result of a send after we
> >>>>>>> commit
> >>>>>>>>>>> though. I'll need to look into that a bit more.
> >>>>>>>>>>>
> >>>>>>>>>>> 2. There were some questions about this in the discussion. The
> >>>>>> plan
> >>>>>>>> is
> >>>>>>>>> to
> >>>>>>>>>>> handle overflow with the mechanism we currently have in the
> >>>>>>> producer.
> >>>>>>>>> If
> >>>>>>>>>> we
> >>>>>>>>>>> try to bump and the epoch will overflow, we actually allocate a
> >>>>>> new
> >>>>>>>>>>> producer ID. I need to confirm the fencing logic on the last
> >>>>>> epoch
> >>>>>>>> (ie,
> >>>>>>>>>> we
> >>>>>>>>>>> probably shouldn't allow any records to be produced with the
> >>>>>> final
> >>>>>>>>> epoch
> >>>>>>>>>>> since we can never properly fence that one).
> >>>>>>>>>>>
> >>>>>>>>>>> 3. I can agree with you that the current error handling is
> >>>>>> messy. I
> >>>>>>>>>> recall
> >>>>>>>>>>> taking a look at your KIP a while back, but I think I mostly saw
> >>>>>>> the
> >>>>>>>>>>> section about how the errors were wrapped. Maybe I need to take
> >>>>>>>> another
> >>>>>>>>>>> look. As for abortable error, the idea was that the handling
> >>>>>> would
> >>>>>>> be
> >>>>>>>>>>> simple -- if this error is seen, the transaction should be
> >>>>>> aborted
> >>>>>>> --
> >>>>>>>>> no
> >>>>>>>>>>> other logic about previous state or requests necessary. Is your
> >>>>>>>> concern
> >>>>>>>>>>> simply about adding new errors? We were hoping to have an error
> >>>>>>> that
> >>>>>>>>>> would
> >>>>>>>>>>> have one meaning and many of the current errors have a history of
> >>>>>>>>> meaning
> >>>>>>>>>>> different things on different client versions. That was the main
> >>>>>>>>>> motivation
> >>>>>>>>>>> for adding a new error.
> >>>>>>>>>>>
> >>>>>>>>>>> 4. This is a good point about record timestamp reordering.
> >>>>>>> Timestamps
> >>>>>>>>>> don't
> >>>>>>>>>>> affect compaction, but they do affect retention deletion. For
> >>>>>> that,
> >>>>>>>>> kafka
> >>>>>>>>>>> considers the largest timestamp in the segment, so I think a
> >>>>>> small
> >>>>>>>>> amount
> >>>>>>>>>>> of reordering (hopefully on the order of milliseconds or even
> >>>>>>>> seconds)
> >>>>>>>>>> will
> >>>>>>>>>>> be ok. We take timestamps from clients so there is already a
> >>>>>>>>> possibility
> >>>>>>>>>>> for some drift and non-monotonically increasing timestamps.
> >>>>>>>>>>>
> >>>>>>>>>>> 5. Thanks for catching. The error is there, but it's actually
> >>>>>> that
> >>>>>>>>> those
> >>>>>>>>>>> fields should be 4+! Due to how the message generator works, I
> >>>>>>>> actually
> >>>>>>>>>>> have to redefine those fields inside the
> >>>>>>>>> `"AddPartitionsToTxnTransaction`
> >>>>>>>>>>> block for it to build correctly. I'll fix it to be correct.
> >>>>>>>>>>>
> >>>>>>>>>>> 6. Correct -- we will only add the request to purgatory if the
> >>>>>>> cache
> >>>>>>>>> has
> >>>>>>>>>> no
> >>>>>>>>>>> ongoing transaction. I can change the wording to make that
> >>>>>> clearer
> >>>>>>>> that
> >>>>>>>>>> we
> >>>>>>>>>>> only place the request in purgatory if we need to contact the
> >>>>>>>>> transaction
> >>>>>>>>>>> coordinator.
> >>>>>>>>>>>
> >>>>>>>>>>> 7. We did take a look at some of the errors and it was hard to
> >>>>>> come
> >>>>>>>> up
> >>>>>>>>>> with
> >>>>>>>>>>> a good one. I agree that InvalidTxnStateException is ideal except
> >>>>>>> for
> >>>>>>>>> the
> >>>>>>>>>>> fact that it hasn't been returned on Produce requests before. The
> >>>>>>>> error
> >>>>>>>>>>> handling for clients is a bit vague (which is why I opened
> >>>>>>>> KAFKA-14439
> >>>>>>>>>>> <https://issues.apache.org/jira/browse/KAFKA-14439>), but the
> >>>>>>>> decision
> >>>>>>>>>> we
> >>>>>>>>>>> made here was to only return errors that have been previously
> >>>>>>>> returned
> >>>>>>>>> to
> >>>>>>>>>>> producers. As for not being fatal, I think part of the theory was
> >>>>>>>> that
> >>>>>>>>> in
> >>>>>>>>>>> many cases, the producer would be disconnected. (See point 1) and
> >>>>>>>> this
> >>>>>>>>>>> would just be an error to return from the server. I did plan to
> >>>>>>> think
> >>>>>>>>>> about
> >>>>>>>>>>> other cases, so let me know if you think of any as well!
> >>>>>>>>>>>
> >>>>>>>>>>> Lots to say! Let me know if you have further thoughts!
> >>>>>>>>>>> Justine
> >>>>>>>>>>>
> >>>>>>>>>>> On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang <
> >>>>>>>>>> guozhang.wang.us@gmail.com>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> Hello Justine,
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thanks for the great write-up! I made a quick pass through it
> >>>>>> and
> >>>>>>>>> here
> >>>>>>>>>>>> are some thoughts (I have not been able to read through this
> >>>>>>> thread
> >>>>>>>>> so
> >>>>>>>>>>>> pardon me if they have overlapped or subsumed by previous
> >>>>>>>> comments):
> >>>>>>>>>>>>
> >>>>>>>>>>>> First are some meta ones:
> >>>>>>>>>>>>
> >>>>>>>>>>>> 1. I think we need to also improve the client's experience once
> >>>>>>> we
> >>>>>>>>>>>> have this defence in place. More concretely, say a user's
> >>>>>>> producer
> >>>>>>>>>>>> code is like following:
> >>>>>>>>>>>>
> >>>>>>>>>>>> future = producer.send();
> >>>>>>>>>>>> // producer.flush();
> >>>>>>>>>>>> producer.commitTransaction();
> >>>>>>>>>>>> future.get();
> >>>>>>>>>>>>
> >>>>>>>>>>>> Which resulted in the order of a) produce-request sent by
> >>>>>>> producer,
> >>>>>>>>> b)
> >>>>>>>>>>>> end-txn-request sent by producer, c) end-txn-response sent
> >>>>>> back,
> >>>>>>> d)
> >>>>>>>>>>>> txn-marker-request sent from coordinator to partition leader,
> >>>>>> e)
> >>>>>>>>>>>> produce-request finally received by the partition leader,
> >>>>>> before
> >>>>>>>> this
> >>>>>>>>>>>> KIP e) step would be accepted causing a dangling txn; now it
> >>>>>>> would
> >>>>>>>> be
> >>>>>>>>>>>> rejected in step e) which is good. But from the client's point
> >>>>>> of
> >>>>>>>>> view
> >>>>>>>>>>>> now it becomes confusing since the `commitTransaction()`
> >>>>>> returns
> >>>>>>>>>>>> successfully, but the "future" throws an invalid-epoch error,
> >>>>>> and
> >>>>>>>>> they
> >>>>>>>>>>>> are not sure if the transaction did succeed or not. In fact, it
> >>>>>>>>>>>> "partially succeeded" with some msgs being rejected but others
> >>>>>>>>>>>> committed successfully.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Of course the easy way to avoid this is, always call
> >>>>>>>>>>>> "producer.flush()" before commitTxn and that's what we do
> >>>>>>>> ourselves,
> >>>>>>>>>>>> and what we recommend users do. But I suspect not everyone does
> >>>>>>> it.
> >>>>>>>>> In
> >>>>>>>>>>>> fact I just checked the javadoc in KafkaProducer and our code
> >>>>>>>> snippet
> >>>>>>>>>>>> does not include a `flush()` call. So I'm thinking maybe we can
> >>>>>>> in
> >>>>>>>>>>>> side the `commitTxn` code to enforce flushing before sending
> >>>>>> the
> >>>>>>>>>>>> end-txn request.
> >>>>>>>>>>>>
> >>>>>>>>>>>> 2. I'd like to clarify a bit details on "just add partitions to
> >>>>>>> the
> >>>>>>>>>>>> transaction on the first produce request during a transaction".
> >>>>>>> My
> >>>>>>>>>>>> understanding is that the partition leader's cache has the
> >>>>>>> producer
> >>>>>>>>> id
> >>>>>>>>>>>> / sequence / epoch for the latest txn, either on-going or is
> >>>>>>>>> completed
> >>>>>>>>>>>> (upon receiving the marker request from coordinator). When a
> >>>>>>>> produce
> >>>>>>>>>>>> request is received, if
> >>>>>>>>>>>>
> >>>>>>>>>>>> * producer's epoch < cached epoch, or producer's epoch ==
> >>>>>> cached
> >>>>>>>>> epoch
> >>>>>>>>>>>> but the latest txn is completed, leader directly reject with
> >>>>>>>>>>>> invalid-epoch.
> >>>>>>>>>>>> * producer's epoch > cached epoch, park the the request and
> >>>>>> send
> >>>>>>>>>>>> add-partitions request to coordinator.
> >>>>>>>>>>>>
> >>>>>>>>>>>> In order to do it, does the coordinator need to bump the
> >>>>>> sequence
> >>>>>>>> and
> >>>>>>>>>>>> reset epoch to 0 when the next epoch is going to overflow? If
> >>>>>> no
> >>>>>>>> need
> >>>>>>>>>>>> to do so, then how we handle the (admittedly rare, but still
> >>>>>> may
> >>>>>>>>>>>> happen) epoch overflow situation?
> >>>>>>>>>>>>
> >>>>>>>>>>>> 3. I'm a bit concerned about adding a generic "ABORTABLE_ERROR"
> >>>>>>>> given
> >>>>>>>>>>>> we already have a pretty messy error classification and error
> >>>>>>>>> handling
> >>>>>>>>>>>> on the producer clients side --- I have a summary about the
> >>>>>>> issues
> >>>>>>>>> and
> >>>>>>>>>>>> a proposal to address this in
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
> >>>>>>>>>>>> -- I understand we do not want to use "UNKNOWN_PRODUCER_ID"
> >>>>>>> anymore
> >>>>>>>>>>>> and in fact we intend to deprecate it in KIP-360 and eventually
> >>>>>>>>> remove
> >>>>>>>>>>>> it; but I'm wondering can we still use specific error codes.
> >>>>>> E.g.
> >>>>>>>>> what
> >>>>>>>>>>>> about "InvalidProducerEpochException" since for new clients,
> >>>>>> the
> >>>>>>>>>>>> actual reason this would actually be rejected is indeed because
> >>>>>>> the
> >>>>>>>>>>>> epoch on the coordinator caused the add-partitions-request from
> >>>>>>> the
> >>>>>>>>>>>> brokers to be rejected anyways?
> >>>>>>>>>>>>
> >>>>>>>>>>>> 4. It seems we put the producer request into purgatory before
> >>>>>> we
> >>>>>>>> ever
> >>>>>>>>>>>> append the records, while other producer's records may still be
> >>>>>>>>>>>> appended during the time; and that potentially may result in
> >>>>>> some
> >>>>>>>>>>>> re-ordering compared with reception order. I'm not super
> >>>>>>> concerned
> >>>>>>>>>>>> about it since Kafka does not guarantee reception ordering
> >>>>>> across
> >>>>>>>>>>>> producers anyways, but it may make the timestamps of records
> >>>>>>>> inside a
> >>>>>>>>>>>> partition to be more out-of-ordered. Are we aware of any
> >>>>>>> scenarios
> >>>>>>>>>>>> such as future enhancements on log compactions that may be
> >>>>>>> affected
> >>>>>>>>> by
> >>>>>>>>>>>> this effect?
> >>>>>>>>>>>>
> >>>>>>>>>>>> Below are just minor comments:
> >>>>>>>>>>>>
> >>>>>>>>>>>> 5. In "AddPartitionsToTxnTransaction" field of
> >>>>>>>>>>>> "AddPartitionsToTxnRequest" RPC, the versions of those inner
> >>>>>>> fields
> >>>>>>>>>>>> are "0-3" while I thought they should be "0+" still?
> >>>>>>>>>>>>
> >>>>>>>>>>>> 6. Regarding "we can place the request in a purgatory of sorts
> >>>>>>> and
> >>>>>>>>>>>> check if there is any state for the transaction on the
> >>>>>> broker": i
> >>>>>>>>>>>> think at this time when we just do the checks against the
> >>>>>> cached
> >>>>>>>>>>>> state, we do not need to put the request to purgatory yet?
> >>>>>>>>>>>>
> >>>>>>>>>>>> 7. This is related to 3) above. I feel using
> >>>>>>>> "InvalidRecordException"
> >>>>>>>>>>>> for older clients may also be a bit confusing, and also it is
> >>>>>> not
> >>>>>>>>>>>> fatal -- for old clients, it better to be fatal since this
> >>>>>>>> indicates
> >>>>>>>>>>>> the clients is doing something wrong and hence it should be
> >>>>>>> closed.
> >>>>>>>>>>>> And in general I'd prefer to use slightly more specific meaning
> >>>>>>>> error
> >>>>>>>>>>>> codes for clients. That being said, I also feel
> >>>>>>>>>>>> "InvalidProducerEpochException" is not suitable for old
> >>>>>> versioned
> >>>>>>>>>>>> clients, and we'd have to pick one that old clients recognize.
> >>>>>>> I'd
> >>>>>>>>>>>> prefer "InvalidTxnStateException" but that one is supposed to
> >>>>>> be
> >>>>>>>>>>>> returned from txn coordinators only today. I'd suggest we do a
> >>>>>>>> quick
> >>>>>>>>>>>> check in the current client's code path and see if that one
> >>>>>> would
> >>>>>>>> be
> >>>>>>>>>>>> handled if it's from a produce-response, and if yes, use this
> >>>>>>> one;
> >>>>>>>>>>>> otherwise, use "ProducerFencedException" which is much less
> >>>>>>>>> meaningful
> >>>>>>>>>>>> but it's still a fatal error.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thanks,
> >>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
> >>>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Yeah -- looks like we already have code to handle bumping the
> >>>>>>>> epoch
> >>>>>>>>>> and
> >>>>>>>>>>>>> when the epoch is Short.MAX_VALUE, we get a new producer ID.
> >>>>>>>> Since
> >>>>>>>>>> this
> >>>>>>>>>>>> is
> >>>>>>>>>>>>> already the behavior, do we want to change it further?
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <
> >>>>>>>>> jolshan@confluent.io
> >>>>>>>>>>>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hey all, just wanted to quickly update and say I've
> >>>>>> modified
> >>>>>>>> the
> >>>>>>>>>> KIP to
> >>>>>>>>>>>>>> explicitly mention that AddOffsetCommitsToTxnRequest will
> >>>>>> be
> >>>>>>>>>> replaced
> >>>>>>>>>>>> by
> >>>>>>>>>>>>>> a coordinator-side (inter-broker) AddPartitionsToTxn
> >>>>>> implicit
> >>>>>>>>>> request.
> >>>>>>>>>>>> This
> >>>>>>>>>>>>>> mirrors the user partitions and will implicitly add offset
> >>>>>>>>>> partitions
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>> transactions when we commit offsets on them. We will
> >>>>>>> deprecate
> >>>>>>>>>>>> AddOffsetCommitsToTxnRequest
> >>>>>>>>>>>>>> for new clients.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Also to address Artem's comments --
> >>>>>>>>>>>>>> I'm a bit unsure if the changes here will change the
> >>>>>> previous
> >>>>>>>>>> behavior
> >>>>>>>>>>>> for
> >>>>>>>>>>>>>> fencing producers. In the case you mention in the first
> >>>>>>>>> paragraph,
> >>>>>>>>>> are
> >>>>>>>>>>>> you
> >>>>>>>>>>>>>> saying we bump the epoch before we try to abort the
> >>>>>>>> transaction?
> >>>>>>>>> I
> >>>>>>>>>>>> think I
> >>>>>>>>>>>>>> need to understand the scenarios you mention a bit better.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> As for the second part -- I think it makes sense to have
> >>>>>> some
> >>>>>>>>> sort
> >>>>>>>>>> of
> >>>>>>>>>>>>>> "sentinel" epoch to signal epoch is about to overflow (I
> >>>>>>> think
> >>>>>>>> we
> >>>>>>>>>> sort
> >>>>>>>>>>>> of
> >>>>>>>>>>>>>> have this value in place in some ways) so we can codify it
> >>>>>> in
> >>>>>>>> the
> >>>>>>>>>> KIP.
> >>>>>>>>>>>> I'll
> >>>>>>>>>>>>>> look into that and try to update soon.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>> Justine.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
> >>>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> It's good to know that KIP-588 addressed some of the
> >>>>>> issues.
> >>>>>>>>>> Looking
> >>>>>>>>>>>> at
> >>>>>>>>>>>>>>> the code, it still looks like there are some cases that
> >>>>>>> would
> >>>>>>>>>> result
> >>>>>>>>>>>> in
> >>>>>>>>>>>>>>> fatal error, e.g. PRODUCER_FENCED is issued by the
> >>>>>>> transaction
> >>>>>>>>>>>> coordinator
> >>>>>>>>>>>>>>> if epoch doesn't match, and the client treats it as a
> >>>>>> fatal
> >>>>>>>>> error
> >>>>>>>>>>>> (code in
> >>>>>>>>>>>>>>> TransactionManager request handling).  If we consider, for
> >>>>>>>>>> example,
> >>>>>>>>>>>>>>> committing a transaction that returns a timeout, but
> >>>>>>> actually
> >>>>>>>>>>>> succeeds,
> >>>>>>>>>>>>>>> trying to abort it or re-commit may result in
> >>>>>>> PRODUCER_FENCED
> >>>>>>>>>> error
> >>>>>>>>>>>>>>> (because of epoch bump).
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> For failed commits, specifically, we need to know the
> >>>>>> actual
> >>>>>>>>>> outcome,
> >>>>>>>>>>>>>>> because if we return an error the application may think
> >>>>>> that
> >>>>>>>> the
> >>>>>>>>>>>>>>> transaction is aborted and redo the work, leading to
> >>>>>>>> duplicates.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Re: overflowing epoch.  We could either do it on the TC
> >>>>>> and
> >>>>>>>>> return
> >>>>>>>>>>>> both
> >>>>>>>>>>>>>>> producer id and epoch (e.g. change the protocol), or
> >>>>>> signal
> >>>>>>>> the
> >>>>>>>>>> client
> >>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>> it needs to get a new producer id.  Checking for max epoch
> >>>>>>>> could
> >>>>>>>>>> be a
> >>>>>>>>>>>>>>> reasonable signal, the value to check should probably be
> >>>>>>>> present
> >>>>>>>>>> in
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>> as this is effectively a part of the contract.  Also, the
> >>>>>> TC
> >>>>>>>>>> should
> >>>>>>>>>>>>>>> probably return an error if the client didn't change
> >>>>>>> producer
> >>>>>>>> id
> >>>>>>>>>> after
> >>>>>>>>>>>>>>> hitting max epoch.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> -Artem
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
> >>>>>>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Thanks for the discussion Artem.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> With respect to the handling of fenced producers, we
> >>>>>> have
> >>>>>>>> some
> >>>>>>>>>>>> behavior
> >>>>>>>>>>>>>>>> already in place. As of KIP-588:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> >>>>>>>>>>>>>>>> ,
> >>>>>>>>>>>>>>>> we handle timeouts more gracefully. The producer can
> >>>>>>>> recover.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Produce requests can also recover from epoch fencing by
> >>>>>>>>>> aborting the
> >>>>>>>>>>>>>>>> transaction and starting over.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> What other cases were you considering that would cause
> >>>>>> us
> >>>>>>> to
> >>>>>>>>>> have a
> >>>>>>>>>>>>>>> fenced
> >>>>>>>>>>>>>>>> epoch but we'd want to recover?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> The first point about handling epoch overflows is fair.
> >>>>>> I
> >>>>>>>>> think
> >>>>>>>>>>>> there is
> >>>>>>>>>>>>>>>> some logic we'd need to consider. (ie, if we are one
> >>>>>> away
> >>>>>>>> from
> >>>>>>>>>> the
> >>>>>>>>>>>> max
> >>>>>>>>>>>>>>>> epoch, we need to reset the producer ID.) I'm still
> >>>>>>>> wondering
> >>>>>>>>> if
> >>>>>>>>>>>> there
> >>>>>>>>>>>>>>> is a
> >>>>>>>>>>>>>>>> way to direct this from the response, or if everything
> >>>>>>>> should
> >>>>>>>>> be
> >>>>>>>>>>>> done on
> >>>>>>>>>>>>>>>> the client side. Let me know if you have any thoughts
> >>>>>>> here.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
> >>>>>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> There are some workflows in the client that are
> >>>>>> implied
> >>>>>>> by
> >>>>>>>>>>>> protocol
> >>>>>>>>>>>>>>>>> changes, e.g.:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> - for new clients, epoch changes with every
> >>>>>> transaction
> >>>>>>>> and
> >>>>>>>>>> can
> >>>>>>>>>>>>>>> overflow,
> >>>>>>>>>>>>>>>>> in old clients this condition was handled
> >>>>>> transparently,
> >>>>>>>>>> because
> >>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>> bumped in InitProducerId and it would return a new
> >>>>>>>> producer
> >>>>>>>>>> id if
> >>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>> overflows, the new clients would need to implement
> >>>>>> some
> >>>>>>>>>> workflow
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> refresh
> >>>>>>>>>>>>>>>>> producer id
> >>>>>>>>>>>>>>>>> - how to handle fenced producers, for new clients
> >>>>>> epoch
> >>>>>>>>>> changes
> >>>>>>>>>>>> with
> >>>>>>>>>>>>>>>> every
> >>>>>>>>>>>>>>>>> transaction, so in presence of failures during
> >>>>>> commits /
> >>>>>>>>>> aborts,
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> producer could get easily fenced, old clients would
> >>>>>>> pretty
> >>>>>>>>>> much
> >>>>>>>>>>>> would
> >>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>> fenced when a new incarnation of the producer was
> >>>>>>>>> initialized
> >>>>>>>>>> with
> >>>>>>>>>>>>>>>>> InitProducerId so it's ok to treat as a fatal error,
> >>>>>> the
> >>>>>>>> new
> >>>>>>>>>>>> clients
> >>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>> need to implement some workflow to handle that error,
> >>>>>>>>>> otherwise
> >>>>>>>>>>>> they
> >>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>> get fenced by themselves
> >>>>>>>>>>>>>>>>> - in particular (as a subset of the previous issue),
> >>>>>>> what
> >>>>>>>>>> would
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> client
> >>>>>>>>>>>>>>>>> do if it got a timeout during commit?  commit could've
> >>>>>>>>>> succeeded
> >>>>>>>>>>>> or
> >>>>>>>>>>>>>>>> failed
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Not sure if this has to be defined in the KIP as
> >>>>>>>>> implementing
> >>>>>>>>>>>> those
> >>>>>>>>>>>>>>>>> probably wouldn't require protocol changes, but we
> >>>>>> have
> >>>>>>>>>> multiple
> >>>>>>>>>>>>>>>>> implementations of Kafka clients, so probably would be
> >>>>>>>> good
> >>>>>>>>> to
> >>>>>>>>>>>> have
> >>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>> client implementation guidance.  Could also be done
> >>>>>> as a
> >>>>>>>>>> separate
> >>>>>>>>>>>> doc.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> -Artem
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
> >>>>>>>>>>>>>>>> <jolshan@confluent.io.invalid
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Hey all, I've updated the KIP to incorporate Jason's
> >>>>>>>>>>>> suggestions.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 1. Use AddPartitionsToTxn + verify flag to check on
> >>>>>>> old
> >>>>>>>>>> clients
> >>>>>>>>>>>>>>>>>> 2. Updated AddPartitionsToTxn API to support
> >>>>>>> transaction
> >>>>>>>>>>>> batching
> >>>>>>>>>>>>>>>>>> 3. Mention IBP bump
> >>>>>>>>>>>>>>>>>> 4. Mention auth change on new AddPartitionsToTxn
> >>>>>>>> version.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I'm planning on opening a vote soon.
> >>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <
> >>>>>>>>>>>> jolshan@confluent.io
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thanks Jason. Those changes make sense to me. I
> >>>>>> will
> >>>>>>>>>> update
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>> KIP.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
> >>>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Hey Justine,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> I was wondering about compatibility here. When
> >>>>>> we
> >>>>>>>>> send
> >>>>>>>>>>>> requests
> >>>>>>>>>>>>>>>>>>>> between brokers, we want to ensure that the
> >>>>>>> receiving
> >>>>>>>>>> broker
> >>>>>>>>>>>>>>>>> understands
> >>>>>>>>>>>>>>>>>>>> the request (specifically the new fields).
> >>>>>>> Typically
> >>>>>>>>>> this is
> >>>>>>>>>>>> done
> >>>>>>>>>>>>>>>> via
> >>>>>>>>>>>>>>>>>>>> IBP/metadata version.
> >>>>>>>>>>>>>>>>>>>> I'm trying to think if there is a way around it
> >>>>>> but
> >>>>>>>> I'm
> >>>>>>>>>> not
> >>>>>>>>>>>> sure
> >>>>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>>> is.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Yes. I think we would gate usage of this behind
> >>>>>> an
> >>>>>>>> IBP
> >>>>>>>>>> bump.
> >>>>>>>>>>>> Does
> >>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>> seem
> >>>>>>>>>>>>>>>>>>>> reasonable?
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> As for the improvements -- can you clarify how
> >>>>>>> the
> >>>>>>>>>> multiple
> >>>>>>>>>>>>>>>>>>>> transactional
> >>>>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking of a case
> >>>>>>>> where
> >>>>>>>>> we
> >>>>>>>>>>>>>>> wait/batch
> >>>>>>>>>>>>>>>>>>>> multiple produce requests together? My
> >>>>>>> understanding
> >>>>>>>>> for
> >>>>>>>>>> now
> >>>>>>>>>>>> was
> >>>>>>>>>>>>>>> 1
> >>>>>>>>>>>>>>>>>>>> transactional ID and one validation per 1 produce
> >>>>>>>>>> request.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Each call to `AddPartitionsToTxn` is essentially
> >>>>>> a
> >>>>>>>>> write
> >>>>>>>>>> to
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>> log and must block on replication. The more we
> >>>>>> can
> >>>>>>>> fit
> >>>>>>>>>> into a
> >>>>>>>>>>>>>>> single
> >>>>>>>>>>>>>>>>>>>> request, the more writes we can do in parallel.
> >>>>>> The
> >>>>>>>>>>>> alternative
> >>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>>>> use of more connections, but usually we prefer
> >>>>>>>> batching
> >>>>>>>>>>>> since the
> >>>>>>>>>>>>>>>>>> network
> >>>>>>>>>>>>>>>>>>>> stack is not really optimized for high
> >>>>>>>>> connection/request
> >>>>>>>>>>>> loads.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Finally with respect to the authorizations, I
> >>>>>>> think
> >>>>>>>>> it
> >>>>>>>>>>>> makes
> >>>>>>>>>>>>>>> sense
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> skip
> >>>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit confused by
> >>>>>> the
> >>>>>>>>>> "leader
> >>>>>>>>>>>> ID"
> >>>>>>>>>>>>>>>> field.
> >>>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the request as
> >>>>>> from a
> >>>>>>>>>> broker
> >>>>>>>>>>>> (does
> >>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>> matter
> >>>>>>>>>>>>>>>>>>>> which one?).
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> We could also make it version-based. For the next
> >>>>>>>>>> version, we
> >>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>>> require
> >>>>>>>>>>>>>>>>>>>> CLUSTER auth. So clients would not be able to use
> >>>>>>> the
> >>>>>>>>> API
> >>>>>>>>>>>>>>> anymore,
> >>>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>> probably what we want.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> -Jason
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> >>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> As a follow up, I was just thinking about the
> >>>>>>>>> batching
> >>>>>>>>>> a
> >>>>>>>>>>>> bit
> >>>>>>>>>>>>>>> more.
> >>>>>>>>>>>>>>>>>>>>> I suppose if we have one request in flight and
> >>>>>> we
> >>>>>>>>>> queue up
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>> produce requests in some sort of purgatory, we
> >>>>>>>> could
> >>>>>>>>>> send
> >>>>>>>>>>>>>>>>> information
> >>>>>>>>>>>>>>>>>>>> out
> >>>>>>>>>>>>>>>>>>>>> for all of them rather than one by one. So that
> >>>>>>>> would
> >>>>>>>>>> be a
> >>>>>>>>>>>>>>> benefit
> >>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>> batching partitions to add per transaction.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> I'll need to think a bit more on the design of
> >>>>>>> this
> >>>>>>>>>> part
> >>>>>>>>>>>> of the
> >>>>>>>>>>>>>>>> KIP,
> >>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>> will update the KIP in the next few days.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan
> >>>>>> <
> >>>>>>>>>>>>>>>>> jolshan@confluent.io>
> >>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Hey Jason -- thanks for the input -- I was
> >>>>>> just
> >>>>>>>>>> digging
> >>>>>>>>>>>> a bit
> >>>>>>>>>>>>>>>>> deeper
> >>>>>>>>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>>>>>>> the design + implementation of the validation
> >>>>>>>> calls
> >>>>>>>>>> here
> >>>>>>>>>>>> and
> >>>>>>>>>>>>>>>> what
> >>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>> say
> >>>>>>>>>>>>>>>>>>>>>> makes sense.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> I was wondering about compatibility here.
> >>>>>> When
> >>>>>>> we
> >>>>>>>>>> send
> >>>>>>>>>>>>>>> requests
> >>>>>>>>>>>>>>>>>>>>>> between brokers, we want to ensure that the
> >>>>>>>>> receiving
> >>>>>>>>>>>> broker
> >>>>>>>>>>>>>>>>>>>> understands
> >>>>>>>>>>>>>>>>>>>>>> the request (specifically the new fields).
> >>>>>>>>> Typically
> >>>>>>>>>>>> this is
> >>>>>>>>>>>>>>>> done
> >>>>>>>>>>>>>>>>>> via
> >>>>>>>>>>>>>>>>>>>>>> IBP/metadata version.
> >>>>>>>>>>>>>>>>>>>>>> I'm trying to think if there is a way around
> >>>>>> it
> >>>>>>>> but
> >>>>>>>>>> I'm
> >>>>>>>>>>>> not
> >>>>>>>>>>>>>>> sure
> >>>>>>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>>>>>> is.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> As for the improvements -- can you clarify
> >>>>>> how
> >>>>>>>> the
> >>>>>>>>>>>> multiple
> >>>>>>>>>>>>>>>>>>>> transactional
> >>>>>>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking of a
> >>>>>>> case
> >>>>>>>>>> where we
> >>>>>>>>>>>>>>>>> wait/batch
> >>>>>>>>>>>>>>>>>>>>>> multiple produce requests together? My
> >>>>>>>>> understanding
> >>>>>>>>>> for
> >>>>>>>>>>>> now
> >>>>>>>>>>>>>>>> was 1
> >>>>>>>>>>>>>>>>>>>>>> transactional ID and one validation per 1
> >>>>>>> produce
> >>>>>>>>>>>> request.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Finally with respect to the authorizations, I
> >>>>>>>> think
> >>>>>>>>>> it
> >>>>>>>>>>>> makes
> >>>>>>>>>>>>>>>> sense
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> skip
> >>>>>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit confused
> >>>>>> by
> >>>>>>>> the
> >>>>>>>>>>>> "leader
> >>>>>>>>>>>>>>> ID"
> >>>>>>>>>>>>>>>>>> field.
> >>>>>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the request as
> >>>>>>>> from a
> >>>>>>>>>>>> broker
> >>>>>>>>>>>>>>> (does
> >>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>> matter
> >>>>>>>>>>>>>>>>>>>>>> which one?).
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> I think I want to adopt these suggestions,
> >>>>>> just
> >>>>>>>> had
> >>>>>>>>>> a few
> >>>>>>>>>>>>>>>>> questions
> >>>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> details.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On Thu, Jan 5, 2023 at 5:05 PM Jason
> >>>>>> Gustafson
> >>>>>>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
> >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Thanks for the proposal.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> I was thinking about the implementation a
> >>>>>>> little
> >>>>>>>>>> bit.
> >>>>>>>>>>>> In the
> >>>>>>>>>>>>>>>>>> current
> >>>>>>>>>>>>>>>>>>>>>>> proposal, the behavior depends on whether we
> >>>>>>>> have
> >>>>>>>>> an
> >>>>>>>>>>>> old or
> >>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>> client.
> >>>>>>>>>>>>>>>>>>>>>>> For
> >>>>>>>>>>>>>>>>>>>>>>> old clients, we send `DescribeTransactions`
> >>>>>>> and
> >>>>>>>>>> verify
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> result
> >>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>> new clients, we send `AddPartitionsToTxn`.
> >>>>>> We
> >>>>>>>>> might
> >>>>>>>>>> be
> >>>>>>>>>>>> able
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> simplify
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> implementation if we can use the same
> >>>>>> request
> >>>>>>>>> type.
> >>>>>>>>>> For
> >>>>>>>>>>>>>>>> example,
> >>>>>>>>>>>>>>>>>>>> what if
> >>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>> bump the protocol version for
> >>>>>>>> `AddPartitionsToTxn`
> >>>>>>>>>> and
> >>>>>>>>>>>> add a
> >>>>>>>>>>>>>>>>>>>>>>> `validateOnly`
> >>>>>>>>>>>>>>>>>>>>>>> flag? For older versions, we can set
> >>>>>>>>>>>> `validateOnly=true` so
> >>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> request only returns successfully if the
> >>>>>>>> partition
> >>>>>>>>>> had
> >>>>>>>>>>>>>>> already
> >>>>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>>>>> added.
> >>>>>>>>>>>>>>>>>>>>>>> For new versions, we can set
> >>>>>>>> `validateOnly=false`
> >>>>>>>>>> and
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>> added to the transaction. The other slightly
> >>>>>>>>>> annoying
> >>>>>>>>>>>> thing
> >>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>> get around is the need to collect the
> >>>>>>>> transaction
> >>>>>>>>>> state
> >>>>>>>>>>>> for
> >>>>>>>>>>>>>>> all
> >>>>>>>>>>>>>>>>>>>>> partitions
> >>>>>>>>>>>>>>>>>>>>>>> even when we only care about a subset.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Some additional improvements to consider:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> - We can give `AddPartitionsToTxn` better
> >>>>>>> batch
> >>>>>>>>>> support
> >>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>> inter-broker
> >>>>>>>>>>>>>>>>>>>>>>> usage. Currently we only allow one
> >>>>>>>>>> `TransactionalId` to
> >>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>> specified,
> >>>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>> the broker may get some benefit being able
> >>>>>> to
> >>>>>>>>> batch
> >>>>>>>>>>>> across
> >>>>>>>>>>>>>>>>> multiple
> >>>>>>>>>>>>>>>>>>>>>>> transactions.
> >>>>>>>>>>>>>>>>>>>>>>> - Another small improvement is skipping
> >>>>>> topic
> >>>>>>>>>>>> authorization
> >>>>>>>>>>>>>>>>> checks
> >>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>> `AddPartitionsToTxn` when the request is
> >>>>>> from
> >>>>>>> a
> >>>>>>>>>> broker.
> >>>>>>>>>>>>>>> Perhaps
> >>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>> add
> >>>>>>>>>>>>>>>>>>>>>>> a field for the `LeaderId` or something like
> >>>>>>>> that
> >>>>>>>>>> and
> >>>>>>>>>>>>>>> require
> >>>>>>>>>>>>>>>>>> CLUSTER
> >>>>>>>>>>>>>>>>>>>>>>> permission when set.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>>>>> Jason
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
> >>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation. It makes sense
> >>>>>>> to
> >>>>>>>> me
> >>>>>>>>>> now.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Jun
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 1:42 PM Justine
> >>>>>>> Olshan
> >>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> My understanding of the mechanism is
> >>>>>> that
> >>>>>>>> when
> >>>>>>>>>> we
> >>>>>>>>>>>> get to
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> last
> >>>>>>>>>>>>>>>>>>>>>>> epoch,
> >>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>> increment to the fencing/last epoch and
> >>>>>> if
> >>>>>>>> any
> >>>>>>>>>>>> further
> >>>>>>>>>>>>>>>>> requests
> >>>>>>>>>>>>>>>>>>>> come
> >>>>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>> this producer ID they are fenced. Then
> >>>>>> the
> >>>>>>>>>> producer
> >>>>>>>>>>>>>>> gets a
> >>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>> ID
> >>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>> restarts with epoch/sequence 0. The
> >>>>>> fenced
> >>>>>>>>> epoch
> >>>>>>>>>>>> sticks
> >>>>>>>>>>>>>>>>> around
> >>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> duration of producer.id.expiration.ms
> >>>>>> and
> >>>>>>>>>> blocks
> >>>>>>>>>>>> any
> >>>>>>>>>>>>>>> late
> >>>>>>>>>>>>>>>>>>>> messages
> >>>>>>>>>>>>>>>>>>>>>>>> there.
> >>>>>>>>>>>>>>>>>>>>>>>>> The new ID will get to take advantage of
> >>>>>>> the
> >>>>>>>>>>>> improved
> >>>>>>>>>>>>>>>>> semantics
> >>>>>>>>>>>>>>>>>>>>> around
> >>>>>>>>>>>>>>>>>>>>>>>>> non-zero start sequences. So I think we
> >>>>>>> are
> >>>>>>>>>> covered.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> The only potential issue is overloading
> >>>>>>> the
> >>>>>>>>>> cache,
> >>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>> hopefully
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> improvements (lowered
> >>>>>>>>> producer.id.expiration.ms
> >>>>>>>>>> )
> >>>>>>>>>>>> will
> >>>>>>>>>>>>>>> help
> >>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>> that.
> >>>>>>>>>>>>>>>>>>>>>>>> Let
> >>>>>>>>>>>>>>>>>>>>>>>>> me know if you still have concerns.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
> >>>>>>>>>>>>>>>>>>>> <ju...@confluent.io.invalid>
> >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> 70. The proposed fencing logic doesn't
> >>>>>>>> apply
> >>>>>>>>>> when
> >>>>>>>>>>>> pid
> >>>>>>>>>>>>>>>>>> changes,
> >>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>> right? If so, I am not sure how
> >>>>>> complete
> >>>>>>>> we
> >>>>>>>>>> are
> >>>>>>>>>>>>>>>> addressing
> >>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>> issue
> >>>>>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>>> the pid changes more frequently.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 16, 2022 at 9:16 AM
> >>>>>> Justine
> >>>>>>>>> Olshan
> >>>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for replying!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> 70.We already do the overflow
> >>>>>>> mechanism,
> >>>>>>>>> so
> >>>>>>>>>> my
> >>>>>>>>>>>>>>> change
> >>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>> happen more often.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> I was also not suggesting a new
> >>>>>> field
> >>>>>>> in
> >>>>>>>>> the
> >>>>>>>>>>>> log,
> >>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> response,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> which would be gated by the client
> >>>>>>>>> version.
> >>>>>>>>>>>> Sorry if
> >>>>>>>>>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>> unclear. I think we are starting to
> >>>>>>>>> diverge.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> The goal of this KIP is to not
> >>>>>> change
> >>>>>>> to
> >>>>>>>>> the
> >>>>>>>>>>>> marker
> >>>>>>>>>>>>>>>>> format
> >>>>>>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>>>>>> all.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> 71. Yes, I guess I was going under
> >>>>>> the
> >>>>>>>>>>>> assumption
> >>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> log
> >>>>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>>>>>> look at its last epoch and treat it
> >>>>>> as
> >>>>>>>> the
> >>>>>>>>>>>> current
> >>>>>>>>>>>>>>>>> epoch. I
> >>>>>>>>>>>>>>>>>>>>>>> suppose
> >>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>> have some special logic that if the
> >>>>>>> last
> >>>>>>>>>> epoch
> >>>>>>>>>>>> was
> >>>>>>>>>>>>>>> on a
> >>>>>>>>>>>>>>>>>>>> marker
> >>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>>>>>>>>> expect the next epoch or something
> >>>>>>> like
> >>>>>>>>>> that. We
> >>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>> distinguish
> >>>>>>>>>>>>>>>>>>>>>>>>>>> based on whether we had a
> >>>>>> commit/abort
> >>>>>>>>>> marker.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> 72.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> if the producer epoch hasn't been
> >>>>>>>> bumped
> >>>>>>>>>> on
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
> >>>>>>>> message
> >>>>>>>>>> will
> >>>>>>>>>>>> fail
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> sequence
> >>>>>>>>>>>>>>>>>>>>>>>>>> validation
> >>>>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the producer
> >>>>>>>> epoch
> >>>>>>>>>> has
> >>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>> bumped,
> >>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>> ignore
> >>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck message
> >>>>>>>> could
> >>>>>>>>>> be
> >>>>>>>>>>>>>>> appended
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> log.
> >>>>>>>>>>>>>>>>>>>>>>>> So,
> >>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
> >>>>>> guard?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I follow that "the
> >>>>>>> message
> >>>>>>>>> will
> >>>>>>>>>>>> fail
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> sequence
> >>>>>>>>>>>>>>>>>>>>>>>>>> validation".
> >>>>>>>>>>>>>>>>>>>>>>>>>>> In some of these cases, we had an
> >>>>>>> abort
> >>>>>>>>>> marker
> >>>>>>>>>>>> (due
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>> error)
> >>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>> then
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the late message comes in with the
> >>>>>>>> correct
> >>>>>>>>>>>> sequence
> >>>>>>>>>>>>>>>>> number.
> >>>>>>>>>>>>>>>>>>>> This
> >>>>>>>>>>>>>>>>>>>>>>> is a
> >>>>>>>>>>>>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>>>>>>>> covered by the KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> The latter case is actually not
> >>>>>>>> something
> >>>>>>>>>> we've
> >>>>>>>>>>>>>>>>> considered
> >>>>>>>>>>>>>>>>>>>>> here. I
> >>>>>>>>>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>>>>>> generally when we bump the epoch, we
> >>>>>>> are
> >>>>>>>>>>>> accepting
> >>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> sequence
> >>>>>>>>>>>>>>>>>>>>>>>>> does
> >>>>>>>>>>>>>>>>>>>>>>>>>>> not need to be checked anymore. My
> >>>>>>>>>>>> understanding is
> >>>>>>>>>>>>>>>> also
> >>>>>>>>>>>>>>>>>>>> that we
> >>>>>>>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>>>>>> typically bump epoch mid transaction
> >>>>>>>>> (based
> >>>>>>>>>> on a
> >>>>>>>>>>>>>>> quick
> >>>>>>>>>>>>>>>>> look
> >>>>>>>>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> code)
> >>>>>>>>>>>>>>>>>>>>>>>>>>> but let me know if that is the case.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 15, 2022 at 12:23 PM Jun
> >>>>>>> Rao
> >>>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the reply.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Assigning a new pid on int
> >>>>>>>> overflow
> >>>>>>>>>> seems
> >>>>>>>>>>>> a
> >>>>>>>>>>>>>>> bit
> >>>>>>>>>>>>>>>>>> hacky.
> >>>>>>>>>>>>>>>>>>>> If
> >>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>> need a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> txn
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> level id, it will be better to
> >>>>>> model
> >>>>>>>>> this
> >>>>>>>>>>>>>>> explicitly.
> >>>>>>>>>>>>>>>>>>>> Adding a
> >>>>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>>>>> field
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> would require a bit more work
> >>>>>> since
> >>>>>>> it
> >>>>>>>>>>>> requires a
> >>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>> txn
> >>>>>>>>>>>>>>>>>>>>> marker
> >>>>>>>>>>>>>>>>>>>>>>>>> format
> >>>>>>>>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the log. So, we probably need to
> >>>>>>> guard
> >>>>>>>>> it
> >>>>>>>>>>>> with an
> >>>>>>>>>>>>>>> IBP
> >>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>> metadata
> >>>>>>>>>>>>>>>>>>>>>>>>>> version
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> and document the impact on
> >>>>>> downgrade
> >>>>>>>>> once
> >>>>>>>>>> the
> >>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>> format
> >>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>> written
> >>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> 71. Hmm, once the marker is
> >>>>>> written,
> >>>>>>>> the
> >>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>> expect
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> next
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> append to be on the next epoch.
> >>>>>> Does
> >>>>>>>>> that
> >>>>>>>>>>>> cover
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>> mentioned?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> 72. Also, just to be clear on the
> >>>>>>>>> stucked
> >>>>>>>>>>>> message
> >>>>>>>>>>>>>>>> issue
> >>>>>>>>>>>>>>>>>>>>>>> described
> >>>>>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> motivation. With EoS, we also
> >>>>>>> validate
> >>>>>>>>> the
> >>>>>>>>>>>>>>> sequence
> >>>>>>>>>>>>>>>> id
> >>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>> idempotency.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> So,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> with the current logic, if the
> >>>>>>>> producer
> >>>>>>>>>> epoch
> >>>>>>>>>>>>>>> hasn't
> >>>>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>>>>>>> bumped on
> >>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
> >>>>>>>>> message
> >>>>>>>>>> will
> >>>>>>>>>>>>>>> fail
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> sequence
> >>>>>>>>>>>>>>>>>>>>>>>>>>> validation
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the
> >>>>>> producer
> >>>>>>>>>> epoch has
> >>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>>>> bumped, we
> >>>>>>>>>>>>>>>>>>>>>>>> ignore
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck
> >>>>>> message
> >>>>>>>>>> could be
> >>>>>>>>>>>>>>>> appended
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> log.
> >>>>>>>>>>>>>>>>>>>>>>>>> So,
> >>>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
> >>>>>>> guard?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 10:44 AM
> >>>>>>>> Justine
> >>>>>>>>>>>> Olshan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> >>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias — thanks again for
> >>>>>> taking
> >>>>>>>>> time
> >>>>>>>>>> to
> >>>>>>>>>>>> look
> >>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>> this.
> >>>>>>>>>>>>>>>>>>>> You
> >>>>>>>>>>>>>>>>>>>>>>> said:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My proposal was only focusing
> >>>>>> to
> >>>>>>>>> avoid
> >>>>>>>>>>>>>>> dangling
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
> >>>>>> added
> >>>>>>>>>> without
> >>>>>>>>>>>>>>>> registered
> >>>>>>>>>>>>>>>>>>>>>>> partition.
> >>>>>>>>>>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more details
> >>>>>> to
> >>>>>>>> the
> >>>>>>>>>> KIP
> >>>>>>>>>>>> about
> >>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>> scenario
> >>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>> better
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I understand what
> >>>>>> you
> >>>>>>>>> mean
> >>>>>>>>>>>> here.
> >>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>>> motivation
> >>>>>>>>>>>>>>>>>>>>>>>>> section
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> describes two scenarios about
> >>>>>> how
> >>>>>>>> the
> >>>>>>>>>> record
> >>>>>>>>>>>>>>> can be
> >>>>>>>>>>>>>>>>>> added
> >>>>>>>>>>>>>>>>>>>>>>>> without a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> registered partition:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Another way hanging
> >>>>>> transactions
> >>>>>>>> can
> >>>>>>>>>>>> occur is
> >>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>> client
> >>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>> buggy
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> may somehow try to write to a
> >>>>>>>>> partition
> >>>>>>>>>>>> before
> >>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>> adds
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> For the first example of this
> >>>>>>> would
> >>>>>>>> it
> >>>>>>>>>> be
> >>>>>>>>>>>>>>> helpful
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> say
> >>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> comes in after the abort, but
> >>>>>>> before
> >>>>>>>>> the
> >>>>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>> added
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> next
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction so it becomes
> >>>>>>> "hanging."
> >>>>>>>>>>>> Perhaps the
> >>>>>>>>>>>>>>>> next
> >>>>>>>>>>>>>>>>>>>>> sentence
> >>>>>>>>>>>>>>>>>>>>>>>>>>> describing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the message becoming part of the
> >>>>>>>> next
> >>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>> (a
> >>>>>>>>>>>>>>>>>>>>> different
> >>>>>>>>>>>>>>>>>>>>>>>>> case)
> >>>>>>>>>>>>>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not properly differentiated.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun — thanks for reading the
> >>>>>> KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. The int typing was a
> >>>>>> concern.
> >>>>>>>>>> Currently
> >>>>>>>>>>>> we
> >>>>>>>>>>>>>>>> have a
> >>>>>>>>>>>>>>>>>>>>>>> mechanism
> >>>>>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>>> place
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> fence the final epoch when the
> >>>>>>> epoch
> >>>>>>>>> is
> >>>>>>>>>>>> about to
> >>>>>>>>>>>>>>>>>> overflow
> >>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>> assign
> >>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer ID with epoch 0. Of
> >>>>>>> course,
> >>>>>>>>>> this
> >>>>>>>>>>>> is a
> >>>>>>>>>>>>>>> bit
> >>>>>>>>>>>>>>>>>> tricky
> >>>>>>>>>>>>>>>>>>>>>>> when it
> >>>>>>>>>>>>>>>>>>>>>>>>>> comes
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response back to the client.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Making this a long could be
> >>>>>>> another
> >>>>>>>>>> option,
> >>>>>>>>>>>> but
> >>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>> wonder
> >>>>>>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> implications on changing this
> >>>>>>> field
> >>>>>>>> if
> >>>>>>>>>> the
> >>>>>>>>>>>>>>> epoch is
> >>>>>>>>>>>>>>>>>>>>> persisted
> >>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>> disk?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to check the usages.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 71.This was something Matthias
> >>>>>>> asked
> >>>>>>>>>> about
> >>>>>>>>>>>> as
> >>>>>>>>>>>>>>>> well. I
> >>>>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>>>>>>>> considering a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible edge case where a
> >>>>>> produce
> >>>>>>>>>> request
> >>>>>>>>>>>> from
> >>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>> somehow
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> gets sent right after the marker
> >>>>>>> is
> >>>>>>>>>>>> written, but
> >>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> alerted of the newly bumped
> >>>>>> epoch.
> >>>>>>>> In
> >>>>>>>>>> this
> >>>>>>>>>>>>>>> case, we
> >>>>>>>>>>>>>>>>> may
> >>>>>>>>>>>>>>>>>>>>>>> include
> >>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> record
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we don't want to. I suppose
> >>>>>>> we
> >>>>>>>>>> could
> >>>>>>>>>>>> try
> >>>>>>>>>>>>>>> to do
> >>>>>>>>>>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>>>>>>>> client
> >>>>>>>>>>>>>>>>>>>>>>>>>>> side
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to bump the epoch after sending
> >>>>>> an
> >>>>>>>>>> endTxn as
> >>>>>>>>>>>>>>> well
> >>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>> scenario
> >>>>>>>>>>>>>>>>>>>>>>>>> —
> >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wonder how it would work when
> >>>>>> the
> >>>>>>>>>> server is
> >>>>>>>>>>>>>>>> aborting
> >>>>>>>>>>>>>>>>>>>> based
> >>>>>>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> error. I could also be missing
> >>>>>>>>>> something and
> >>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>> scenario
> >>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again to everyone reading
> >>>>>>> and
> >>>>>>>>>>>> commenting.
> >>>>>>>>>>>>>>>> Let
> >>>>>>>>>>>>>>>>> me
> >>>>>>>>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> further questions or comments.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 9:41 AM
> >>>>>>> Jun
> >>>>>>>>> Rao
> >>>>>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP. A couple
> >>>>>> of
> >>>>>>>>>> comments.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Currently, the producer
> >>>>>>> epoch
> >>>>>>>> is
> >>>>>>>>>> an
> >>>>>>>>>>>> int.
> >>>>>>>>>>>>>>> I am
> >>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>> sure
> >>>>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> enough
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to accommodate all
> >>>>>> transactions
> >>>>>>> in
> >>>>>>>>> the
> >>>>>>>>>>>>>>> lifetime
> >>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>> producer.
> >>>>>>>>>>>>>>>>>>>>>>>>>> Should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> change that to a long or add a
> >>>>>>> new
> >>>>>>>>>> long
> >>>>>>>>>>>> field
> >>>>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>>>> txnId?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 71. "it will write the prepare
> >>>>>>>>> commit
> >>>>>>>>>>>> message
> >>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>> bumped
> >>>>>>>>>>>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> send
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WriteTxnMarkerRequests with
> >>>>>> the
> >>>>>>>>> bumped
> >>>>>>>>>>>> epoch."
> >>>>>>>>>>>>>>>> Hmm,
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> associated
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the current txn right?
> >>>>>> So,
> >>>>>>> it
> >>>>>>>>>> seems
> >>>>>>>>>>>>>>> weird to
> >>>>>>>>>>>>>>>>>>>> write a
> >>>>>>>>>>>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with a bumped epoch. Should we
> >>>>>>>> only
> >>>>>>>>>> bump
> >>>>>>>>>>>> up
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>>> EndTxnResponse
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rename the field to sth like
> >>>>>>>>>>>>>>> nextProducerEpoch?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 12, 2022 at 8:54
> >>>>>> PM
> >>>>>>>>>> Matthias
> >>>>>>>>>>>> J.
> >>>>>>>>>>>>>>> Sax <
> >>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the background.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30: SGTM. My proposal was
> >>>>>>>> only
> >>>>>>>>>>>> focusing
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> avoid
> >>>>>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
> >>>>>>>> added
> >>>>>>>>>>>> without
> >>>>>>>>>>>>>>>>>> registered
> >>>>>>>>>>>>>>>>>>>>>>>> partition.
> >>>>>>>>>>>>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more
> >>>>>> details
> >>>>>>>> to
> >>>>>>>>>> the
> >>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>> scenario
> >>>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40: I think you hit a fair
> >>>>>>> point
> >>>>>>>>>> about
> >>>>>>>>>>>> race
> >>>>>>>>>>>>>>>>>>>> conditions
> >>>>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>>> client
> >>>>>>>>>>>>>>>>>>>>>>>>>>> bugs
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (incorrectly not bumping the
> >>>>>>>>>> epoch). The
> >>>>>>>>>>>>>>>>>>>>>>> complexity/confusion
> >>>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> using
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the bumped epoch I see, is
> >>>>>>>> mainly
> >>>>>>>>>> for
> >>>>>>>>>>>>>>> internal
> >>>>>>>>>>>>>>>>>>>>> debugging,
> >>>>>>>>>>>>>>>>>>>>>>> ie,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> inspecting
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log segment dumps -- it
> >>>>>> seems
> >>>>>>>>>> harder to
> >>>>>>>>>>>>>>> reason
> >>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> system
> >>>>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> us
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> humans. But if we get better
> >>>>>>>>>>>> guarantees, it
> >>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>> worth to
> >>>>>>>>>>>>>>>>>>>>>>>>> use
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped epoch.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60: as I mentioned already,
> >>>>>> I
> >>>>>>>>> don't
> >>>>>>>>>>>> know the
> >>>>>>>>>>>>>>>>> broker
> >>>>>>>>>>>>>>>>>>>>>>> internals
> >>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> provide
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more input. So if nobody
> >>>>>> else
> >>>>>>>>> chimes
> >>>>>>>>>>>> in, we
> >>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>> move
> >>>>>>>>>>>>>>>>>>>>>>>>>>> forward
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with your proposal.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 12/6/22 4:22 PM, Justine
> >>>>>>>> Olshan
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi all,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> After Artem's questions
> >>>>>>> about
> >>>>>>>>>> error
> >>>>>>>>>>>>>>> behavior,
> >>>>>>>>>>>>>>>>>> I've
> >>>>>>>>>>>>>>>>>>>>>>>>> re-evaluated
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unknown producer ID
> >>>>>>> exception
> >>>>>>>>> and
> >>>>>>>>>> had
> >>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>> discussions
> >>>>>>>>>>>>>>>>>>>>>>>>> offline.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think generally it makes
> >>>>>>>> sense
> >>>>>>>>>> to
> >>>>>>>>>>>>>>> simplify
> >>>>>>>>>>>>>>>>>> error
> >>>>>>>>>>>>>>>>>>>>>>> handling
> >>>>>>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> cases
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this and the
> >>>>>>>> UNKNOWN_PRODUCER_ID
> >>>>>>>>>> error
> >>>>>>>>>>>>>>> has a
> >>>>>>>>>>>>>>>>>> pretty
> >>>>>>>>>>>>>>>>>>>>> long
> >>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complicated
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> history. Because of this,
> >>>>>> I
> >>>>>>>>>> propose
> >>>>>>>>>>>>>>> adding a
> >>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>> error
> >>>>>>>>>>>>>>>>>>>>>>> code
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ABORTABLE_ERROR
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that when encountered by
> >>>>>> new
> >>>>>>>>>> clients
> >>>>>>>>>>>>>>> (gated
> >>>>>>>>>>>>>>>> by
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> produce
> >>>>>>>>>>>>>>>>>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will simply abort the
> >>>>>>>>> transaction.
> >>>>>>>>>>>> This
> >>>>>>>>>>>>>>>> allows
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> server
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> say
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in whether the client
> >>>>>> aborts
> >>>>>>>> and
> >>>>>>>>>> makes
> >>>>>>>>>>>>>>>> handling
> >>>>>>>>>>>>>>>>>>>> much
> >>>>>>>>>>>>>>>>>>>>>>>> simpler.
> >>>>>>>>>>>>>>>>>>>>>>>>>> In
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> future, we can also use
> >>>>>> this
> >>>>>>>>>> error in
> >>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>> situations
> >>>>>>>>>>>>>>>>>>>>>>>> where
> >>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> want
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> abort the transactions. We
> >>>>>>> can
> >>>>>>>>>> even
> >>>>>>>>>>>> use on
> >>>>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>> apis.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've added this to the
> >>>>>> KIP.
> >>>>>>>> Let
> >>>>>>>>> me
> >>>>>>>>>>>> know if
> >>>>>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>>>>>>> questions
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 2, 2022 at
> >>>>>> 10:22
> >>>>>>>> AM
> >>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>> Olshan
> >>>>>>>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> jolshan@confluent.io
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey Matthias,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30 — Maybe I also
> >>>>>> didn't
> >>>>>>>>>> express
> >>>>>>>>>>>>>>> myself
> >>>>>>>>>>>>>>>>>>>> clearly.
> >>>>>>>>>>>>>>>>>>>>> For
> >>>>>>>>>>>>>>>>>>>>>>>>> older
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't have a way to
> >>>>>>>> distinguish
> >>>>>>>>>>>> between a
> >>>>>>>>>>>>>>>>>> previous
> >>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> current
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction since we
> >>>>>> don't
> >>>>>>>> have
> >>>>>>>>>> the
> >>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>> bump.
> >>>>>>>>>>>>>>>>>>>> This
> >>>>>>>>>>>>>>>>>>>>>>> means
> >>>>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> late
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message from the previous
> >>>>>>>>>> transaction
> >>>>>>>>>>>>>>> may be
> >>>>>>>>>>>>>>>>>>>> added to
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>>>>>> one.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> With
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older clients — we can't
> >>>>>>>>>> guarantee
> >>>>>>>>>>>> this
> >>>>>>>>>>>>>>>> won't
> >>>>>>>>>>>>>>>>>>>> happen
> >>>>>>>>>>>>>>>>>>>>>>> if we
> >>>>>>>>>>>>>>>>>>>>>>>>>>> already
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call
> >>>>>>> (why
> >>>>>>>> we
> >>>>>>>>>> make
> >>>>>>>>>>>>>>> changes
> >>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> newer
> >>>>>>>>>>>>>>>>>>>>>>>>>>> client)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can at least gate some by
> >>>>>>>>>> ensuring
> >>>>>>>>>>>> that
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>>>>>>> has
> >>>>>>>>>>>>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> added
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. The
> >>>>>> rationale
> >>>>>>>> here
> >>>>>>>>>> is
> >>>>>>>>>>>> that
> >>>>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>>>>>> likely
> >>>>>>>>>>>>>>>>>>>>>>>>> LESS
> >>>>>>>>>>>>>>>>>>>>>>>>>>> late
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrivals
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as time goes on, so
> >>>>>>> hopefully
> >>>>>>>>>> most
> >>>>>>>>>>>> late
> >>>>>>>>>>>>>>>>> arrivals
> >>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>> come
> >>>>>>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> BEFORE
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call.
> >>>>>>>> Those
> >>>>>>>>>> that
> >>>>>>>>>>>>>>> arrive
> >>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>> properly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> gated
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> >>>>>>> describeTransactions
> >>>>>>>>>>>> approach.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we take the approach
> >>>>>> you
> >>>>>>>>>>>> suggested,
> >>>>>>>>>>>>>>> ANY
> >>>>>>>>>>>>>>>>> late
> >>>>>>>>>>>>>>>>>>>>> arrival
> >>>>>>>>>>>>>>>>>>>>>>>>> from a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> previous
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction will be
> >>>>>> added.
> >>>>>>>> And
> >>>>>>>>> we
> >>>>>>>>>>>> don't
> >>>>>>>>>>>>>>> want
> >>>>>>>>>>>>>>>>>>>> that. I
> >>>>>>>>>>>>>>>>>>>>>>> also
> >>>>>>>>>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> see
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> benefit in sending
> >>>>>>>>>> addPartitionsToTxn
> >>>>>>>>>>>>>>> over
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> describeTxns
> >>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> They
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> both be one extra RPC to
> >>>>>>> the
> >>>>>>>>> Txn
> >>>>>>>>>>>>>>>> coordinator.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To be clear — newer
> >>>>>> clients
> >>>>>>>>> will
> >>>>>>>>>> use
> >>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> >>>>>>>>>>>>>>>>>>>>>>>>>> instead
> >>>>>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTxns.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that if we
> >>>>>>> have
> >>>>>>>>>> some
> >>>>>>>>>>>> delay
> >>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> client
> >>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>> bump
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it could continue to send
> >>>>>>>> epoch
> >>>>>>>>>> 73
> >>>>>>>>>>>> and
> >>>>>>>>>>>>>>> those
> >>>>>>>>>>>>>>>>>>>> records
> >>>>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fenced.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Perhaps this is not an
> >>>>>>> issue
> >>>>>>>> if
> >>>>>>>>>> we
> >>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>> allow
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> next
> >>>>>>>>>>>>>>>>>>>>>>>>>> produce
> >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> go
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through before the EndTxn
> >>>>>>>>> request
> >>>>>>>>>>>>>>> returns.
> >>>>>>>>>>>>>>>> I'm
> >>>>>>>>>>>>>>>>>>>> also
> >>>>>>>>>>>>>>>>>>>>>>>> thinking
> >>>>>>>>>>>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cases of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure. I will need to
> >>>>>>> think
> >>>>>>>>> on
> >>>>>>>>>>>> this a
> >>>>>>>>>>>>>>> bit.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I wasn't sure if it was
> >>>>>>> that
> >>>>>>>>>>>> confusing.
> >>>>>>>>>>>>>>> But
> >>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>> think it
> >>>>>>>>>>>>>>>>>>>>>>>>> is,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> investigate other ways.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure these are
> >>>>>> the
> >>>>>>>> same
> >>>>>>>>>>>>>>> purgatories
> >>>>>>>>>>>>>>>>>> since
> >>>>>>>>>>>>>>>>>>>> one
> >>>>>>>>>>>>>>>>>>>>>>> is a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> produce
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory (I was planning
> >>>>>>> on
> >>>>>>>>>> using a
> >>>>>>>>>>>>>>>> callback
> >>>>>>>>>>>>>>>>>>>> rather
> >>>>>>>>>>>>>>>>>>>>>>> than
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the other is simply a
> >>>>>>> request
> >>>>>>>>> to
> >>>>>>>>>>>> append
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> log.
> >>>>>>>>>>>>>>>>>>>>> Not
> >>>>>>>>>>>>>>>>>>>>>>>> sure
> >>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> structure here for
> >>>>>>> ordering,
> >>>>>>>>> but
> >>>>>>>>>> my
> >>>>>>>>>>>>>>>>>> understanding
> >>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handle the write request
> >>>>>>>> before
> >>>>>>>>>> it
> >>>>>>>>>>>> hears
> >>>>>>>>>>>>>>>> back
> >>>>>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> Txn
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Coordinator.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if I
> >>>>>>>> misunderstood
> >>>>>>>>>>>> something
> >>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>>>>>>>>> unclear.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 1, 2022 at
> >>>>>>> 12:15
> >>>>>>>> PM
> >>>>>>>>>>>> Matthias
> >>>>>>>>>>>>>>> J.
> >>>>>>>>>>>>>>>>> Sax
> >>>>>>>>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the details
> >>>>>>>>> Justine!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side change
> >>>>>>> for
> >>>>>>>> 2
> >>>>>>>>> is
> >>>>>>>>>>>>>>> removing
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> addPartitions
> >>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need to
> >>>>>>> make
> >>>>>>>>>> this
> >>>>>>>>>>>> from
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> txn
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think I did not
> >>>>>> express
> >>>>>>>>> myself
> >>>>>>>>>>>>>>> clearly. I
> >>>>>>>>>>>>>>>>>>>>> understand
> >>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should) change the
> >>>>>>> producer
> >>>>>>>> to
> >>>>>>>>>> not
> >>>>>>>>>>>> send
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> longer. But I don't
> >>>>>> thinks
> >>>>>>>>> it's
> >>>>>>>>>>>>>>> requirement
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> change
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> broker?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What I am trying to say
> >>>>>>> is:
> >>>>>>>>> as a
> >>>>>>>>>>>>>>> safe-guard
> >>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>> improvement
> >>>>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producers, the partition
> >>>>>>>>> leader
> >>>>>>>>>> can
> >>>>>>>>>>>> just
> >>>>>>>>>>>>>>>> send
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request to the
> >>>>>>>> TX-coordinator
> >>>>>>>>>> in any
> >>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> old
> >>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> correctly did send the
> >>>>>>>>>>>> `addPartition`
> >>>>>>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> already, the
> >>>>>>> TX-coordinator
> >>>>>>>>> can
> >>>>>>>>>> just
> >>>>>>>>>>>>>>>> "ignore"
> >>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>>>>> idempotent.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if the old producer has
> >>>>>> a
> >>>>>>>> bug
> >>>>>>>>>> and
> >>>>>>>>>>>> did
> >>>>>>>>>>>>>>>> forget
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> sent
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `addPartition`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request, we would now
> >>>>>>> ensure
> >>>>>>>>>> that
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>> indeed
> >>>>>>>>>>>>>>>>>>>>>>>>>> added
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX and thus fix a
> >>>>>>> potential
> >>>>>>>>>>>> producer bug
> >>>>>>>>>>>>>>>>> (even
> >>>>>>>>>>>>>>>>>>>> if we
> >>>>>>>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fencing via the bump
> >>>>>>> epoch).
> >>>>>>>>> --
> >>>>>>>>>> It
> >>>>>>>>>>>>>>> seems to
> >>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>> good
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Or
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there a reason to not do
> >>>>>>>> this?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is ongoing
> >>>>>> =
> >>>>>>>>>> partition
> >>>>>>>>>>>> was
> >>>>>>>>>>>>>>>> added
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>> via
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn. We
> >>>>>>>> check
> >>>>>>>>>> this
> >>>>>>>>>>>> with
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this wasn't
> >>>>>>>>>> sufficiently
> >>>>>>>>>>>>>>>>> explained
> >>>>>>>>>>>>>>>>>>>> here:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do what I propose
> >>>>>> in
> >>>>>>>>>> (20), we
> >>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>> really
> >>>>>>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `DescribeTransaction`
> >>>>>>> call,
> >>>>>>>> as
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>>>> leader
> >>>>>>>>>>>>>>>>>>>>>>> adds
> >>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for older clients and we
> >>>>>>> get
> >>>>>>>>>> this
> >>>>>>>>>>>> check
> >>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>> free.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is that
> >>>>>> if
> >>>>>>>> any
> >>>>>>>>>>>> messages
> >>>>>>>>>>>>>>>>> somehow
> >>>>>>>>>>>>>>>>>>>> come
> >>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the producer,
> >>>>>>> they
> >>>>>>>>>> will be
> >>>>>>>>>>>>>>>> fenced.
> >>>>>>>>>>>>>>>>>>>>> However,
> >>>>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it can be
> >>>>>>>>>> discussed
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree that we should
> >>>>>>> have
> >>>>>>>>>> epoch
> >>>>>>>>>>>>>>> fencing.
> >>>>>>>>>>>>>>>> My
> >>>>>>>>>>>>>>>>>>>>>>> question is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> different:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Assume we are at epoch
> >>>>>> 73,
> >>>>>>>> and
> >>>>>>>>>> we
> >>>>>>>>>>>> have
> >>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>> ongoing
> >>>>>>>>>>>>>>>>>>>>>>>>>> transaction,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed. It seems
> >>>>>>> natural
> >>>>>>>> to
> >>>>>>>>>>>> write the
> >>>>>>>>>>>>>>>>>> "prepare
> >>>>>>>>>>>>>>>>>>>>>>> commit"
> >>>>>>>>>>>>>>>>>>>>>>>>>>> marker
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
> >>>>>>> both
> >>>>>>>>> with
> >>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>> 73,
> >>>>>>>>>>>>>>>>> too,
> >>>>>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>> belongs
> >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current transaction. Of
> >>>>>>>>> course,
> >>>>>>>>>> we
> >>>>>>>>>>>> now
> >>>>>>>>>>>>>>> also
> >>>>>>>>>>>>>>>>>> bump
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> expect
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the next requests to
> >>>>>> have
> >>>>>>>>> epoch
> >>>>>>>>>> 74,
> >>>>>>>>>>>> and
> >>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>> reject
> >>>>>>>>>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch 73, as the
> >>>>>>>> corresponding
> >>>>>>>>>> TX
> >>>>>>>>>>>> for
> >>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>> 73
> >>>>>>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>>>>>> already
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems you propose to
> >>>>>>>> write
> >>>>>>>>>> the
> >>>>>>>>>>>>>>> "prepare
> >>>>>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>>>>>> marker"
> >>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
> >>>>>>> with
> >>>>>>>>>> epoch 74
> >>>>>>>>>>>>>>>> though,
> >>>>>>>>>>>>>>>>>> what
> >>>>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>> work,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems confusing. Is
> >>>>>> there
> >>>>>>> a
> >>>>>>>>>> reason
> >>>>>>>>>>>> why
> >>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>> use
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> bumped
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 74
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of the current
> >>>>>>> epoch
> >>>>>>>>> 73?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are checking if
> >>>>>>> the
> >>>>>>>>>>>>>>> transaction is
> >>>>>>>>>>>>>>>>>>>> ongoing,
> >>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the leader
> >>>>>>>>> partition
> >>>>>>>>>> to
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for this
> >>>>>>>>>> message to
> >>>>>>>>>>>> come
> >>>>>>>>>>>>>>>>> back,
> >>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>> theory
> >>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
> >>>>>> that
> >>>>>>>>> would
> >>>>>>>>>>>> make the
> >>>>>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>>>>>>>>> result
> >>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why we
> >>>>>> can
> >>>>>>>>> check
> >>>>>>>>>> the
> >>>>>>>>>>>>>>> leader
> >>>>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>> write
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks. Got it.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, is this really
> >>>>>> an
> >>>>>>>>>> issue?
> >>>>>>>>>>>> We put
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> produce
> >>>>>>>>>>>>>>>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory, so how could
> >>>>>> we
> >>>>>>>>>> process
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> first?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Don't we need to put the
> >>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
> >>>>>>>>>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> too,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for this case, and
> >>>>>> process
> >>>>>>>>> both
> >>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>>> in-order?
> >>>>>>>>>>>>>>>>>>>>>>> (Again,
> >>>>>>>>>>>>>>>>>>>>>>>>> my
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> broker
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> knowledge is limited and
> >>>>>>>> maybe
> >>>>>>>>>> we
> >>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>> maintain
> >>>>>>>>>>>>>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>>>>>>>>>>> order
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case, what seems to be
> >>>>>> an
> >>>>>>>>> issue
> >>>>>>>>>>>> IMHO,
> >>>>>>>>>>>>>>> and I
> >>>>>>>>>>>>>>>>> am
> >>>>>>>>>>>>>>>>>>>>>>> wondering
> >>>>>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> changing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request handling to
> >>>>>>> preserve
> >>>>>>>>>> order
> >>>>>>>>>>>> for
> >>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>>>> might be
> >>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> cleaner
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution?)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/30/22 3:28 PM,
> >>>>>> Artem
> >>>>>>>>>> Livshits
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think the interesting
> >>>>>>>> part
> >>>>>>>>> is
> >>>>>>>>>>>> not in
> >>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>> logic
> >>>>>>>>>>>>>>>>>>>>>>>> (because
> >>>>>>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> tries
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> figure out when
> >>>>>>>>>>>> UNKNOWN_PRODUCER_ID is
> >>>>>>>>>>>>>>>>>> retriable
> >>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retryable,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's definitely not
> >>>>>>> fatal),
> >>>>>>>>> but
> >>>>>>>>>>>> what
> >>>>>>>>>>>>>>>> happens
> >>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>> logic
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'true' and falls
> >>>>>> through.
> >>>>>>>> In
> >>>>>>>>>> the
> >>>>>>>>>>>> old
> >>>>>>>>>>>>>>>>> clients
> >>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> keep the behavior in
> >>>>>> the
> >>>>>>>> new
> >>>>>>>>>>>> clients,
> >>>>>>>>>>>>>>> I'd
> >>>>>>>>>>>>>>>>>>>> expect it
> >>>>>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> well.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 29, 2022 at
> >>>>>>>> 11:57
> >>>>>>>>>> AM
> >>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>> Olshan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>> <jolshan@confluent.io.invalid
> >>>>>>>>>>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Artem and Jeff,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
> >>>>>> look
> >>>>>>>> and
> >>>>>>>>>>>> sorry for
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> slow
> >>>>>>>>>>>>>>>>>>>>>>>> response.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You both mentioned the
> >>>>>>>>> change
> >>>>>>>>>> to
> >>>>>>>>>>>>>>> handle
> >>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear — this error
> >>>>>> code
> >>>>>>>> will
> >>>>>>>>>> only
> >>>>>>>>>>>> be
> >>>>>>>>>>>>>>> sent
> >>>>>>>>>>>>>>>>>> again
> >>>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> client's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version is high enough
> >>>>>>> to
> >>>>>>>>>> ensure
> >>>>>>>>>>>> we
> >>>>>>>>>>>>>>>> handle
> >>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>> correctly.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The current (Java)
> >>>>>>> client
> >>>>>>>>>> handles
> >>>>>>>>>>>>>>> this by
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> following
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> (somewhat
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> long)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code snippet:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // An
> >>>>>>> UNKNOWN_PRODUCER_ID
> >>>>>>>>>> means
> >>>>>>>>>>>> that
> >>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>> lost
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker. Depending on
> >>>>>> the
> >>>>>>>> log
> >>>>>>>>>> start
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // offset, we may want
> >>>>>>> to
> >>>>>>>>>> retry
> >>>>>>>>>>>>>>> these, as
> >>>>>>>>>>>>>>>>>>>>> described
> >>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>> each
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below. If
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> none of those apply,
> >>>>>>> then
> >>>>>>>>> for
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // idempotent
> >>>>>> producer,
> >>>>>>> we
> >>>>>>>>>> will
> >>>>>>>>>>>>>>> locally
> >>>>>>>>>>>>>>>>> bump
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> reset
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence numbers of
> >>>>>>>>> in-flight
> >>>>>>>>>>>> batches
> >>>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // sequence 0, then
> >>>>>>> retry
> >>>>>>>>> the
> >>>>>>>>>>>> failed
> >>>>>>>>>>>>>>>> batch,
> >>>>>>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>>>>>>>> now
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> succeed.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the transactional
> >>>>>>>> producer,
> >>>>>>>>>> allow
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // batch to fail. When
> >>>>>>>>>> processing
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> failed
> >>>>>>>>>>>>>>>>>>>>> batch,
> >>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transition
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an abortable error and
> >>>>>>>> set a
> >>>>>>>>>> flag
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // indicating that we
> >>>>>>> need
> >>>>>>>>> to
> >>>>>>>>>>>> bump the
> >>>>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>>>> (if
> >>>>>>>>>>>>>>>>>>>>>>>>> supported
> >>>>>>>>>>>>>>>>>>>>>>>>>> by
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if (error ==
> >>>>>>>>>>>>>>>> Errors.*UNKNOWN_PRODUCER_ID*)
> >>>>>>>>>>>>>>>>> {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> >>>>>>>>>> (response.logStartOffset
> >>>>>>>>>>>> ==
> >>>>>>>>>>>>>>> -1)
> >>>>>>>>>>>>>>>> {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // We don't
> >>>>>>> know
> >>>>>>>>>> the log
> >>>>>>>>>>>>>>> start
> >>>>>>>>>>>>>>>>>> offset
> >>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> response.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the request
> >>>>>>>> until
> >>>>>>>>>> we
> >>>>>>>>>>>> get
> >>>>>>>>>>>>>>> it.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // The
> >>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> >>>>>>>>>>>>>>>> error
> >>>>>>>>>>>>>>>>>> code
> >>>>>>>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>>>>>> added
> >>>>>>>>>>>>>>>>>>>>>>>>>>> along
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProduceResponse which
> >>>>>>>>>> includes the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              //
> >>>>>>>> logStartOffset.
> >>>>>>>>>> So
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>> '-1'
> >>>>>>>>>>>>>>>>>>>> sentinel
> >>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> backward
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatibility.
> >>>>>> Instead,
> >>>>>>> it
> >>>>>>>>> is
> >>>>>>>>>>>> possible
> >>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // a broker
> >>>>>> to
> >>>>>>>> not
> >>>>>>>>>> know
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> logStartOffset at
> >>>>>>>>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returning
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response because
> >>>>>> the
> >>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // may have
> >>>>>>>> moved
> >>>>>>>>>> away
> >>>>>>>>>>>> from
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> broker
> >>>>>>>>>>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> time
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error was
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initially raised to
> >>>>>> the
> >>>>>>>> time
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // response
> >>>>>>> was
> >>>>>>>>>> being
> >>>>>>>>>>>>>>>>> constructed.
> >>>>>>>>>>>>>>>>>> In
> >>>>>>>>>>>>>>>>>>>>>>> these
> >>>>>>>>>>>>>>>>>>>>>>>>>> cases,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retry the request: we
> >>>>>>> are
> >>>>>>>>>>>> guaranteed
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // to
> >>>>>>> eventually
> >>>>>>>>>> get a
> >>>>>>>>>>>>>>>>>> logStartOffset
> >>>>>>>>>>>>>>>>>>>>> once
> >>>>>>>>>>>>>>>>>>>>>>>>> things
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> settle
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> down.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> >>>>>>>>>>>>>>> (batch.sequenceHasBeenReset()) {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // When the
> >>>>>>>> first
> >>>>>>>>>>>> inflight
> >>>>>>>>>>>>>>>> batch
> >>>>>>>>>>>>>>>>>>>> fails
> >>>>>>>>>>>>>>>>>>>>>>> due to
> >>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> truncation
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> then the sequences of
> >>>>>>> all
> >>>>>>>>> the
> >>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // in flight
> >>>>>>>>> batches
> >>>>>>>>>>>> would
> >>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>>>>>>>> restarted
> >>>>>>>>>>>>>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> beginning.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, when those
> >>>>>>>>> responses
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // come back
> >>>>>>>> from
> >>>>>>>>>> the
> >>>>>>>>>>>>>>> broker,
> >>>>>>>>>>>>>>>>> they
> >>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>> also
> >>>>>>>>>>>>>>>>>>>>>>>>>> come
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> >>>>>>> error.
> >>>>>>>>> In
> >>>>>>>>>> this
> >>>>>>>>>>>>>>> case,
> >>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // reset the
> >>>>>>>>>> sequence
> >>>>>>>>>>>>>>> numbers
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> beginning.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          } else if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>> (lastAckedOffset(batch.topicPartition).orElse(
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> >>>>>>>>>>>>>>>>>>>>>>>>>> response.logStartOffset) {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // The head
> >>>>>> of
> >>>>>>>> the
> >>>>>>>>>> log
> >>>>>>>>>>>> has
> >>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>>>> removed,
> >>>>>>>>>>>>>>>>>>>>>>>>> probably
> >>>>>>>>>>>>>>>>>>>>>>>>>>> due
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retention time
> >>>>>> elapsing.
> >>>>>>>> In
> >>>>>>>>>> this
> >>>>>>>>>>>> case,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // we expect
> >>>>>>> to
> >>>>>>>>>> lose the
> >>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>>>> state.
> >>>>>>>>>>>>>>>>>>>>>>> For
> >>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer, reset the
> >>>>>>>>> sequences
> >>>>>>>>>> of
> >>>>>>>>>>>> all
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // inflight
> >>>>>>>>> batches
> >>>>>>>>>> to
> >>>>>>>>>>>> be
> >>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> beginning
> >>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> retry
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the transaction
> >>>>>>> does
> >>>>>>>>> not
> >>>>>>>>>>>> need to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // be
> >>>>>> aborted.
> >>>>>>>> For
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>> idempotent
> >>>>>>>>>>>>>>>>>>>>>>> producer,
> >>>>>>>>>>>>>>>>>>>>>>>>> bump
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> avoid
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reusing (sequence,
> >>>>>>> epoch)
> >>>>>>>>>> pairs
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              if
> >>>>>>>>>> (isTransactional()) {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>> txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>> this.producerIdAndEpoch);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              } else {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> requestEpochBumpForPartition(batch.topicPartition);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
> >>>>>>>>> (!isTransactional())
> >>>>>>>>>> {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // For the
> >>>>>>>>>> idempotent
> >>>>>>>>>>>>>>> producer,
> >>>>>>>>>>>>>>>>>>>> always
> >>>>>>>>>>>>>>>>>>>>>>> retry
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors. If the batch
> >>>>>> has
> >>>>>>>> the
> >>>>>>>>>>>> current
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // producer
> >>>>>> ID
> >>>>>>>> and
> >>>>>>>>>>>> epoch,
> >>>>>>>>>>>>>>>>> request a
> >>>>>>>>>>>>>>>>>>>> bump
> >>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Otherwise
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the
> >>>>>> produce.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>> requestEpochBumpForPartition(batch.topicPartition);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I was considering
> >>>>>>> keeping
> >>>>>>>>> this
> >>>>>>>>>>>>>>> behavior —
> >>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>> am
> >>>>>>>>>>>>>>>>>>>>>>> open
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simplifying
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We are leaving changes
> >>>>>>> to
> >>>>>>>>>> older
> >>>>>>>>>>>>>>> clients
> >>>>>>>>>>>>>>>> off
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> table
> >>>>>>>>>>>>>>>>>>>>>>>>> here
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> since
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> caused
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many issues for
> >>>>>> clients
> >>>>>>> in
> >>>>>>>>> the
> >>>>>>>>>>>> past.
> >>>>>>>>>>>>>>>>>> Previously
> >>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we didn't have the
> >>>>>>>>> mechanisms
> >>>>>>>>>> in
> >>>>>>>>>>>>>>> place to
> >>>>>>>>>>>>>>>>>>>> detect
> >>>>>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> legitimate
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case vs some bug or
> >>>>>> gap
> >>>>>>> in
> >>>>>>>>> the
> >>>>>>>>>>>>>>> protocol.
> >>>>>>>>>>>>>>>>>>>> Ensuring
> >>>>>>>>>>>>>>>>>>>>>>> each
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> its
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> own epoch should close
> >>>>>>>> this
> >>>>>>>>>> gap.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> And to address Jeff's
> >>>>>>>> second
> >>>>>>>>>>>> point:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *does the typical
> >>>>>>> produce
> >>>>>>>>>> request
> >>>>>>>>>>>> path
> >>>>>>>>>>>>>>>>> append
> >>>>>>>>>>>>>>>>>>>>>>> records
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> local
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> log
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along*
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *with the
> >>>>>>>>>> currentTxnFirstOffset
> >>>>>>>>>>>>>>>>> information?
> >>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand*
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *when the field is
> >>>>>>> written
> >>>>>>>>> to
> >>>>>>>>>>>> disk.*
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, the first produce
> >>>>>>>>> request
> >>>>>>>>>>>>>>> populates
> >>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>> field
> >>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>> writes
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> offset
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as part of the record
> >>>>>>>> batch
> >>>>>>>>>> and
> >>>>>>>>>>>> also
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> snapshot.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we reload the records
> >>>>>> on
> >>>>>>>>>> restart
> >>>>>>>>>>>>>>> and/or
> >>>>>>>>>>>>>>>>>>>>>>> reassignment,
> >>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> repopulate
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> field with the
> >>>>>> snapshot
> >>>>>>>> from
> >>>>>>>>>> disk
> >>>>>>>>>>>>>>> along
> >>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> rest
> >>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if there
> >>>>>> are
> >>>>>>>>>> further
> >>>>>>>>>>>>>>> comments
> >>>>>>>>>>>>>>>>>>>> and/or
> >>>>>>>>>>>>>>>>>>>>>>>>> questions.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> >>>>>> at
> >>>>>>>> 9:00
> >>>>>>>>>> PM
> >>>>>>>>>>>> Jeff
> >>>>>>>>>>>>>>> Kim
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>> <jeff.kim@confluent.io.invalid
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP! I
> >>>>>>>> have
> >>>>>>>>>> two
> >>>>>>>>>>>>>>>> questions:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1) For new clients,
> >>>>>> we
> >>>>>>>> can
> >>>>>>>>>> once
> >>>>>>>>>>>> again
> >>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>>>> error
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for sequences
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that are non-zero
> >>>>>> when
> >>>>>>>>> there
> >>>>>>>>>> is
> >>>>>>>>>>>> no
> >>>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>>>>>> present
> >>>>>>>>>>>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This will indicate we
> >>>>>>>>> missed
> >>>>>>>>>> the
> >>>>>>>>>>>> 0
> >>>>>>>>>>>>>>>>> sequence
> >>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>>>>> yet
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> want
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the log.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would like to
> >>>>>>>> understand
> >>>>>>>>>> the
> >>>>>>>>>>>>>>> current
> >>>>>>>>>>>>>>>>>>>> behavior
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>> handle
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and if there are any
> >>>>>>>>> changes
> >>>>>>>>>> we
> >>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>> making.
> >>>>>>>>>>>>>>>>>>>> Maybe
> >>>>>>>>>>>>>>>>>>>>>>> I'm
> >>>>>>>>>>>>>>>>>>>>>>>>>>> missing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but we would want to
> >>>>>>>>> identify
> >>>>>>>>>>>>>>> whether we
> >>>>>>>>>>>>>>>>>>>> missed
> >>>>>>>>>>>>>>>>>>>>>>> the 0
> >>>>>>>>>>>>>>>>>>>>>>>>>>> sequence
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients, no?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2) Upon returning
> >>>>>> from
> >>>>>>>> the
> >>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>> coordinator, we
> >>>>>>>>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>> set
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as ongoing on the
> >>>>>>> leader
> >>>>>>>> by
> >>>>>>>>>>>>>>> populating
> >>>>>>>>>>>>>>>>>>>>>>>>>> currentTxnFirstOffset
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through the typical
> >>>>>>>> produce
> >>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>>> handling.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does the typical
> >>>>>>> produce
> >>>>>>>>>> request
> >>>>>>>>>>>> path
> >>>>>>>>>>>>>>>>> append
> >>>>>>>>>>>>>>>>>>>>>>> records
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> local
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> log
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> >>>>>>>>>> currentTxnFirstOffset
> >>>>>>>>>>>>>>>>> information?
> >>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when the field is
> >>>>>>> written
> >>>>>>>>> to
> >>>>>>>>>>>> disk.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeff
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> >>>>>> at
> >>>>>>>>> 4:44
> >>>>>>>>>> PM
> >>>>>>>>>>>> Artem
> >>>>>>>>>>>>>>>>>> Livshits
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
> >>>>>>> alivshits@confluent.io
> >>>>>>>>>> .invalid>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for the
> >>>>>> KIP.
> >>>>>>>> I
> >>>>>>>>>> have
> >>>>>>>>>>>> one
> >>>>>>>>>>>>>>>>>> question.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 5) For new clients,
> >>>>>> we
> >>>>>>>> can
> >>>>>>>>>> once
> >>>>>>>>>>>>>>> again
> >>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>>>> error
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe we had
> >>>>>>>> problems
> >>>>>>>>>> in the
> >>>>>>>>>>>>>>> past
> >>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>> returning
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because it was
> >>>>>>>> considered
> >>>>>>>>>> fatal
> >>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>> required
> >>>>>>>>>>>>>>>>>>>>>>> client
> >>>>>>>>>>>>>>>>>>>>>>>>>>> restart.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> It
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good to spell out
> >>>>>> the
> >>>>>>>> new
> >>>>>>>>>> client
> >>>>>>>>>>>>>>>> behavior
> >>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>> receives
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> >>>>>>> at
> >>>>>>>>>> 10:00 AM
> >>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>> Olshan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>> <jo...@confluent.io.invalid>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
> >>>>>>>> look
> >>>>>>>>>>>> Matthias.
> >>>>>>>>>>>>>>>> I've
> >>>>>>>>>>>>>>>>>>>> tried
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>> answer
> >>>>>>>>>>>>>>>>>>>>>>>>>>> your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> questions
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 10)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Right — so the
> >>>>>>> hanging
> >>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>>>> occurs
> >>>>>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> come in and the
> >>>>>>>> partition
> >>>>>>>>>> is
> >>>>>>>>>>>> never
> >>>>>>>>>>>>>>>> added
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> again.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never add the
> >>>>>>> partition
> >>>>>>>>> to
> >>>>>>>>>> a
> >>>>>>>>>>>>>>>>> transaction,
> >>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>> never
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> marker
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never advance the
> >>>>>>> LSO.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do end up
> >>>>>>> adding
> >>>>>>>>> the
> >>>>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>> (I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suppose
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happen before or
> >>>>>>> after
> >>>>>>>>> the
> >>>>>>>>>> late
> >>>>>>>>>>>>>>>> message
> >>>>>>>>>>>>>>>>>>>> comes
> >>>>>>>>>>>>>>>>>>>>> in)
> >>>>>>>>>>>>>>>>>>>>>>>> then
> >>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> include
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late message in the
> >>>>>>>> next
> >>>>>>>>>>>>>>> (incorrect)
> >>>>>>>>>>>>>>>>>>>>> transaction.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> So perhaps it is
> >>>>>>>> clearer
> >>>>>>>>> to
> >>>>>>>>>>>> make
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> distinction
> >>>>>>>>>>>>>>>>>>>>>>>>> between
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> eventually get
> >>>>>> added
> >>>>>>> to
> >>>>>>>>> the
> >>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>> (but
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> wrong
> >>>>>>>>>>>>>>>>>>>>>>>>>>> one)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that never get
> >>>>>> added
> >>>>>>>> and
> >>>>>>>>>> become
> >>>>>>>>>>>>>>>> hanging.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side
> >>>>>>> change
> >>>>>>>>> for
> >>>>>>>>>> 2 is
> >>>>>>>>>>>>>>>> removing
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> addPartitions
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need
> >>>>>>> to
> >>>>>>>>> make
> >>>>>>>>>>>> this
> >>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> txn
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In my opinion, the
> >>>>>>>> issue
> >>>>>>>>>> with
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> >>>>>>>>>>>>>>>>>>>>>>>>>> call
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is that we don't
> >>>>>> have
> >>>>>>>> the
> >>>>>>>>>> epoch
> >>>>>>>>>>>>>>> bump,
> >>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> belongs to the
> >>>>>>> previous
> >>>>>>>>>>>>>>> transaction or
> >>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>> one.
> >>>>>>>>>>>>>>>>>>>>>>> We
> >>>>>>>>>>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition has been
> >>>>>>>> added
> >>>>>>>>> to
> >>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>> transaction.
> >>>>>>>>>>>>>>>>>>>>> Of
> >>>>>>>>>>>>>>>>>>>>>>>>> course,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> means
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't completely
> >>>>>>> cover
> >>>>>>>>> the
> >>>>>>>>>> case
> >>>>>>>>>>>>>>> where
> >>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>> have a
> >>>>>>>>>>>>>>>>>>>>>>>> really
> >>>>>>>>>>>>>>>>>>>>>>>>>>> late
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have added the
> >>>>>>>> partition
> >>>>>>>>> to
> >>>>>>>>>>>> the new
> >>>>>>>>>>>>>>>>>>>>> transaction,
> >>>>>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>>>> that's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something we will
> >>>>>>> need
> >>>>>>>>> the
> >>>>>>>>>> new
> >>>>>>>>>>>>>>> clients
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> cover.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is
> >>>>>>> ongoing
> >>>>>>>> =
> >>>>>>>>>>>> partition
> >>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>> added to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> via
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn.
> >>>>>>> We
> >>>>>>>>>> check
> >>>>>>>>>>>> this
> >>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this
> >>>>>>> wasn't
> >>>>>>>>>>>> sufficiently
> >>>>>>>>>>>>>>>>>>>> explained
> >>>>>>>>>>>>>>>>>>>>>>> here:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is
> >>>>>> that
> >>>>>>>> if
> >>>>>>>>>> any
> >>>>>>>>>>>>>>> messages
> >>>>>>>>>>>>>>>>>>>> somehow
> >>>>>>>>>>>>>>>>>>>>>>> come
> >>>>>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the
> >>>>>>> producer,
> >>>>>>>>> they
> >>>>>>>>>>>> will be
> >>>>>>>>>>>>>>>>>> fenced.
> >>>>>>>>>>>>>>>>>>>>>>> However,
> >>>>>>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it
> >>>>>> can
> >>>>>>> be
> >>>>>>>>>>>> discussed
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 50)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It should be
> >>>>>>>> synchronous
> >>>>>>>>>>>> because
> >>>>>>>>>>>>>>> if we
> >>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>>>> event
> >>>>>>>>>>>>>>>>>>>>>>>>>> (ie,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes us to need
> >>>>>> to
> >>>>>>>>> abort
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> transaction,
> >>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions to send
> >>>>>>>>>> transaction
> >>>>>>>>>>>>>>> markers
> >>>>>>>>>>>>>>>>> to.
> >>>>>>>>>>>>>>>>>>>> We
> >>>>>>>>>>>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we added them to
> >>>>>> the
> >>>>>>>>>>>> coordinator
> >>>>>>>>>>>>>>> via
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Previously we have
> >>>>>>> had
> >>>>>>>>>>>> asynchronous
> >>>>>>>>>>>>>>>>> calls
> >>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> past
> >>>>>>>>>>>>>>>>>>>>>>>>>> (ie,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit markers when
> >>>>>>> the
> >>>>>>>>>>>>>>> transaction is
> >>>>>>>>>>>>>>>>>>>>> completed)
> >>>>>>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>>>> often
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes confusion as
> >>>>>>> we
> >>>>>>>>>> need to
> >>>>>>>>>>>> wait
> >>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>>>>> operations
> >>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complete.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing commit
> >>>>>>> markers
> >>>>>>>>>> case,
> >>>>>>>>>>>>>>> clients
> >>>>>>>>>>>>>>>>> often
> >>>>>>>>>>>>>>>>>>>> see
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>> CONCURRENT_TRANSACTIONs
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error messages and
> >>>>>>> that
> >>>>>>>>>> can be
> >>>>>>>>>>>>>>>>> confusing.
> >>>>>>>>>>>>>>>>>>>> For
> >>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>> reason,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> may
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simpler to just
> >>>>>> have
> >>>>>>>>>>>> synchronous
> >>>>>>>>>>>>>>>> calls —
> >>>>>>>>>>>>>>>>>>>>>>> especially
> >>>>>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> block
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some operation's
> >>>>>>>>> completion
> >>>>>>>>>>>> anyway
> >>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>> start
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> next
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. And
> >>>>>>> yes, I
> >>>>>>>>>> meant
> >>>>>>>>>>>>>>>>>> coordinator. I
> >>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>> fix
> >>>>>>>>>>>>>>>>>>>>>>>>>>> that.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are
> >>>>>> checking
> >>>>>>> if
> >>>>>>>>> the
> >>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>> ongoing,
> >>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the
> >>>>>> leader
> >>>>>>>>>> partition
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for
> >>>>>>> this
> >>>>>>>>>>>> message to
> >>>>>>>>>>>>>>>> come
> >>>>>>>>>>>>>>>>>>>> back,
> >>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>> theory
> >>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
> >>>>>>>> that
> >>>>>>>>>> would
> >>>>>>>>>>>> make
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>>>>>>>>>>> result
> >>>>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why
> >>>>>> we
> >>>>>>>> can
> >>>>>>>>>> check
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> leader
> >>>>>>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm happy to update
> >>>>>>> the
> >>>>>>>>>> KIP if
> >>>>>>>>>>>>>>> some of
> >>>>>>>>>>>>>>>>>> these
> >>>>>>>>>>>>>>>>>>>>>>> things
> >>>>>>>>>>>>>>>>>>>>>>>>> were
> >>>>>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Nov 21,
> >>>>>> 2022
> >>>>>>> at
> >>>>>>>>>> 7:11 PM
> >>>>>>>>>>>>>>>> Matthias
> >>>>>>>>>>>>>>>>>> J.
> >>>>>>>>>>>>>>>>>>>>> Sax <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the
> >>>>>> KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Couple of
> >>>>>>>> clarification
> >>>>>>>>>>>> questions
> >>>>>>>>>>>>>>> (I
> >>>>>>>>>>>>>>>> am
> >>>>>>>>>>>>>>>>>>>> not a
> >>>>>>>>>>>>>>>>>>>>>>>> broker
> >>>>>>>>>>>>>>>>>>>>>>>>>>> expert
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> do
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some question are
> >>>>>>>>> obvious
> >>>>>>>>>> for
> >>>>>>>>>>>>>>> others,
> >>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>> me
> >>>>>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> my
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lack
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker knowledge).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (10)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What happens if
> >>>>>> the
> >>>>>>>>>> message
> >>>>>>>>>>>> come
> >>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> next
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request? It seems
> >>>>>>> the
> >>>>>>>>>> broker
> >>>>>>>>>>>>>>> hosting
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> data
> >>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anything about it
> >>>>>>> and
> >>>>>>>>>> append
> >>>>>>>>>>>> it to
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> partition,
> >>>>>>>>>>>>>>>>>>>>>>>>> too?
> >>>>>>>>>>>>>>>>>>>>>>>>>>> What
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> difference between
> >>>>>>>> both
> >>>>>>>>>> cases?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, it seems a
> >>>>>> TX
> >>>>>>>>> would
> >>>>>>>>>> only
> >>>>>>>>>>>>>>> hang,
> >>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>>>>>>>> is no
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> following
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> either committer
> >>>>>> or
> >>>>>>>>>> aborted?
> >>>>>>>>>>>> Thus,
> >>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>>>>>> above,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> TX
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> might
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually not hang
> >>>>>>> (of
> >>>>>>>>>> course,
> >>>>>>>>>>>> we
> >>>>>>>>>>>>>>>> might
> >>>>>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>>>> EOS
> >>>>>>>>>>>>>>>>>>>>>>>>>>> violation
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX was aborted and
> >>>>>>> the
> >>>>>>>>>> second
> >>>>>>>>>>>>>>>>> committed,
> >>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>>>>>>> way
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (20)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Of course, 1 and
> >>>>>> 2
> >>>>>>>>>> require
> >>>>>>>>>>>>>>>> client-side
> >>>>>>>>>>>>>>>>>>>>>>> changes, so
> >>>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> those approaches
> >>>>>>> won’t
> >>>>>>>>>> apply.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For (1) I
> >>>>>> understand
> >>>>>>>>> why a
> >>>>>>>>>>>> client
> >>>>>>>>>>>>>>>>> change
> >>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>> necessary,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we need a client
> >>>>>>>> change
> >>>>>>>>>> for
> >>>>>>>>>>>> (2).
> >>>>>>>>>>>>>>> Can
> >>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>> elaborate?
> >>>>>>>>>>>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Later
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> explain
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that we should
> >>>>>> send
> >>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>> DescribeTransactionRequest,
> >>>>>>>>>>>>>>>>>>>>>>>>> but I
> >>>>>>>>>>>>>>>>>>>>>>>>>>> am
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Can't we not just
> >>>>>> do
> >>>>>>>> an
> >>>>>>>>>>>> implicit
> >>>>>>>>>>>>>>>>>>>>>>> AddPartiitonToTx,
> >>>>>>>>>>>>>>>>>>>>>>>>> too?
> >>>>>>>>>>>>>>>>>>>>>>>>>>> If
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> old
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer correctly
> >>>>>>>>>> registered
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>>>>>>>> already,
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can just ignore it
> >>>>>>> as
> >>>>>>>>>> it's an
> >>>>>>>>>>>>>>>>> idempotent
> >>>>>>>>>>>>>>>>>>>>>>> operation?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (30)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To cover older
> >>>>>>>> clients,
> >>>>>>>>>> we
> >>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>> ensure a
> >>>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ongoing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we write to a
> >>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Not sure what you
> >>>>>>> mean
> >>>>>>>>> by
> >>>>>>>>>>>> this?
> >>>>>>>>>>>>>>> Can
> >>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>> elaborate?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (40)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [the
> >>>>>>> TX-coordinator]
> >>>>>>>>> will
> >>>>>>>>>>>> write
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> prepare
> >>>>>>>>>>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>>>>>>>>>> message
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch and send
> >>>>>>>>>>>>>>> WriteTxnMarkerRequests
> >>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> bumped
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why do we use the
> >>>>>>>> bumped
> >>>>>>>>>>>> epoch for
> >>>>>>>>>>>>>>>>> both?
> >>>>>>>>>>>>>>>>>> It
> >>>>>>>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> intuitive
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the current epoch,
> >>>>>>> and
> >>>>>>>>>> only
> >>>>>>>>>>>> return
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> bumped
> >>>>>>>>>>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (50) "Implicit
> >>>>>>>>>>>>>>>>> AddPartitionToTransaction"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why does the
> >>>>>>>> implicitly
> >>>>>>>>>> sent
> >>>>>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> synchronous?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also says
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in case we need
> >>>>>> to
> >>>>>>>>> abort
> >>>>>>>>>> and
> >>>>>>>>>>>>>>> need to
> >>>>>>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What do you mean
> >>>>>> by
> >>>>>>>>> this?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we don’t want to
> >>>>>>>> write
> >>>>>>>>>> to it
> >>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> manager
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Do you mean
> >>>>>>>>> TX-coordinator
> >>>>>>>>>>>>>>> instead of
> >>>>>>>>>>>>>>>>>>>>> "manager"?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (60)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For older clients
> >>>>>>> and
> >>>>>>>>>> ensuring
> >>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> TX
> >>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>> ongoing,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> describe a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> race condition. I
> >>>>>> am
> >>>>>>>> not
> >>>>>>>>>> sure
> >>>>>>>>>>>> if I
> >>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>> follow
> >>>>>>>>>>>>>>>>>>>>>>> here.
> >>>>>>>>>>>>>>>>>>>>>>>>> Can
> >>>>>>>>>>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> elaborate?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/18/22 1:21
> >>>>>> PM,
> >>>>>>>>>> Justine
> >>>>>>>>>>>>>>> Olshan
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey all!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to
> >>>>>> start a
> >>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>> on my
> >>>>>>>>>>>>>>>>>>>> proposal
> >>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>> add
> >>>>>>>>>>>>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checks on
> >>>>>>>> transactions
> >>>>>>>>> to
> >>>>>>>>>>>> avoid
> >>>>>>>>>>>>>>>>> hanging
> >>>>>>>>>>>>>>>>>>>>>>>>> transactions.
> >>>>>>>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an issue for some
> >>>>>>>> time,
> >>>>>>>>>> so I
> >>>>>>>>>>>>>>> really
> >>>>>>>>>>>>>>>>> hope
> >>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> helpful
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users of EOS.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The KIP includes
> >>>>>>>>> changes
> >>>>>>>>>> that
> >>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>> compatible
> >>>>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>>>> old
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> changes to
> >>>>>> improve
> >>>>>>>>>>>> performance
> >>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>> correctness
> >>>>>>>>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Please take a
> >>>>>> look
> >>>>>>>> and
> >>>>>>>>>> leave
> >>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>> comments
> >>>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>> may
> >>>>>>>>>>>>>>>>>>>>>>>>>> have!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> JIRA:
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-14402
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> >

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by "Matthias J. Sax" <mj...@apache.org>.
So the timestamp would be set when the write happens and thus no 
out-of-order data (base in time) can be introduced with "append_time" 
config even if a request sits in purgatory first while we check the TX 
status.

That does make sense. Thanks for confirming, that there is no 
out-of-order issue for this case.


-Matthias

On 1/25/23 5:04 PM, Justine Olshan wrote:
> Hey Matthias,
> Let me put it this way, if a producer is checking if a transaction is
> ongoing, then no writes to the partition from the producer will go through
> until the transaction is confirmed ongoing.
>  From then, I think I can apply the writes in the order they came in. Does
> that make sense?
> 
> Let me know if I'm missing something.
> Justine
> 
> On Wed, Jan 25, 2023 at 4:57 PM Matthias J. Sax <mj...@apache.org> wrote:
> 
>>> would it build an offset map with just the latest timestamp for a key?
>>
>> Cannot remember the details without reading the KIP, but yes, something
>> like this (I believe it actually needs to track both, offset and
>> timestamp per key).
>>
>>> I wonder if ordering assumptions are baked in there, why not use
>> offset-based compaction.
>>
>> The use case is a compacted topic that does contain out-of-order data.
>> If you set key k1=v1 @ 5 offset 100 and later key1 = v0 @ 3 at offset
>> 200 we want to cleanup v0 with higher offset because it's out-of-order
>> based on time, but keep v1 what is the actual latest version of k1.
>>
>>
>>> I was also not aware of this "guarantee" with regards to broker side
>> time.
>>
>> As already said: I am not sure if it's a public contract, but based on
>> my experience, people might reply on it as "implicit contract". -- Maybe
>> somebody else knows if it's public or not, and if it would be ok to
>> "break" it.
>>
>>> Let me know if you have any concerns here.
>>
>> My understanding is: While we cannot make an offset-order guarantee for
>> interleaved writes of different producer, if the topic is configures
>> with "append_time", we "guarantee" (cf. my comment above") timestamp
>> order... If that's the case, it would be an issue if we break this
>> "guarantee".
>>
>> I am not sure when the broker sets the timestamp for "append_time"
>> config? If we do it before putting the request into purgatory, we have a
>> problem. However, if we set the timestamp when we actually process the
>> request and do the actual append, it seems there is no issue, as the
>> request that was waiting in purgatory get the "newest" timestamp and
>> thus cannot introduce out-of-order data.
>>
>>
>> -Matthias
>>
>>
>> On 1/24/23 10:44 AM, Justine Olshan wrote:
>>> Hey Matthias,
>>>
>>> I have actually never heard of KIP-280 so thanks for bringing it up. That
>>> seems interesting. I wonder how it would work though -- would it build an
>>> offset map with just the latest timestamp for a key? I wonder if ordering
>>> assumptions are baked in there, why not use offset-based compaction.
>>>
>>> I was also not aware of this "guarantee" with regards to broker side
>> time.
>>> I think that we can do in order handling for a given producer, but not
>>> across all producers. However, we can't guarantee that anyway.
>>>
>>> Let me know if you have any concerns here.
>>>
>>> Thanks,
>>> Justine
>>>
>>> On Mon, Jan 23, 2023 at 6:33 PM Matthias J. Sax <mj...@apache.org>
>> wrote:
>>>
>>>> Just a side note about Guozhang comments about timestamps.
>>>>
>>>> If the producer sets the timestamp, putting the record into purgatory
>>>> seems not to be an issue (as already said: for this case we don't
>>>> guarantee timestamp order between writes of different producers anyway).
>>>> However, if the broker sets the timestamp, the expectation is that there
>>>> is no out-of-order data in the partition ever; if we would introduce
>>>> out-of-order data for this case (for interleaved writes of different
>>>> producers), it seems we would violate the current contract? (To be fair:
>>>> I don't know if that's an official contract, but I assume people rely on
>>>> this behavior -- and it "advertised" in many public talks...)
>>>>
>>>> About compaction: there is actually KIP-280 that adds timestamp based
>>>> compaction what is a very useful feature for Kafka Streams with regard
>>>> to out-of-order data handling. So the impact if we introduce
>>>> out-of-order data could be larger scoped.
>>>>
>>>>
>>>> -Matthias
>>>>
>>>>
>>>> On 1/20/23 4:48 PM, Justine Olshan wrote:
>>>>> Hey Artem,
>>>>>
>>>>> I see there is a check for transactional producers. I'm wondering if we
>>>>> don't handle the epoch overflow case. I'm also not sure it will be a
>> huge
>>>>> issue to extend to transactional producers, but maybe I'm missing
>>>> something.
>>>>>
>>>>> As for the recovery path -- I think Guozhang's point was if we have a
>> bad
>>>>> client that repeatedly tries to produce without adding to the
>> transaction
>>>>> we would do the following:
>>>>> a) if not fatal, we just fail the produce request over and over
>>>>> b) if fatal, we fence the producer
>>>>>
>>>>> Here with B, the issue with the client would be made clear more
>> quickly.
>>>> I
>>>>> suppose there are some intermediate cases where the issue only occurs
>>>>> sometimes, but I wonder if we should consider how to recover with
>> clients
>>>>> who don't behave as expected anyway.
>>>>>
>>>>> I think there is a place for the abortable error that we are adding --
>>>> just
>>>>> abort and try again. But I think there are also some cases where trying
>>>> to
>>>>> recover overcomplicates some logic. Especially if we are considering
>>>> older
>>>>> clients -- there I'm not sure if there's a ton we can do besides fail
>> the
>>>>> batch or fence the producer. With newer clients, we can consider more
>>>>> options for what can just be recovered after aborting. But epochs might
>>>> be
>>>>> a hard one unless we also want to reset producer ID.
>>>>>
>>>>> Thanks,
>>>>> Justine
>>>>>
>>>>>
>>>>>
>>>>> On Fri, Jan 20, 2023 at 3:59 PM Artem Livshits
>>>>> <al...@confluent.io.invalid> wrote:
>>>>>
>>>>>>>     besides the poorly written client case
>>>>>>
>>>>>> A poorly written client could create a lot of grief to people who run
>>>> Kafka
>>>>>> brokers :-), so when deciding to make an error fatal I would see if
>>>> there
>>>>>> is a reasonable recovery path rather than how often it could happen.
>>>> If we
>>>>>> have solid implementation of transactions (which I hope we'll do as a
>>>>>> result of this KIP), it would help to recover from a large class of
>>>> errors
>>>>>> by just aborting a transaction, even if the cause of error is a race
>>>>>> condition or etc.
>>>>>>
>>>>>> -Artem
>>>>>>
>>>>>> On Fri, Jan 20, 2023 at 3:26 PM Justine Olshan
>>>>>> <jo...@confluent.io.invalid>
>>>>>> wrote:
>>>>>>
>>>>>>> Artem --
>>>>>>> I guess the discussion path we were going down is when we expect to
>> see
>>>>>>> this error. I mentioned that it was hard to come up with cases for
>> when
>>>>>> the
>>>>>>> producer would still be around to receive the error besides the
>> poorly
>>>>>>> written client case.
>>>>>>> If we don't expect to have a producer to receive the response, it
>> sort
>>>> of
>>>>>>> makes sense for it to be fatal.
>>>>>>>
>>>>>>> I had some discussion with Jason offline about the epoch being off
>>>> cases
>>>>>>> and I'm not sure we could find a ton (outside of produce requests)
>>>> where
>>>>>> we
>>>>>>> could/should recover. I'd be happy to hear some examples though,
>> maybe
>>>>>> I'm
>>>>>>> missing something.
>>>>>>>
>>>>>>> Thanks,
>>>>>>> Justine
>>>>>>>
>>>>>>> On Fri, Jan 20, 2023 at 3:19 PM Artem Livshits
>>>>>>> <al...@confluent.io.invalid> wrote:
>>>>>>>
>>>>>>>> In general, I'd like to avoid fatal errors as much as possible, in
>>>> some
>>>>>>>> sense fatal errors just push out recovery logic to the application
>>>>>> which
>>>>>>>> either complicates the application or leads to disruption (we've
>> seen
>>>>>>> cases
>>>>>>>> when a transient broker error could lead to work stoppage when
>>>>>>> applications
>>>>>>>> need to be manually restarted).  I think we should strive to define
>>>>>>>> recovery logic for most errors (and/or encapsulate it in the Kafka
>>>>>> client
>>>>>>>> as much as possible).
>>>>>>>>
>>>>>>>> One benefit of transactions is that they simplify recovery from
>>>> errors,
>>>>>>>> pretty much any error (that's not handled transparently by retries
>> in
>>>>>>> Kafka
>>>>>>>> client) can be handled by the application via aborting the
>> transaction
>>>>>>> and
>>>>>>>> repeating the transactional logic again.  One tricky error is an
>> error
>>>>>>>> during commit, because we don't know the outcome.  For commit
>> errors,
>>>>>> the
>>>>>>>> recommendation should be to retry the commit until it returns the
>>>>>>> specific
>>>>>>>> result (committed or aborted).
>>>>>>>>
>>>>>>>> -Artem
>>>>>>>>
>>>>>>>> On Fri, Jan 20, 2023 at 2:52 PM Justine Olshan
>>>>>>>> <jo...@confluent.io.invalid>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> That's a fair point about other clients.
>>>>>>>>>
>>>>>>>>> I think the abortable error case is interesting because I'm curious
>>>>>> how
>>>>>>>>> other clients would handle this. I assume they would need to
>>>>>> implement
>>>>>>>>> handling for the error code unless they did something like "any
>>>>>> unknown
>>>>>>>>> error codes/any codes that aren't x,y,z are retriable." I would
>> hope
>>>>>>> that
>>>>>>>>> unknown error codes were fatal, and if the code was implemented it
>>>>>>> would
>>>>>>>>> abort the transaction. But I will think on this too.
>>>>>>>>>
>>>>>>>>> As for InvalidRecord -- you mentioned it was not fatal, but I'm
>>>>>> taking
>>>>>>> a
>>>>>>>>> look through the code. We would see this on handling the produce
>>>>>>>> response.
>>>>>>>>> If I recall correctly, we check if errors are retriable. I think
>> this
>>>>>>>> error
>>>>>>>>> would not be retriable. But I guess the concern here is that it is
>>>>>> not
>>>>>>>>> enough for just that batch to fail. I guess I hadn't considered
>> fully
>>>>>>>>> fencing the old producer but there are valid arguments here why we
>>>>>>> would
>>>>>>>>> want to.
>>>>>>>>>
>>>>>>>>> Thanks,
>>>>>>>>> Justine
>>>>>>>>>
>>>>>>>>> On Fri, Jan 20, 2023 at 2:35 PM Guozhang Wang <
>>>>>>>> guozhang.wang.us@gmail.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Thanks Justine for the replies! I agree with most of your
>> thoughts.
>>>>>>>>>>
>>>>>>>>>> Just for 3/7), though I agree for our own AK producer, since we do
>>>>>>>>>> "nextRequest(boolean hasIncompleteBatches)", we guarantee the
>>>>>> end-txn
>>>>>>>>>> would not be sent until we've effectively flushed, but I was
>>>>>>> referring
>>>>>>>>>> to any future bugs or other buggy clients that the same client may
>>>>>>> get
>>>>>>>>>> into this situation, in which case we should give the client a
>>>>>> clear
>>>>>>>>>> msg that "you did something wrong, and hence now you should
>> fatally
>>>>>>>>>> close yourself". What I'm concerned about is that, by seeing an
>>>>>>>>>> "abortable error" or in some rare cases an "invalid record", the
>>>>>>>>>> client could not realize "something that's really bad happened".
>> So
>>>>>>>>>> it's not about adding a new error, it's mainly about those real
>>>>>> buggy
>>>>>>>>>> situations causing such "should never happen" cases, the errors
>>>>>>> return
>>>>>>>>>> would not be informative enough.
>>>>>>>>>>
>>>>>>>>>> Thinking in other ways, if we believe that for most cases such
>>>>>> error
>>>>>>>>>> codes would not reach the original clients since they would be
>>>>>>>>>> disconnected or even gone by that time, and only in some rare
>> cases
>>>>>>>>>> they would still be seen by the sending clients, then why not make
>>>>>>>>>> them more fatal and more specific than generic.
>>>>>>>>>>
>>>>>>>>>> Guozhang
>>>>>>>>>>
>>>>>>>>>> On Fri, Jan 20, 2023 at 1:59 PM Justine Olshan
>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
>>>>>>>>>>>
>>>>>>>>>>> Hey Guozhang. Thanks for taking a look and for the detailed
>>>>>>> comments!
>>>>>>>>>> I'll
>>>>>>>>>>> do my best to address below.
>>>>>>>>>>>
>>>>>>>>>>> 1. I see what you are saying here, but I think I need to look
>>>>>>> through
>>>>>>>>> the
>>>>>>>>>>> sequence of events you mention. Typically we've seen this issue
>>>>>> in
>>>>>>> a
>>>>>>>>> few
>>>>>>>>>>> cases.
>>>>>>>>>>>
>>>>>>>>>>>     One is when we have a producer disconnect when trying to
>>>>>> produce.
>>>>>>>>>>> Typically in these cases, we abort the transaction. We've seen
>>>>>> that
>>>>>>>>> after
>>>>>>>>>>> the markers are written, the disconnection can sometimes cause
>>>>>> the
>>>>>>>>>> request
>>>>>>>>>>> to get flushed to the broker. In this case, we don't need client
>>>>>>>>> handling
>>>>>>>>>>> because the producer we are responding to is gone. We just needed
>>>>>>> to
>>>>>>>>> make
>>>>>>>>>>> sure we didn't write to the log on the broker side. I'm trying to
>>>>>>>> think
>>>>>>>>>> of
>>>>>>>>>>> a case where we do have the client to return to. I'd think the
>>>>>> same
>>>>>>>>>> client
>>>>>>>>>>> couldn't progress to committing the transaction unless the
>>>>>> produce
>>>>>>>>>> request
>>>>>>>>>>> returned right? Of course, there is the incorrectly written
>>>>>> clients
>>>>>>>>> case.
>>>>>>>>>>> I'll think on this a bit more and let you know if I come up with
>>>>>>>>> another
>>>>>>>>>>> scenario when we would return to an active client when the
>>>>>>>> transaction
>>>>>>>>> is
>>>>>>>>>>> no longer ongoing.
>>>>>>>>>>>
>>>>>>>>>>> I was not aware that we checked the result of a send after we
>>>>>>> commit
>>>>>>>>>>> though. I'll need to look into that a bit more.
>>>>>>>>>>>
>>>>>>>>>>> 2. There were some questions about this in the discussion. The
>>>>>> plan
>>>>>>>> is
>>>>>>>>> to
>>>>>>>>>>> handle overflow with the mechanism we currently have in the
>>>>>>> producer.
>>>>>>>>> If
>>>>>>>>>> we
>>>>>>>>>>> try to bump and the epoch will overflow, we actually allocate a
>>>>>> new
>>>>>>>>>>> producer ID. I need to confirm the fencing logic on the last
>>>>>> epoch
>>>>>>>> (ie,
>>>>>>>>>> we
>>>>>>>>>>> probably shouldn't allow any records to be produced with the
>>>>>> final
>>>>>>>>> epoch
>>>>>>>>>>> since we can never properly fence that one).
>>>>>>>>>>>
>>>>>>>>>>> 3. I can agree with you that the current error handling is
>>>>>> messy. I
>>>>>>>>>> recall
>>>>>>>>>>> taking a look at your KIP a while back, but I think I mostly saw
>>>>>>> the
>>>>>>>>>>> section about how the errors were wrapped. Maybe I need to take
>>>>>>>> another
>>>>>>>>>>> look. As for abortable error, the idea was that the handling
>>>>>> would
>>>>>>> be
>>>>>>>>>>> simple -- if this error is seen, the transaction should be
>>>>>> aborted
>>>>>>> --
>>>>>>>>> no
>>>>>>>>>>> other logic about previous state or requests necessary. Is your
>>>>>>>> concern
>>>>>>>>>>> simply about adding new errors? We were hoping to have an error
>>>>>>> that
>>>>>>>>>> would
>>>>>>>>>>> have one meaning and many of the current errors have a history of
>>>>>>>>> meaning
>>>>>>>>>>> different things on different client versions. That was the main
>>>>>>>>>> motivation
>>>>>>>>>>> for adding a new error.
>>>>>>>>>>>
>>>>>>>>>>> 4. This is a good point about record timestamp reordering.
>>>>>>> Timestamps
>>>>>>>>>> don't
>>>>>>>>>>> affect compaction, but they do affect retention deletion. For
>>>>>> that,
>>>>>>>>> kafka
>>>>>>>>>>> considers the largest timestamp in the segment, so I think a
>>>>>> small
>>>>>>>>> amount
>>>>>>>>>>> of reordering (hopefully on the order of milliseconds or even
>>>>>>>> seconds)
>>>>>>>>>> will
>>>>>>>>>>> be ok. We take timestamps from clients so there is already a
>>>>>>>>> possibility
>>>>>>>>>>> for some drift and non-monotonically increasing timestamps.
>>>>>>>>>>>
>>>>>>>>>>> 5. Thanks for catching. The error is there, but it's actually
>>>>>> that
>>>>>>>>> those
>>>>>>>>>>> fields should be 4+! Due to how the message generator works, I
>>>>>>>> actually
>>>>>>>>>>> have to redefine those fields inside the
>>>>>>>>> `"AddPartitionsToTxnTransaction`
>>>>>>>>>>> block for it to build correctly. I'll fix it to be correct.
>>>>>>>>>>>
>>>>>>>>>>> 6. Correct -- we will only add the request to purgatory if the
>>>>>>> cache
>>>>>>>>> has
>>>>>>>>>> no
>>>>>>>>>>> ongoing transaction. I can change the wording to make that
>>>>>> clearer
>>>>>>>> that
>>>>>>>>>> we
>>>>>>>>>>> only place the request in purgatory if we need to contact the
>>>>>>>>> transaction
>>>>>>>>>>> coordinator.
>>>>>>>>>>>
>>>>>>>>>>> 7. We did take a look at some of the errors and it was hard to
>>>>>> come
>>>>>>>> up
>>>>>>>>>> with
>>>>>>>>>>> a good one. I agree that InvalidTxnStateException is ideal except
>>>>>>> for
>>>>>>>>> the
>>>>>>>>>>> fact that it hasn't been returned on Produce requests before. The
>>>>>>>> error
>>>>>>>>>>> handling for clients is a bit vague (which is why I opened
>>>>>>>> KAFKA-14439
>>>>>>>>>>> <https://issues.apache.org/jira/browse/KAFKA-14439>), but the
>>>>>>>> decision
>>>>>>>>>> we
>>>>>>>>>>> made here was to only return errors that have been previously
>>>>>>>> returned
>>>>>>>>> to
>>>>>>>>>>> producers. As for not being fatal, I think part of the theory was
>>>>>>>> that
>>>>>>>>> in
>>>>>>>>>>> many cases, the producer would be disconnected. (See point 1) and
>>>>>>>> this
>>>>>>>>>>> would just be an error to return from the server. I did plan to
>>>>>>> think
>>>>>>>>>> about
>>>>>>>>>>> other cases, so let me know if you think of any as well!
>>>>>>>>>>>
>>>>>>>>>>> Lots to say! Let me know if you have further thoughts!
>>>>>>>>>>> Justine
>>>>>>>>>>>
>>>>>>>>>>> On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang <
>>>>>>>>>> guozhang.wang.us@gmail.com>
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Hello Justine,
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks for the great write-up! I made a quick pass through it
>>>>>> and
>>>>>>>>> here
>>>>>>>>>>>> are some thoughts (I have not been able to read through this
>>>>>>> thread
>>>>>>>>> so
>>>>>>>>>>>> pardon me if they have overlapped or subsumed by previous
>>>>>>>> comments):
>>>>>>>>>>>>
>>>>>>>>>>>> First are some meta ones:
>>>>>>>>>>>>
>>>>>>>>>>>> 1. I think we need to also improve the client's experience once
>>>>>>> we
>>>>>>>>>>>> have this defence in place. More concretely, say a user's
>>>>>>> producer
>>>>>>>>>>>> code is like following:
>>>>>>>>>>>>
>>>>>>>>>>>> future = producer.send();
>>>>>>>>>>>> // producer.flush();
>>>>>>>>>>>> producer.commitTransaction();
>>>>>>>>>>>> future.get();
>>>>>>>>>>>>
>>>>>>>>>>>> Which resulted in the order of a) produce-request sent by
>>>>>>> producer,
>>>>>>>>> b)
>>>>>>>>>>>> end-txn-request sent by producer, c) end-txn-response sent
>>>>>> back,
>>>>>>> d)
>>>>>>>>>>>> txn-marker-request sent from coordinator to partition leader,
>>>>>> e)
>>>>>>>>>>>> produce-request finally received by the partition leader,
>>>>>> before
>>>>>>>> this
>>>>>>>>>>>> KIP e) step would be accepted causing a dangling txn; now it
>>>>>>> would
>>>>>>>> be
>>>>>>>>>>>> rejected in step e) which is good. But from the client's point
>>>>>> of
>>>>>>>>> view
>>>>>>>>>>>> now it becomes confusing since the `commitTransaction()`
>>>>>> returns
>>>>>>>>>>>> successfully, but the "future" throws an invalid-epoch error,
>>>>>> and
>>>>>>>>> they
>>>>>>>>>>>> are not sure if the transaction did succeed or not. In fact, it
>>>>>>>>>>>> "partially succeeded" with some msgs being rejected but others
>>>>>>>>>>>> committed successfully.
>>>>>>>>>>>>
>>>>>>>>>>>> Of course the easy way to avoid this is, always call
>>>>>>>>>>>> "producer.flush()" before commitTxn and that's what we do
>>>>>>>> ourselves,
>>>>>>>>>>>> and what we recommend users do. But I suspect not everyone does
>>>>>>> it.
>>>>>>>>> In
>>>>>>>>>>>> fact I just checked the javadoc in KafkaProducer and our code
>>>>>>>> snippet
>>>>>>>>>>>> does not include a `flush()` call. So I'm thinking maybe we can
>>>>>>> in
>>>>>>>>>>>> side the `commitTxn` code to enforce flushing before sending
>>>>>> the
>>>>>>>>>>>> end-txn request.
>>>>>>>>>>>>
>>>>>>>>>>>> 2. I'd like to clarify a bit details on "just add partitions to
>>>>>>> the
>>>>>>>>>>>> transaction on the first produce request during a transaction".
>>>>>>> My
>>>>>>>>>>>> understanding is that the partition leader's cache has the
>>>>>>> producer
>>>>>>>>> id
>>>>>>>>>>>> / sequence / epoch for the latest txn, either on-going or is
>>>>>>>>> completed
>>>>>>>>>>>> (upon receiving the marker request from coordinator). When a
>>>>>>>> produce
>>>>>>>>>>>> request is received, if
>>>>>>>>>>>>
>>>>>>>>>>>> * producer's epoch < cached epoch, or producer's epoch ==
>>>>>> cached
>>>>>>>>> epoch
>>>>>>>>>>>> but the latest txn is completed, leader directly reject with
>>>>>>>>>>>> invalid-epoch.
>>>>>>>>>>>> * producer's epoch > cached epoch, park the the request and
>>>>>> send
>>>>>>>>>>>> add-partitions request to coordinator.
>>>>>>>>>>>>
>>>>>>>>>>>> In order to do it, does the coordinator need to bump the
>>>>>> sequence
>>>>>>>> and
>>>>>>>>>>>> reset epoch to 0 when the next epoch is going to overflow? If
>>>>>> no
>>>>>>>> need
>>>>>>>>>>>> to do so, then how we handle the (admittedly rare, but still
>>>>>> may
>>>>>>>>>>>> happen) epoch overflow situation?
>>>>>>>>>>>>
>>>>>>>>>>>> 3. I'm a bit concerned about adding a generic "ABORTABLE_ERROR"
>>>>>>>> given
>>>>>>>>>>>> we already have a pretty messy error classification and error
>>>>>>>>> handling
>>>>>>>>>>>> on the producer clients side --- I have a summary about the
>>>>>>> issues
>>>>>>>>> and
>>>>>>>>>>>> a proposal to address this in
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
>>>>>>>>>>>> -- I understand we do not want to use "UNKNOWN_PRODUCER_ID"
>>>>>>> anymore
>>>>>>>>>>>> and in fact we intend to deprecate it in KIP-360 and eventually
>>>>>>>>> remove
>>>>>>>>>>>> it; but I'm wondering can we still use specific error codes.
>>>>>> E.g.
>>>>>>>>> what
>>>>>>>>>>>> about "InvalidProducerEpochException" since for new clients,
>>>>>> the
>>>>>>>>>>>> actual reason this would actually be rejected is indeed because
>>>>>>> the
>>>>>>>>>>>> epoch on the coordinator caused the add-partitions-request from
>>>>>>> the
>>>>>>>>>>>> brokers to be rejected anyways?
>>>>>>>>>>>>
>>>>>>>>>>>> 4. It seems we put the producer request into purgatory before
>>>>>> we
>>>>>>>> ever
>>>>>>>>>>>> append the records, while other producer's records may still be
>>>>>>>>>>>> appended during the time; and that potentially may result in
>>>>>> some
>>>>>>>>>>>> re-ordering compared with reception order. I'm not super
>>>>>>> concerned
>>>>>>>>>>>> about it since Kafka does not guarantee reception ordering
>>>>>> across
>>>>>>>>>>>> producers anyways, but it may make the timestamps of records
>>>>>>>> inside a
>>>>>>>>>>>> partition to be more out-of-ordered. Are we aware of any
>>>>>>> scenarios
>>>>>>>>>>>> such as future enhancements on log compactions that may be
>>>>>>> affected
>>>>>>>>> by
>>>>>>>>>>>> this effect?
>>>>>>>>>>>>
>>>>>>>>>>>> Below are just minor comments:
>>>>>>>>>>>>
>>>>>>>>>>>> 5. In "AddPartitionsToTxnTransaction" field of
>>>>>>>>>>>> "AddPartitionsToTxnRequest" RPC, the versions of those inner
>>>>>>> fields
>>>>>>>>>>>> are "0-3" while I thought they should be "0+" still?
>>>>>>>>>>>>
>>>>>>>>>>>> 6. Regarding "we can place the request in a purgatory of sorts
>>>>>>> and
>>>>>>>>>>>> check if there is any state for the transaction on the
>>>>>> broker": i
>>>>>>>>>>>> think at this time when we just do the checks against the
>>>>>> cached
>>>>>>>>>>>> state, we do not need to put the request to purgatory yet?
>>>>>>>>>>>>
>>>>>>>>>>>> 7. This is related to 3) above. I feel using
>>>>>>>> "InvalidRecordException"
>>>>>>>>>>>> for older clients may also be a bit confusing, and also it is
>>>>>> not
>>>>>>>>>>>> fatal -- for old clients, it better to be fatal since this
>>>>>>>> indicates
>>>>>>>>>>>> the clients is doing something wrong and hence it should be
>>>>>>> closed.
>>>>>>>>>>>> And in general I'd prefer to use slightly more specific meaning
>>>>>>>> error
>>>>>>>>>>>> codes for clients. That being said, I also feel
>>>>>>>>>>>> "InvalidProducerEpochException" is not suitable for old
>>>>>> versioned
>>>>>>>>>>>> clients, and we'd have to pick one that old clients recognize.
>>>>>>> I'd
>>>>>>>>>>>> prefer "InvalidTxnStateException" but that one is supposed to
>>>>>> be
>>>>>>>>>>>> returned from txn coordinators only today. I'd suggest we do a
>>>>>>>> quick
>>>>>>>>>>>> check in the current client's code path and see if that one
>>>>>> would
>>>>>>>> be
>>>>>>>>>>>> handled if it's from a produce-response, and if yes, use this
>>>>>>> one;
>>>>>>>>>>>> otherwise, use "ProducerFencedException" which is much less
>>>>>>>>> meaningful
>>>>>>>>>>>> but it's still a fatal error.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks,
>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
>>>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>> Yeah -- looks like we already have code to handle bumping the
>>>>>>>> epoch
>>>>>>>>>> and
>>>>>>>>>>>>> when the epoch is Short.MAX_VALUE, we get a new producer ID.
>>>>>>>> Since
>>>>>>>>>> this
>>>>>>>>>>>> is
>>>>>>>>>>>>> already the behavior, do we want to change it further?
>>>>>>>>>>>>>
>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <
>>>>>>>>> jolshan@confluent.io
>>>>>>>>>>>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hey all, just wanted to quickly update and say I've
>>>>>> modified
>>>>>>>> the
>>>>>>>>>> KIP to
>>>>>>>>>>>>>> explicitly mention that AddOffsetCommitsToTxnRequest will
>>>>>> be
>>>>>>>>>> replaced
>>>>>>>>>>>> by
>>>>>>>>>>>>>> a coordinator-side (inter-broker) AddPartitionsToTxn
>>>>>> implicit
>>>>>>>>>> request.
>>>>>>>>>>>> This
>>>>>>>>>>>>>> mirrors the user partitions and will implicitly add offset
>>>>>>>>>> partitions
>>>>>>>>>>>> to
>>>>>>>>>>>>>> transactions when we commit offsets on them. We will
>>>>>>> deprecate
>>>>>>>>>>>> AddOffsetCommitsToTxnRequest
>>>>>>>>>>>>>> for new clients.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Also to address Artem's comments --
>>>>>>>>>>>>>> I'm a bit unsure if the changes here will change the
>>>>>> previous
>>>>>>>>>> behavior
>>>>>>>>>>>> for
>>>>>>>>>>>>>> fencing producers. In the case you mention in the first
>>>>>>>>> paragraph,
>>>>>>>>>> are
>>>>>>>>>>>> you
>>>>>>>>>>>>>> saying we bump the epoch before we try to abort the
>>>>>>>> transaction?
>>>>>>>>> I
>>>>>>>>>>>> think I
>>>>>>>>>>>>>> need to understand the scenarios you mention a bit better.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> As for the second part -- I think it makes sense to have
>>>>>> some
>>>>>>>>> sort
>>>>>>>>>> of
>>>>>>>>>>>>>> "sentinel" epoch to signal epoch is about to overflow (I
>>>>>>> think
>>>>>>>> we
>>>>>>>>>> sort
>>>>>>>>>>>> of
>>>>>>>>>>>>>> have this value in place in some ways) so we can codify it
>>>>>> in
>>>>>>>> the
>>>>>>>>>> KIP.
>>>>>>>>>>>> I'll
>>>>>>>>>>>>>> look into that and try to update soon.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>> Justine.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
>>>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> It's good to know that KIP-588 addressed some of the
>>>>>> issues.
>>>>>>>>>> Looking
>>>>>>>>>>>> at
>>>>>>>>>>>>>>> the code, it still looks like there are some cases that
>>>>>>> would
>>>>>>>>>> result
>>>>>>>>>>>> in
>>>>>>>>>>>>>>> fatal error, e.g. PRODUCER_FENCED is issued by the
>>>>>>> transaction
>>>>>>>>>>>> coordinator
>>>>>>>>>>>>>>> if epoch doesn't match, and the client treats it as a
>>>>>> fatal
>>>>>>>>> error
>>>>>>>>>>>> (code in
>>>>>>>>>>>>>>> TransactionManager request handling).  If we consider, for
>>>>>>>>>> example,
>>>>>>>>>>>>>>> committing a transaction that returns a timeout, but
>>>>>>> actually
>>>>>>>>>>>> succeeds,
>>>>>>>>>>>>>>> trying to abort it or re-commit may result in
>>>>>>> PRODUCER_FENCED
>>>>>>>>>> error
>>>>>>>>>>>>>>> (because of epoch bump).
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> For failed commits, specifically, we need to know the
>>>>>> actual
>>>>>>>>>> outcome,
>>>>>>>>>>>>>>> because if we return an error the application may think
>>>>>> that
>>>>>>>> the
>>>>>>>>>>>>>>> transaction is aborted and redo the work, leading to
>>>>>>>> duplicates.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Re: overflowing epoch.  We could either do it on the TC
>>>>>> and
>>>>>>>>> return
>>>>>>>>>>>> both
>>>>>>>>>>>>>>> producer id and epoch (e.g. change the protocol), or
>>>>>> signal
>>>>>>>> the
>>>>>>>>>> client
>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>> it needs to get a new producer id.  Checking for max epoch
>>>>>>>> could
>>>>>>>>>> be a
>>>>>>>>>>>>>>> reasonable signal, the value to check should probably be
>>>>>>>> present
>>>>>>>>>> in
>>>>>>>>>>>> the
>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>> as this is effectively a part of the contract.  Also, the
>>>>>> TC
>>>>>>>>>> should
>>>>>>>>>>>>>>> probably return an error if the client didn't change
>>>>>>> producer
>>>>>>>> id
>>>>>>>>>> after
>>>>>>>>>>>>>>> hitting max epoch.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> -Artem
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
>>>>>>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks for the discussion Artem.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> With respect to the handling of fenced producers, we
>>>>>> have
>>>>>>>> some
>>>>>>>>>>>> behavior
>>>>>>>>>>>>>>>> already in place. As of KIP-588:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
>>>>>>>>>>>>>>>> ,
>>>>>>>>>>>>>>>> we handle timeouts more gracefully. The producer can
>>>>>>>> recover.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Produce requests can also recover from epoch fencing by
>>>>>>>>>> aborting the
>>>>>>>>>>>>>>>> transaction and starting over.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> What other cases were you considering that would cause
>>>>>> us
>>>>>>> to
>>>>>>>>>> have a
>>>>>>>>>>>>>>> fenced
>>>>>>>>>>>>>>>> epoch but we'd want to recover?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> The first point about handling epoch overflows is fair.
>>>>>> I
>>>>>>>>> think
>>>>>>>>>>>> there is
>>>>>>>>>>>>>>>> some logic we'd need to consider. (ie, if we are one
>>>>>> away
>>>>>>>> from
>>>>>>>>>> the
>>>>>>>>>>>> max
>>>>>>>>>>>>>>>> epoch, we need to reset the producer ID.) I'm still
>>>>>>>> wondering
>>>>>>>>> if
>>>>>>>>>>>> there
>>>>>>>>>>>>>>> is a
>>>>>>>>>>>>>>>> way to direct this from the response, or if everything
>>>>>>>> should
>>>>>>>>> be
>>>>>>>>>>>> done on
>>>>>>>>>>>>>>>> the client side. Let me know if you have any thoughts
>>>>>>> here.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
>>>>>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> There are some workflows in the client that are
>>>>>> implied
>>>>>>> by
>>>>>>>>>>>> protocol
>>>>>>>>>>>>>>>>> changes, e.g.:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> - for new clients, epoch changes with every
>>>>>> transaction
>>>>>>>> and
>>>>>>>>>> can
>>>>>>>>>>>>>>> overflow,
>>>>>>>>>>>>>>>>> in old clients this condition was handled
>>>>>> transparently,
>>>>>>>>>> because
>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>> bumped in InitProducerId and it would return a new
>>>>>>>> producer
>>>>>>>>>> id if
>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>> overflows, the new clients would need to implement
>>>>>> some
>>>>>>>>>> workflow
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> refresh
>>>>>>>>>>>>>>>>> producer id
>>>>>>>>>>>>>>>>> - how to handle fenced producers, for new clients
>>>>>> epoch
>>>>>>>>>> changes
>>>>>>>>>>>> with
>>>>>>>>>>>>>>>> every
>>>>>>>>>>>>>>>>> transaction, so in presence of failures during
>>>>>> commits /
>>>>>>>>>> aborts,
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> producer could get easily fenced, old clients would
>>>>>>> pretty
>>>>>>>>>> much
>>>>>>>>>>>> would
>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>> fenced when a new incarnation of the producer was
>>>>>>>>> initialized
>>>>>>>>>> with
>>>>>>>>>>>>>>>>> InitProducerId so it's ok to treat as a fatal error,
>>>>>> the
>>>>>>>> new
>>>>>>>>>>>> clients
>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>> need to implement some workflow to handle that error,
>>>>>>>>>> otherwise
>>>>>>>>>>>> they
>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>> get fenced by themselves
>>>>>>>>>>>>>>>>> - in particular (as a subset of the previous issue),
>>>>>>> what
>>>>>>>>>> would
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> client
>>>>>>>>>>>>>>>>> do if it got a timeout during commit?  commit could've
>>>>>>>>>> succeeded
>>>>>>>>>>>> or
>>>>>>>>>>>>>>>> failed
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Not sure if this has to be defined in the KIP as
>>>>>>>>> implementing
>>>>>>>>>>>> those
>>>>>>>>>>>>>>>>> probably wouldn't require protocol changes, but we
>>>>>> have
>>>>>>>>>> multiple
>>>>>>>>>>>>>>>>> implementations of Kafka clients, so probably would be
>>>>>>>> good
>>>>>>>>> to
>>>>>>>>>>>> have
>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>> client implementation guidance.  Could also be done
>>>>>> as a
>>>>>>>>>> separate
>>>>>>>>>>>> doc.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> -Artem
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
>>>>>>>>>>>>>>>> <jolshan@confluent.io.invalid
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hey all, I've updated the KIP to incorporate Jason's
>>>>>>>>>>>> suggestions.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 1. Use AddPartitionsToTxn + verify flag to check on
>>>>>>> old
>>>>>>>>>> clients
>>>>>>>>>>>>>>>>>> 2. Updated AddPartitionsToTxn API to support
>>>>>>> transaction
>>>>>>>>>>>> batching
>>>>>>>>>>>>>>>>>> 3. Mention IBP bump
>>>>>>>>>>>>>>>>>> 4. Mention auth change on new AddPartitionsToTxn
>>>>>>>> version.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I'm planning on opening a vote soon.
>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <
>>>>>>>>>>>> jolshan@confluent.io
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks Jason. Those changes make sense to me. I
>>>>>> will
>>>>>>>>>> update
>>>>>>>>>>>> the
>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
>>>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Hey Justine,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I was wondering about compatibility here. When
>>>>>> we
>>>>>>>>> send
>>>>>>>>>>>> requests
>>>>>>>>>>>>>>>>>>>> between brokers, we want to ensure that the
>>>>>>> receiving
>>>>>>>>>> broker
>>>>>>>>>>>>>>>>> understands
>>>>>>>>>>>>>>>>>>>> the request (specifically the new fields).
>>>>>>> Typically
>>>>>>>>>> this is
>>>>>>>>>>>> done
>>>>>>>>>>>>>>>> via
>>>>>>>>>>>>>>>>>>>> IBP/metadata version.
>>>>>>>>>>>>>>>>>>>> I'm trying to think if there is a way around it
>>>>>> but
>>>>>>>> I'm
>>>>>>>>>> not
>>>>>>>>>>>> sure
>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>> is.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Yes. I think we would gate usage of this behind
>>>>>> an
>>>>>>>> IBP
>>>>>>>>>> bump.
>>>>>>>>>>>> Does
>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>> seem
>>>>>>>>>>>>>>>>>>>> reasonable?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> As for the improvements -- can you clarify how
>>>>>>> the
>>>>>>>>>> multiple
>>>>>>>>>>>>>>>>>>>> transactional
>>>>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking of a case
>>>>>>>> where
>>>>>>>>> we
>>>>>>>>>>>>>>> wait/batch
>>>>>>>>>>>>>>>>>>>> multiple produce requests together? My
>>>>>>> understanding
>>>>>>>>> for
>>>>>>>>>> now
>>>>>>>>>>>> was
>>>>>>>>>>>>>>> 1
>>>>>>>>>>>>>>>>>>>> transactional ID and one validation per 1 produce
>>>>>>>>>> request.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Each call to `AddPartitionsToTxn` is essentially
>>>>>> a
>>>>>>>>> write
>>>>>>>>>> to
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>> log and must block on replication. The more we
>>>>>> can
>>>>>>>> fit
>>>>>>>>>> into a
>>>>>>>>>>>>>>> single
>>>>>>>>>>>>>>>>>>>> request, the more writes we can do in parallel.
>>>>>> The
>>>>>>>>>>>> alternative
>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>> use of more connections, but usually we prefer
>>>>>>>> batching
>>>>>>>>>>>> since the
>>>>>>>>>>>>>>>>>> network
>>>>>>>>>>>>>>>>>>>> stack is not really optimized for high
>>>>>>>>> connection/request
>>>>>>>>>>>> loads.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Finally with respect to the authorizations, I
>>>>>>> think
>>>>>>>>> it
>>>>>>>>>>>> makes
>>>>>>>>>>>>>>> sense
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> skip
>>>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit confused by
>>>>>> the
>>>>>>>>>> "leader
>>>>>>>>>>>> ID"
>>>>>>>>>>>>>>>> field.
>>>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the request as
>>>>>> from a
>>>>>>>>>> broker
>>>>>>>>>>>> (does
>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>> matter
>>>>>>>>>>>>>>>>>>>> which one?).
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> We could also make it version-based. For the next
>>>>>>>>>> version, we
>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>>> CLUSTER auth. So clients would not be able to use
>>>>>>> the
>>>>>>>>> API
>>>>>>>>>>>>>>> anymore,
>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>> probably what we want.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> -Jason
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> As a follow up, I was just thinking about the
>>>>>>>>> batching
>>>>>>>>>> a
>>>>>>>>>>>> bit
>>>>>>>>>>>>>>> more.
>>>>>>>>>>>>>>>>>>>>> I suppose if we have one request in flight and
>>>>>> we
>>>>>>>>>> queue up
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>> produce requests in some sort of purgatory, we
>>>>>>>> could
>>>>>>>>>> send
>>>>>>>>>>>>>>>>> information
>>>>>>>>>>>>>>>>>>>> out
>>>>>>>>>>>>>>>>>>>>> for all of them rather than one by one. So that
>>>>>>>> would
>>>>>>>>>> be a
>>>>>>>>>>>>>>> benefit
>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>> batching partitions to add per transaction.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I'll need to think a bit more on the design of
>>>>>>> this
>>>>>>>>>> part
>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>> KIP,
>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> will update the KIP in the next few days.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan
>>>>>> <
>>>>>>>>>>>>>>>>> jolshan@confluent.io>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Hey Jason -- thanks for the input -- I was
>>>>>> just
>>>>>>>>>> digging
>>>>>>>>>>>> a bit
>>>>>>>>>>>>>>>>> deeper
>>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>> the design + implementation of the validation
>>>>>>>> calls
>>>>>>>>>> here
>>>>>>>>>>>> and
>>>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>> say
>>>>>>>>>>>>>>>>>>>>>> makes sense.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I was wondering about compatibility here.
>>>>>> When
>>>>>>> we
>>>>>>>>>> send
>>>>>>>>>>>>>>> requests
>>>>>>>>>>>>>>>>>>>>>> between brokers, we want to ensure that the
>>>>>>>>> receiving
>>>>>>>>>>>> broker
>>>>>>>>>>>>>>>>>>>> understands
>>>>>>>>>>>>>>>>>>>>>> the request (specifically the new fields).
>>>>>>>>> Typically
>>>>>>>>>>>> this is
>>>>>>>>>>>>>>>> done
>>>>>>>>>>>>>>>>>> via
>>>>>>>>>>>>>>>>>>>>>> IBP/metadata version.
>>>>>>>>>>>>>>>>>>>>>> I'm trying to think if there is a way around
>>>>>> it
>>>>>>>> but
>>>>>>>>>> I'm
>>>>>>>>>>>> not
>>>>>>>>>>>>>>> sure
>>>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>>>> is.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> As for the improvements -- can you clarify
>>>>>> how
>>>>>>>> the
>>>>>>>>>>>> multiple
>>>>>>>>>>>>>>>>>>>> transactional
>>>>>>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking of a
>>>>>>> case
>>>>>>>>>> where we
>>>>>>>>>>>>>>>>> wait/batch
>>>>>>>>>>>>>>>>>>>>>> multiple produce requests together? My
>>>>>>>>> understanding
>>>>>>>>>> for
>>>>>>>>>>>> now
>>>>>>>>>>>>>>>> was 1
>>>>>>>>>>>>>>>>>>>>>> transactional ID and one validation per 1
>>>>>>> produce
>>>>>>>>>>>> request.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Finally with respect to the authorizations, I
>>>>>>>> think
>>>>>>>>>> it
>>>>>>>>>>>> makes
>>>>>>>>>>>>>>>> sense
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> skip
>>>>>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit confused
>>>>>> by
>>>>>>>> the
>>>>>>>>>>>> "leader
>>>>>>>>>>>>>>> ID"
>>>>>>>>>>>>>>>>>> field.
>>>>>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the request as
>>>>>>>> from a
>>>>>>>>>>>> broker
>>>>>>>>>>>>>>> (does
>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>> matter
>>>>>>>>>>>>>>>>>>>>>> which one?).
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I think I want to adopt these suggestions,
>>>>>> just
>>>>>>>> had
>>>>>>>>>> a few
>>>>>>>>>>>>>>>>> questions
>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> details.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Thu, Jan 5, 2023 at 5:05 PM Jason
>>>>>> Gustafson
>>>>>>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Thanks for the proposal.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> I was thinking about the implementation a
>>>>>>> little
>>>>>>>>>> bit.
>>>>>>>>>>>> In the
>>>>>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>> proposal, the behavior depends on whether we
>>>>>>>> have
>>>>>>>>> an
>>>>>>>>>>>> old or
>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>> client.
>>>>>>>>>>>>>>>>>>>>>>> For
>>>>>>>>>>>>>>>>>>>>>>> old clients, we send `DescribeTransactions`
>>>>>>> and
>>>>>>>>>> verify
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>> new clients, we send `AddPartitionsToTxn`.
>>>>>> We
>>>>>>>>> might
>>>>>>>>>> be
>>>>>>>>>>>> able
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> simplify
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> implementation if we can use the same
>>>>>> request
>>>>>>>>> type.
>>>>>>>>>> For
>>>>>>>>>>>>>>>> example,
>>>>>>>>>>>>>>>>>>>> what if
>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>> bump the protocol version for
>>>>>>>> `AddPartitionsToTxn`
>>>>>>>>>> and
>>>>>>>>>>>> add a
>>>>>>>>>>>>>>>>>>>>>>> `validateOnly`
>>>>>>>>>>>>>>>>>>>>>>> flag? For older versions, we can set
>>>>>>>>>>>> `validateOnly=true` so
>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> request only returns successfully if the
>>>>>>>> partition
>>>>>>>>>> had
>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>> added.
>>>>>>>>>>>>>>>>>>>>>>> For new versions, we can set
>>>>>>>> `validateOnly=false`
>>>>>>>>>> and
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>> added to the transaction. The other slightly
>>>>>>>>>> annoying
>>>>>>>>>>>> thing
>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>> get around is the need to collect the
>>>>>>>> transaction
>>>>>>>>>> state
>>>>>>>>>>>> for
>>>>>>>>>>>>>>> all
>>>>>>>>>>>>>>>>>>>>> partitions
>>>>>>>>>>>>>>>>>>>>>>> even when we only care about a subset.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Some additional improvements to consider:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> - We can give `AddPartitionsToTxn` better
>>>>>>> batch
>>>>>>>>>> support
>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>> inter-broker
>>>>>>>>>>>>>>>>>>>>>>> usage. Currently we only allow one
>>>>>>>>>> `TransactionalId` to
>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>> specified,
>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>> the broker may get some benefit being able
>>>>>> to
>>>>>>>>> batch
>>>>>>>>>>>> across
>>>>>>>>>>>>>>>>> multiple
>>>>>>>>>>>>>>>>>>>>>>> transactions.
>>>>>>>>>>>>>>>>>>>>>>> - Another small improvement is skipping
>>>>>> topic
>>>>>>>>>>>> authorization
>>>>>>>>>>>>>>>>> checks
>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>> `AddPartitionsToTxn` when the request is
>>>>>> from
>>>>>>> a
>>>>>>>>>> broker.
>>>>>>>>>>>>>>> Perhaps
>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>> add
>>>>>>>>>>>>>>>>>>>>>>> a field for the `LeaderId` or something like
>>>>>>>> that
>>>>>>>>>> and
>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>> CLUSTER
>>>>>>>>>>>>>>>>>>>>>>> permission when set.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>> Jason
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation. It makes sense
>>>>>>> to
>>>>>>>> me
>>>>>>>>>> now.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Jun
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 1:42 PM Justine
>>>>>>> Olshan
>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> My understanding of the mechanism is
>>>>>> that
>>>>>>>> when
>>>>>>>>>> we
>>>>>>>>>>>> get to
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> last
>>>>>>>>>>>>>>>>>>>>>>> epoch,
>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>> increment to the fencing/last epoch and
>>>>>> if
>>>>>>>> any
>>>>>>>>>>>> further
>>>>>>>>>>>>>>>>> requests
>>>>>>>>>>>>>>>>>>>> come
>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>> this producer ID they are fenced. Then
>>>>>> the
>>>>>>>>>> producer
>>>>>>>>>>>>>>> gets a
>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>> ID
>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>> restarts with epoch/sequence 0. The
>>>>>> fenced
>>>>>>>>> epoch
>>>>>>>>>>>> sticks
>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> duration of producer.id.expiration.ms
>>>>>> and
>>>>>>>>>> blocks
>>>>>>>>>>>> any
>>>>>>>>>>>>>>> late
>>>>>>>>>>>>>>>>>>>> messages
>>>>>>>>>>>>>>>>>>>>>>>> there.
>>>>>>>>>>>>>>>>>>>>>>>>> The new ID will get to take advantage of
>>>>>>> the
>>>>>>>>>>>> improved
>>>>>>>>>>>>>>>>> semantics
>>>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>>>>>> non-zero start sequences. So I think we
>>>>>>> are
>>>>>>>>>> covered.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> The only potential issue is overloading
>>>>>>> the
>>>>>>>>>> cache,
>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>> hopefully
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> improvements (lowered
>>>>>>>>> producer.id.expiration.ms
>>>>>>>>>> )
>>>>>>>>>>>> will
>>>>>>>>>>>>>>> help
>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>> that.
>>>>>>>>>>>>>>>>>>>>>>>> Let
>>>>>>>>>>>>>>>>>>>>>>>>> me know if you still have concerns.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
>>>>>>>>>>>>>>>>>>>> <ju...@confluent.io.invalid>
>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 70. The proposed fencing logic doesn't
>>>>>>>> apply
>>>>>>>>>> when
>>>>>>>>>>>> pid
>>>>>>>>>>>>>>>>>> changes,
>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>> right? If so, I am not sure how
>>>>>> complete
>>>>>>>> we
>>>>>>>>>> are
>>>>>>>>>>>>>>>> addressing
>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>> issue
>>>>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>> the pid changes more frequently.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 16, 2022 at 9:16 AM
>>>>>> Justine
>>>>>>>>> Olshan
>>>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for replying!
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> 70.We already do the overflow
>>>>>>> mechanism,
>>>>>>>>> so
>>>>>>>>>> my
>>>>>>>>>>>>>>> change
>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>> happen more often.
>>>>>>>>>>>>>>>>>>>>>>>>>>> I was also not suggesting a new
>>>>>> field
>>>>>>> in
>>>>>>>>> the
>>>>>>>>>>>> log,
>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> response,
>>>>>>>>>>>>>>>>>>>>>>>>>>> which would be gated by the client
>>>>>>>>> version.
>>>>>>>>>>>> Sorry if
>>>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>> unclear. I think we are starting to
>>>>>>>>> diverge.
>>>>>>>>>>>>>>>>>>>>>>>>>>> The goal of this KIP is to not
>>>>>> change
>>>>>>> to
>>>>>>>>> the
>>>>>>>>>>>> marker
>>>>>>>>>>>>>>>>> format
>>>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> 71. Yes, I guess I was going under
>>>>>> the
>>>>>>>>>>>> assumption
>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> log
>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>> look at its last epoch and treat it
>>>>>> as
>>>>>>>> the
>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>> epoch. I
>>>>>>>>>>>>>>>>>>>>>>> suppose
>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>> have some special logic that if the
>>>>>>> last
>>>>>>>>>> epoch
>>>>>>>>>>>> was
>>>>>>>>>>>>>>> on a
>>>>>>>>>>>>>>>>>>>> marker
>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>>> expect the next epoch or something
>>>>>>> like
>>>>>>>>>> that. We
>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> distinguish
>>>>>>>>>>>>>>>>>>>>>>>>>>> based on whether we had a
>>>>>> commit/abort
>>>>>>>>>> marker.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> 72.
>>>>>>>>>>>>>>>>>>>>>>>>>>>> if the producer epoch hasn't been
>>>>>>>> bumped
>>>>>>>>>> on
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
>>>>>>>> message
>>>>>>>>>> will
>>>>>>>>>>>> fail
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> sequence
>>>>>>>>>>>>>>>>>>>>>>>>>> validation
>>>>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the producer
>>>>>>>> epoch
>>>>>>>>>> has
>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>> bumped,
>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>> ignore
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck message
>>>>>>>> could
>>>>>>>>>> be
>>>>>>>>>>>>>>> appended
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> log.
>>>>>>>>>>>>>>>>>>>>>>>> So,
>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
>>>>>> guard?
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I follow that "the
>>>>>>> message
>>>>>>>>> will
>>>>>>>>>>>> fail
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> sequence
>>>>>>>>>>>>>>>>>>>>>>>>>> validation".
>>>>>>>>>>>>>>>>>>>>>>>>>>> In some of these cases, we had an
>>>>>>> abort
>>>>>>>>>> marker
>>>>>>>>>>>> (due
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>> error)
>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>> then
>>>>>>>>>>>>>>>>>>>>>>>>>>> the late message comes in with the
>>>>>>>> correct
>>>>>>>>>>>> sequence
>>>>>>>>>>>>>>>>> number.
>>>>>>>>>>>>>>>>>>>> This
>>>>>>>>>>>>>>>>>>>>>>> is a
>>>>>>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>> covered by the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>> The latter case is actually not
>>>>>>>> something
>>>>>>>>>> we've
>>>>>>>>>>>>>>>>> considered
>>>>>>>>>>>>>>>>>>>>> here. I
>>>>>>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>> generally when we bump the epoch, we
>>>>>>> are
>>>>>>>>>>>> accepting
>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> sequence
>>>>>>>>>>>>>>>>>>>>>>>>> does
>>>>>>>>>>>>>>>>>>>>>>>>>>> not need to be checked anymore. My
>>>>>>>>>>>> understanding is
>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>> that we
>>>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>> typically bump epoch mid transaction
>>>>>>>>> (based
>>>>>>>>>> on a
>>>>>>>>>>>>>>> quick
>>>>>>>>>>>>>>>>> look
>>>>>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> code)
>>>>>>>>>>>>>>>>>>>>>>>>>>> but let me know if that is the case.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 15, 2022 at 12:23 PM Jun
>>>>>>> Rao
>>>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the reply.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Assigning a new pid on int
>>>>>>>> overflow
>>>>>>>>>> seems
>>>>>>>>>>>> a
>>>>>>>>>>>>>>> bit
>>>>>>>>>>>>>>>>>> hacky.
>>>>>>>>>>>>>>>>>>>> If
>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>> need a
>>>>>>>>>>>>>>>>>>>>>>>>>>> txn
>>>>>>>>>>>>>>>>>>>>>>>>>>>> level id, it will be better to
>>>>>> model
>>>>>>>>> this
>>>>>>>>>>>>>>> explicitly.
>>>>>>>>>>>>>>>>>>>> Adding a
>>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>> field
>>>>>>>>>>>>>>>>>>>>>>>>>>>> would require a bit more work
>>>>>> since
>>>>>>> it
>>>>>>>>>>>> requires a
>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>> txn
>>>>>>>>>>>>>>>>>>>>> marker
>>>>>>>>>>>>>>>>>>>>>>>>> format
>>>>>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the log. So, we probably need to
>>>>>>> guard
>>>>>>>>> it
>>>>>>>>>>>> with an
>>>>>>>>>>>>>>> IBP
>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>> metadata
>>>>>>>>>>>>>>>>>>>>>>>>>> version
>>>>>>>>>>>>>>>>>>>>>>>>>>>> and document the impact on
>>>>>> downgrade
>>>>>>>>> once
>>>>>>>>>> the
>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>> format
>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>> written
>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 71. Hmm, once the marker is
>>>>>> written,
>>>>>>>> the
>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>> expect
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>>>> append to be on the next epoch.
>>>>>> Does
>>>>>>>>> that
>>>>>>>>>>>> cover
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>> mentioned?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 72. Also, just to be clear on the
>>>>>>>>> stucked
>>>>>>>>>>>> message
>>>>>>>>>>>>>>>> issue
>>>>>>>>>>>>>>>>>>>>>>> described
>>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> motivation. With EoS, we also
>>>>>>> validate
>>>>>>>>> the
>>>>>>>>>>>>>>> sequence
>>>>>>>>>>>>>>>> id
>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>> idempotency.
>>>>>>>>>>>>>>>>>>>>>>>>>>> So,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the current logic, if the
>>>>>>>> producer
>>>>>>>>>> epoch
>>>>>>>>>>>>>>> hasn't
>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>>>> bumped on
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
>>>>>>>>> message
>>>>>>>>>> will
>>>>>>>>>>>>>>> fail
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> sequence
>>>>>>>>>>>>>>>>>>>>>>>>>>> validation
>>>>>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the
>>>>>> producer
>>>>>>>>>> epoch has
>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>> bumped, we
>>>>>>>>>>>>>>>>>>>>>>>> ignore
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck
>>>>>> message
>>>>>>>>>> could be
>>>>>>>>>>>>>>>> appended
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> log.
>>>>>>>>>>>>>>>>>>>>>>>>> So,
>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
>>>>>>> guard?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 10:44 AM
>>>>>>>> Justine
>>>>>>>>>>>> Olshan
>>>>>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias — thanks again for
>>>>>> taking
>>>>>>>>> time
>>>>>>>>>> to
>>>>>>>>>>>> look
>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>>>>>> You
>>>>>>>>>>>>>>>>>>>>>>> said:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My proposal was only focusing
>>>>>> to
>>>>>>>>> avoid
>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
>>>>>> added
>>>>>>>>>> without
>>>>>>>>>>>>>>>> registered
>>>>>>>>>>>>>>>>>>>>>>> partition.
>>>>>>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more details
>>>>>> to
>>>>>>>> the
>>>>>>>>>> KIP
>>>>>>>>>>>> about
>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>> scenario
>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I understand what
>>>>>> you
>>>>>>>>> mean
>>>>>>>>>>>> here.
>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>> motivation
>>>>>>>>>>>>>>>>>>>>>>>>> section
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> describes two scenarios about
>>>>>> how
>>>>>>>> the
>>>>>>>>>> record
>>>>>>>>>>>>>>> can be
>>>>>>>>>>>>>>>>>> added
>>>>>>>>>>>>>>>>>>>>>>>> without a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> registered partition:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Another way hanging
>>>>>> transactions
>>>>>>>> can
>>>>>>>>>>>> occur is
>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>> client
>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>> buggy
>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> may somehow try to write to a
>>>>>>>>> partition
>>>>>>>>>>>> before
>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>> adds
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For the first example of this
>>>>>>> would
>>>>>>>> it
>>>>>>>>>> be
>>>>>>>>>>>>>>> helpful
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> say
>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comes in after the abort, but
>>>>>>> before
>>>>>>>>> the
>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>> added
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction so it becomes
>>>>>>> "hanging."
>>>>>>>>>>>> Perhaps the
>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>> sentence
>>>>>>>>>>>>>>>>>>>>>>>>>>> describing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the message becoming part of the
>>>>>>>> next
>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>> (a
>>>>>>>>>>>>>>>>>>>>> different
>>>>>>>>>>>>>>>>>>>>>>>>> case)
>>>>>>>>>>>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not properly differentiated.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun — thanks for reading the
>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. The int typing was a
>>>>>> concern.
>>>>>>>>>> Currently
>>>>>>>>>>>> we
>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>> mechanism
>>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>> place
>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fence the final epoch when the
>>>>>>> epoch
>>>>>>>>> is
>>>>>>>>>>>> about to
>>>>>>>>>>>>>>>>>> overflow
>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>> assign
>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer ID with epoch 0. Of
>>>>>>> course,
>>>>>>>>>> this
>>>>>>>>>>>> is a
>>>>>>>>>>>>>>> bit
>>>>>>>>>>>>>>>>>> tricky
>>>>>>>>>>>>>>>>>>>>>>> when it
>>>>>>>>>>>>>>>>>>>>>>>>>> comes
>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response back to the client.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Making this a long could be
>>>>>>> another
>>>>>>>>>> option,
>>>>>>>>>>>> but
>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>> wonder
>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implications on changing this
>>>>>>> field
>>>>>>>> if
>>>>>>>>>> the
>>>>>>>>>>>>>>> epoch is
>>>>>>>>>>>>>>>>>>>>> persisted
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>> disk?
>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to check the usages.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 71.This was something Matthias
>>>>>>> asked
>>>>>>>>>> about
>>>>>>>>>>>> as
>>>>>>>>>>>>>>>> well. I
>>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>>>>>>> considering a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible edge case where a
>>>>>> produce
>>>>>>>>>> request
>>>>>>>>>>>> from
>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>> somehow
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> gets sent right after the marker
>>>>>>> is
>>>>>>>>>>>> written, but
>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> alerted of the newly bumped
>>>>>> epoch.
>>>>>>>> In
>>>>>>>>>> this
>>>>>>>>>>>>>>> case, we
>>>>>>>>>>>>>>>>> may
>>>>>>>>>>>>>>>>>>>>>>> include
>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we don't want to. I suppose
>>>>>>> we
>>>>>>>>>> could
>>>>>>>>>>>> try
>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>> client
>>>>>>>>>>>>>>>>>>>>>>>>>>> side
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to bump the epoch after sending
>>>>>> an
>>>>>>>>>> endTxn as
>>>>>>>>>>>>>>> well
>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>> scenario
>>>>>>>>>>>>>>>>>>>>>>>>> —
>>>>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wonder how it would work when
>>>>>> the
>>>>>>>>>> server is
>>>>>>>>>>>>>>>> aborting
>>>>>>>>>>>>>>>>>>>> based
>>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error. I could also be missing
>>>>>>>>>> something and
>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>> scenario
>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again to everyone reading
>>>>>>> and
>>>>>>>>>>>> commenting.
>>>>>>>>>>>>>>>> Let
>>>>>>>>>>>>>>>>> me
>>>>>>>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> further questions or comments.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 9:41 AM
>>>>>>> Jun
>>>>>>>>> Rao
>>>>>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP. A couple
>>>>>> of
>>>>>>>>>> comments.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Currently, the producer
>>>>>>> epoch
>>>>>>>> is
>>>>>>>>>> an
>>>>>>>>>>>> int.
>>>>>>>>>>>>>>> I am
>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>> sure
>>>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>> enough
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to accommodate all
>>>>>> transactions
>>>>>>> in
>>>>>>>>> the
>>>>>>>>>>>>>>> lifetime
>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>> producer.
>>>>>>>>>>>>>>>>>>>>>>>>>> Should
>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> change that to a long or add a
>>>>>>> new
>>>>>>>>>> long
>>>>>>>>>>>> field
>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>> txnId?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 71. "it will write the prepare
>>>>>>>>> commit
>>>>>>>>>>>> message
>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>> bumped
>>>>>>>>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> send
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WriteTxnMarkerRequests with
>>>>>> the
>>>>>>>>> bumped
>>>>>>>>>>>> epoch."
>>>>>>>>>>>>>>>> Hmm,
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> associated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the current txn right?
>>>>>> So,
>>>>>>> it
>>>>>>>>>> seems
>>>>>>>>>>>>>>> weird to
>>>>>>>>>>>>>>>>>>>> write a
>>>>>>>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with a bumped epoch. Should we
>>>>>>>> only
>>>>>>>>>> bump
>>>>>>>>>>>> up
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>> EndTxnResponse
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rename the field to sth like
>>>>>>>>>>>>>>> nextProducerEpoch?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 12, 2022 at 8:54
>>>>>> PM
>>>>>>>>>> Matthias
>>>>>>>>>>>> J.
>>>>>>>>>>>>>>> Sax <
>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the background.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30: SGTM. My proposal was
>>>>>>>> only
>>>>>>>>>>>> focusing
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> avoid
>>>>>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
>>>>>>>> added
>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>> registered
>>>>>>>>>>>>>>>>>>>>>>>> partition.
>>>>>>>>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more
>>>>>> details
>>>>>>>> to
>>>>>>>>>> the
>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>> scenario
>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40: I think you hit a fair
>>>>>>> point
>>>>>>>>>> about
>>>>>>>>>>>> race
>>>>>>>>>>>>>>>>>>>> conditions
>>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>> client
>>>>>>>>>>>>>>>>>>>>>>>>>>> bugs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (incorrectly not bumping the
>>>>>>>>>> epoch). The
>>>>>>>>>>>>>>>>>>>>>>> complexity/confusion
>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the bumped epoch I see, is
>>>>>>>> mainly
>>>>>>>>>> for
>>>>>>>>>>>>>>> internal
>>>>>>>>>>>>>>>>>>>>> debugging,
>>>>>>>>>>>>>>>>>>>>>>> ie,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inspecting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log segment dumps -- it
>>>>>> seems
>>>>>>>>>> harder to
>>>>>>>>>>>>>>> reason
>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> system
>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>> us
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> humans. But if we get better
>>>>>>>>>>>> guarantees, it
>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>> worth to
>>>>>>>>>>>>>>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped epoch.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60: as I mentioned already,
>>>>>> I
>>>>>>>>> don't
>>>>>>>>>>>> know the
>>>>>>>>>>>>>>>>> broker
>>>>>>>>>>>>>>>>>>>>>>> internals
>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> provide
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more input. So if nobody
>>>>>> else
>>>>>>>>> chimes
>>>>>>>>>>>> in, we
>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>> move
>>>>>>>>>>>>>>>>>>>>>>>>>>> forward
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with your proposal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 12/6/22 4:22 PM, Justine
>>>>>>>> Olshan
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi all,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> After Artem's questions
>>>>>>> about
>>>>>>>>>> error
>>>>>>>>>>>>>>> behavior,
>>>>>>>>>>>>>>>>>> I've
>>>>>>>>>>>>>>>>>>>>>>>>> re-evaluated
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unknown producer ID
>>>>>>> exception
>>>>>>>>> and
>>>>>>>>>> had
>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>> discussions
>>>>>>>>>>>>>>>>>>>>>>>>> offline.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think generally it makes
>>>>>>>> sense
>>>>>>>>>> to
>>>>>>>>>>>>>>> simplify
>>>>>>>>>>>>>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>> handling
>>>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>> cases
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this and the
>>>>>>>> UNKNOWN_PRODUCER_ID
>>>>>>>>>> error
>>>>>>>>>>>>>>> has a
>>>>>>>>>>>>>>>>>> pretty
>>>>>>>>>>>>>>>>>>>>> long
>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complicated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> history. Because of this,
>>>>>> I
>>>>>>>>>> propose
>>>>>>>>>>>>>>> adding a
>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ABORTABLE_ERROR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that when encountered by
>>>>>> new
>>>>>>>>>> clients
>>>>>>>>>>>>>>> (gated
>>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> produce
>>>>>>>>>>>>>>>>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will simply abort the
>>>>>>>>> transaction.
>>>>>>>>>>>> This
>>>>>>>>>>>>>>>> allows
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> server
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> say
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in whether the client
>>>>>> aborts
>>>>>>>> and
>>>>>>>>>> makes
>>>>>>>>>>>>>>>> handling
>>>>>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>>>>>> simpler.
>>>>>>>>>>>>>>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> future, we can also use
>>>>>> this
>>>>>>>>>> error in
>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>> situations
>>>>>>>>>>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>> want
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> abort the transactions. We
>>>>>>> can
>>>>>>>>>> even
>>>>>>>>>>>> use on
>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>> apis.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've added this to the
>>>>>> KIP.
>>>>>>>> Let
>>>>>>>>> me
>>>>>>>>>>>> know if
>>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>>>>> questions
>>>>>>>>>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 2, 2022 at
>>>>>> 10:22
>>>>>>>> AM
>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>> Olshan
>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>>>> jolshan@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey Matthias,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30 — Maybe I also
>>>>>> didn't
>>>>>>>>>> express
>>>>>>>>>>>>>>> myself
>>>>>>>>>>>>>>>>>>>> clearly.
>>>>>>>>>>>>>>>>>>>>> For
>>>>>>>>>>>>>>>>>>>>>>>>> older
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't have a way to
>>>>>>>> distinguish
>>>>>>>>>>>> between a
>>>>>>>>>>>>>>>>>> previous
>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction since we
>>>>>> don't
>>>>>>>> have
>>>>>>>>>> the
>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>> bump.
>>>>>>>>>>>>>>>>>>>> This
>>>>>>>>>>>>>>>>>>>>>>> means
>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message from the previous
>>>>>>>>>> transaction
>>>>>>>>>>>>>>> may be
>>>>>>>>>>>>>>>>>>>> added to
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>> one.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> With
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older clients — we can't
>>>>>>>>>> guarantee
>>>>>>>>>>>> this
>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>> happen
>>>>>>>>>>>>>>>>>>>>>>> if we
>>>>>>>>>>>>>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call
>>>>>>> (why
>>>>>>>> we
>>>>>>>>>> make
>>>>>>>>>>>>>>> changes
>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> newer
>>>>>>>>>>>>>>>>>>>>>>>>>>> client)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can at least gate some by
>>>>>>>>>> ensuring
>>>>>>>>>>>> that
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>>>>> has
>>>>>>>>>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>>>>>>>>> added
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. The
>>>>>> rationale
>>>>>>>> here
>>>>>>>>>> is
>>>>>>>>>>>> that
>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>> likely
>>>>>>>>>>>>>>>>>>>>>>>>> LESS
>>>>>>>>>>>>>>>>>>>>>>>>>>> late
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrivals
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as time goes on, so
>>>>>>> hopefully
>>>>>>>>>> most
>>>>>>>>>>>> late
>>>>>>>>>>>>>>>>> arrivals
>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>> come
>>>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>> BEFORE
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call.
>>>>>>>> Those
>>>>>>>>>> that
>>>>>>>>>>>>>>> arrive
>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>> properly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> gated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>> describeTransactions
>>>>>>>>>>>> approach.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we take the approach
>>>>>> you
>>>>>>>>>>>> suggested,
>>>>>>>>>>>>>>> ANY
>>>>>>>>>>>>>>>>> late
>>>>>>>>>>>>>>>>>>>>> arrival
>>>>>>>>>>>>>>>>>>>>>>>>> from a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> previous
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction will be
>>>>>> added.
>>>>>>>> And
>>>>>>>>> we
>>>>>>>>>>>> don't
>>>>>>>>>>>>>>> want
>>>>>>>>>>>>>>>>>>>> that. I
>>>>>>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>> see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> benefit in sending
>>>>>>>>>> addPartitionsToTxn
>>>>>>>>>>>>>>> over
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> describeTxns
>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> They
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> both be one extra RPC to
>>>>>>> the
>>>>>>>>> Txn
>>>>>>>>>>>>>>>> coordinator.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To be clear — newer
>>>>>> clients
>>>>>>>>> will
>>>>>>>>>> use
>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTxns.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that if we
>>>>>>> have
>>>>>>>>>> some
>>>>>>>>>>>> delay
>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> client
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> bump
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it could continue to send
>>>>>>>> epoch
>>>>>>>>>> 73
>>>>>>>>>>>> and
>>>>>>>>>>>>>>> those
>>>>>>>>>>>>>>>>>>>> records
>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fenced.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Perhaps this is not an
>>>>>>> issue
>>>>>>>> if
>>>>>>>>>> we
>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>> allow
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>> produce
>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> go
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through before the EndTxn
>>>>>>>>> request
>>>>>>>>>>>>>>> returns.
>>>>>>>>>>>>>>>> I'm
>>>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cases of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure. I will need to
>>>>>>> think
>>>>>>>>> on
>>>>>>>>>>>> this a
>>>>>>>>>>>>>>> bit.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I wasn't sure if it was
>>>>>>> that
>>>>>>>>>>>> confusing.
>>>>>>>>>>>>>>> But
>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> investigate other ways.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure these are
>>>>>> the
>>>>>>>> same
>>>>>>>>>>>>>>> purgatories
>>>>>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>>>>>>> is a
>>>>>>>>>>>>>>>>>>>>>>>>>>> produce
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory (I was planning
>>>>>>> on
>>>>>>>>>> using a
>>>>>>>>>>>>>>>> callback
>>>>>>>>>>>>>>>>>>>> rather
>>>>>>>>>>>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the other is simply a
>>>>>>> request
>>>>>>>>> to
>>>>>>>>>>>> append
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> log.
>>>>>>>>>>>>>>>>>>>>> Not
>>>>>>>>>>>>>>>>>>>>>>>> sure
>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> structure here for
>>>>>>> ordering,
>>>>>>>>> but
>>>>>>>>>> my
>>>>>>>>>>>>>>>>>> understanding
>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handle the write request
>>>>>>>> before
>>>>>>>>>> it
>>>>>>>>>>>> hears
>>>>>>>>>>>>>>>> back
>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> Txn
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Coordinator.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if I
>>>>>>>> misunderstood
>>>>>>>>>>>> something
>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>>>>>>>> unclear.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 1, 2022 at
>>>>>>> 12:15
>>>>>>>> PM
>>>>>>>>>>>> Matthias
>>>>>>>>>>>>>>> J.
>>>>>>>>>>>>>>>>> Sax
>>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the details
>>>>>>>>> Justine!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side change
>>>>>>> for
>>>>>>>> 2
>>>>>>>>> is
>>>>>>>>>>>>>>> removing
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> addPartitions
>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need to
>>>>>>> make
>>>>>>>>>> this
>>>>>>>>>>>> from
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> txn
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think I did not
>>>>>> express
>>>>>>>>> myself
>>>>>>>>>>>>>>> clearly. I
>>>>>>>>>>>>>>>>>>>>> understand
>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should) change the
>>>>>>> producer
>>>>>>>> to
>>>>>>>>>> not
>>>>>>>>>>>> send
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
>>>>>>>>>>>>>>>>>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> longer. But I don't
>>>>>> thinks
>>>>>>>>> it's
>>>>>>>>>>>>>>> requirement
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> change
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> broker?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What I am trying to say
>>>>>>> is:
>>>>>>>>> as a
>>>>>>>>>>>>>>> safe-guard
>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>> improvement
>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producers, the partition
>>>>>>>>> leader
>>>>>>>>>> can
>>>>>>>>>>>> just
>>>>>>>>>>>>>>>> send
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request to the
>>>>>>>> TX-coordinator
>>>>>>>>>> in any
>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> old
>>>>>>>>>>>>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> correctly did send the
>>>>>>>>>>>> `addPartition`
>>>>>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> already, the
>>>>>>> TX-coordinator
>>>>>>>>> can
>>>>>>>>>> just
>>>>>>>>>>>>>>>> "ignore"
>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>> idempotent.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if the old producer has
>>>>>> a
>>>>>>>> bug
>>>>>>>>>> and
>>>>>>>>>>>> did
>>>>>>>>>>>>>>>> forget
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> sent
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `addPartition`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request, we would now
>>>>>>> ensure
>>>>>>>>>> that
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>> indeed
>>>>>>>>>>>>>>>>>>>>>>>>>> added
>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX and thus fix a
>>>>>>> potential
>>>>>>>>>>>> producer bug
>>>>>>>>>>>>>>>>> (even
>>>>>>>>>>>>>>>>>>>> if we
>>>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fencing via the bump
>>>>>>> epoch).
>>>>>>>>> --
>>>>>>>>>> It
>>>>>>>>>>>>>>> seems to
>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there a reason to not do
>>>>>>>> this?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is ongoing
>>>>>> =
>>>>>>>>>> partition
>>>>>>>>>>>> was
>>>>>>>>>>>>>>>> added
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>> via
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn. We
>>>>>>>> check
>>>>>>>>>> this
>>>>>>>>>>>> with
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this wasn't
>>>>>>>>>> sufficiently
>>>>>>>>>>>>>>>>> explained
>>>>>>>>>>>>>>>>>>>> here:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do what I propose
>>>>>> in
>>>>>>>>>> (20), we
>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `DescribeTransaction`
>>>>>>> call,
>>>>>>>> as
>>>>>>>>>> the
>>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>> leader
>>>>>>>>>>>>>>>>>>>>>>> adds
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for older clients and we
>>>>>>> get
>>>>>>>>>> this
>>>>>>>>>>>> check
>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>> free.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is that
>>>>>> if
>>>>>>>> any
>>>>>>>>>>>> messages
>>>>>>>>>>>>>>>>> somehow
>>>>>>>>>>>>>>>>>>>> come
>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the producer,
>>>>>>> they
>>>>>>>>>> will be
>>>>>>>>>>>>>>>> fenced.
>>>>>>>>>>>>>>>>>>>>> However,
>>>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it can be
>>>>>>>>>> discussed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree that we should
>>>>>>> have
>>>>>>>>>> epoch
>>>>>>>>>>>>>>> fencing.
>>>>>>>>>>>>>>>> My
>>>>>>>>>>>>>>>>>>>>>>> question is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> different:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Assume we are at epoch
>>>>>> 73,
>>>>>>>> and
>>>>>>>>>> we
>>>>>>>>>>>> have
>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>> ongoing
>>>>>>>>>>>>>>>>>>>>>>>>>> transaction,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed. It seems
>>>>>>> natural
>>>>>>>> to
>>>>>>>>>>>> write the
>>>>>>>>>>>>>>>>>> "prepare
>>>>>>>>>>>>>>>>>>>>>>> commit"
>>>>>>>>>>>>>>>>>>>>>>>>>>> marker
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
>>>>>>> both
>>>>>>>>> with
>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>> 73,
>>>>>>>>>>>>>>>>> too,
>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>> belongs
>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current transaction. Of
>>>>>>>>> course,
>>>>>>>>>> we
>>>>>>>>>>>> now
>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>> bump
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> expect
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the next requests to
>>>>>> have
>>>>>>>>> epoch
>>>>>>>>>> 74,
>>>>>>>>>>>> and
>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>> reject
>>>>>>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch 73, as the
>>>>>>>> corresponding
>>>>>>>>>> TX
>>>>>>>>>>>> for
>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>> 73
>>>>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems you propose to
>>>>>>>> write
>>>>>>>>>> the
>>>>>>>>>>>>>>> "prepare
>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>>>>> marker"
>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
>>>>>>> with
>>>>>>>>>> epoch 74
>>>>>>>>>>>>>>>> though,
>>>>>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>> work,
>>>>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems confusing. Is
>>>>>> there
>>>>>>> a
>>>>>>>>>> reason
>>>>>>>>>>>> why
>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> bumped
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 74
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of the current
>>>>>>> epoch
>>>>>>>>> 73?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are checking if
>>>>>>> the
>>>>>>>>>>>>>>> transaction is
>>>>>>>>>>>>>>>>>>>> ongoing,
>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the leader
>>>>>>>>> partition
>>>>>>>>>> to
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for this
>>>>>>>>>> message to
>>>>>>>>>>>> come
>>>>>>>>>>>>>>>>> back,
>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>> theory
>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
>>>>>> that
>>>>>>>>> would
>>>>>>>>>>>> make the
>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why we
>>>>>> can
>>>>>>>>> check
>>>>>>>>>> the
>>>>>>>>>>>>>>> leader
>>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks. Got it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, is this really
>>>>>> an
>>>>>>>>>> issue?
>>>>>>>>>>>> We put
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> produce
>>>>>>>>>>>>>>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory, so how could
>>>>>> we
>>>>>>>>>> process
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Don't we need to put the
>>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
>>>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> too,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for this case, and
>>>>>> process
>>>>>>>>> both
>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>>> in-order?
>>>>>>>>>>>>>>>>>>>>>>> (Again,
>>>>>>>>>>>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> knowledge is limited and
>>>>>>>> maybe
>>>>>>>>>> we
>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>> maintain
>>>>>>>>>>>>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>>>>>>>>>>> order
>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case, what seems to be
>>>>>> an
>>>>>>>>> issue
>>>>>>>>>>>> IMHO,
>>>>>>>>>>>>>>> and I
>>>>>>>>>>>>>>>>> am
>>>>>>>>>>>>>>>>>>>>>>> wondering
>>>>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> changing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request handling to
>>>>>>> preserve
>>>>>>>>>> order
>>>>>>>>>>>> for
>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>> might be
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cleaner
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution?)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/30/22 3:28 PM,
>>>>>> Artem
>>>>>>>>>> Livshits
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think the interesting
>>>>>>>> part
>>>>>>>>> is
>>>>>>>>>>>> not in
>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>> (because
>>>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> tries
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> figure out when
>>>>>>>>>>>> UNKNOWN_PRODUCER_ID is
>>>>>>>>>>>>>>>>>> retriable
>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retryable,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's definitely not
>>>>>>> fatal),
>>>>>>>>> but
>>>>>>>>>>>> what
>>>>>>>>>>>>>>>> happens
>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'true' and falls
>>>>>> through.
>>>>>>>> In
>>>>>>>>>> the
>>>>>>>>>>>> old
>>>>>>>>>>>>>>>>> clients
>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> keep the behavior in
>>>>>> the
>>>>>>>> new
>>>>>>>>>>>> clients,
>>>>>>>>>>>>>>> I'd
>>>>>>>>>>>>>>>>>>>> expect it
>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> well.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 29, 2022 at
>>>>>>>> 11:57
>>>>>>>>>> AM
>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>> Olshan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>> <jolshan@confluent.io.invalid
>>>>>>>>>>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Artem and Jeff,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
>>>>>> look
>>>>>>>> and
>>>>>>>>>>>> sorry for
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> slow
>>>>>>>>>>>>>>>>>>>>>>>> response.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You both mentioned the
>>>>>>>>> change
>>>>>>>>>> to
>>>>>>>>>>>>>>> handle
>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear — this error
>>>>>> code
>>>>>>>> will
>>>>>>>>>> only
>>>>>>>>>>>> be
>>>>>>>>>>>>>>> sent
>>>>>>>>>>>>>>>>>> again
>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> client's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version is high enough
>>>>>>> to
>>>>>>>>>> ensure
>>>>>>>>>>>> we
>>>>>>>>>>>>>>>> handle
>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>> correctly.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The current (Java)
>>>>>>> client
>>>>>>>>>> handles
>>>>>>>>>>>>>>> this by
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>>>> (somewhat
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> long)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code snippet:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // An
>>>>>>> UNKNOWN_PRODUCER_ID
>>>>>>>>>> means
>>>>>>>>>>>> that
>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>> lost
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker. Depending on
>>>>>> the
>>>>>>>> log
>>>>>>>>>> start
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // offset, we may want
>>>>>>> to
>>>>>>>>>> retry
>>>>>>>>>>>>>>> these, as
>>>>>>>>>>>>>>>>>>>>> described
>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below. If
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> none of those apply,
>>>>>>> then
>>>>>>>>> for
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // idempotent
>>>>>> producer,
>>>>>>> we
>>>>>>>>>> will
>>>>>>>>>>>>>>> locally
>>>>>>>>>>>>>>>>> bump
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>> reset
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence numbers of
>>>>>>>>> in-flight
>>>>>>>>>>>> batches
>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // sequence 0, then
>>>>>>> retry
>>>>>>>>> the
>>>>>>>>>>>> failed
>>>>>>>>>>>>>>>> batch,
>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>> now
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> succeed.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the transactional
>>>>>>>> producer,
>>>>>>>>>> allow
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // batch to fail. When
>>>>>>>>>> processing
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> failed
>>>>>>>>>>>>>>>>>>>>> batch,
>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transition
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an abortable error and
>>>>>>>> set a
>>>>>>>>>> flag
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // indicating that we
>>>>>>> need
>>>>>>>>> to
>>>>>>>>>>>> bump the
>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>>> (if
>>>>>>>>>>>>>>>>>>>>>>>>> supported
>>>>>>>>>>>>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if (error ==
>>>>>>>>>>>>>>>> Errors.*UNKNOWN_PRODUCER_ID*)
>>>>>>>>>>>>>>>>> {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
>>>>>>>>>> (response.logStartOffset
>>>>>>>>>>>> ==
>>>>>>>>>>>>>>> -1)
>>>>>>>>>>>>>>>> {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // We don't
>>>>>>> know
>>>>>>>>>> the log
>>>>>>>>>>>>>>> start
>>>>>>>>>>>>>>>>>> offset
>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> response.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the request
>>>>>>>> until
>>>>>>>>>> we
>>>>>>>>>>>> get
>>>>>>>>>>>>>>> it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // The
>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
>>>>>>>>>>>>>>>> error
>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>>>>> added
>>>>>>>>>>>>>>>>>>>>>>>>>>> along
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProduceResponse which
>>>>>>>>>> includes the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              //
>>>>>>>> logStartOffset.
>>>>>>>>>> So
>>>>>>>>>>>> the
>>>>>>>>>>>>>>> '-1'
>>>>>>>>>>>>>>>>>>>> sentinel
>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> backward
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatibility.
>>>>>> Instead,
>>>>>>> it
>>>>>>>>> is
>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // a broker
>>>>>> to
>>>>>>>> not
>>>>>>>>>> know
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> logStartOffset at
>>>>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returning
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response because
>>>>>> the
>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // may have
>>>>>>>> moved
>>>>>>>>>> away
>>>>>>>>>>>> from
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> broker
>>>>>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> time
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initially raised to
>>>>>> the
>>>>>>>> time
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // response
>>>>>>> was
>>>>>>>>>> being
>>>>>>>>>>>>>>>>> constructed.
>>>>>>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>>>>>>>> cases,
>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retry the request: we
>>>>>>> are
>>>>>>>>>>>> guaranteed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // to
>>>>>>> eventually
>>>>>>>>>> get a
>>>>>>>>>>>>>>>>>> logStartOffset
>>>>>>>>>>>>>>>>>>>>> once
>>>>>>>>>>>>>>>>>>>>>>>>> things
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> settle
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> down.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
>>>>>>>>>>>>>>> (batch.sequenceHasBeenReset()) {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // When the
>>>>>>>> first
>>>>>>>>>>>> inflight
>>>>>>>>>>>>>>>> batch
>>>>>>>>>>>>>>>>>>>> fails
>>>>>>>>>>>>>>>>>>>>>>> due to
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> truncation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> then the sequences of
>>>>>>> all
>>>>>>>>> the
>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // in flight
>>>>>>>>> batches
>>>>>>>>>>>> would
>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>>>>> restarted
>>>>>>>>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> beginning.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, when those
>>>>>>>>> responses
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // come back
>>>>>>>> from
>>>>>>>>>> the
>>>>>>>>>>>>>>> broker,
>>>>>>>>>>>>>>>>> they
>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>>>>>>>> come
>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
>>>>>>> error.
>>>>>>>>> In
>>>>>>>>>> this
>>>>>>>>>>>>>>> case,
>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // reset the
>>>>>>>>>> sequence
>>>>>>>>>>>>>>> numbers
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> beginning.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          } else if
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>> (lastAckedOffset(batch.topicPartition).orElse(
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
>>>>>>>>>>>>>>>>>>>>>>>>>> response.logStartOffset) {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // The head
>>>>>> of
>>>>>>>> the
>>>>>>>>>> log
>>>>>>>>>>>> has
>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>> removed,
>>>>>>>>>>>>>>>>>>>>>>>>> probably
>>>>>>>>>>>>>>>>>>>>>>>>>>> due
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retention time
>>>>>> elapsing.
>>>>>>>> In
>>>>>>>>>> this
>>>>>>>>>>>> case,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // we expect
>>>>>>> to
>>>>>>>>>> lose the
>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>>>> state.
>>>>>>>>>>>>>>>>>>>>>>> For
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer, reset the
>>>>>>>>> sequences
>>>>>>>>>> of
>>>>>>>>>>>> all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // inflight
>>>>>>>>> batches
>>>>>>>>>> to
>>>>>>>>>>>> be
>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> beginning
>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>> retry
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the transaction
>>>>>>> does
>>>>>>>>> not
>>>>>>>>>>>> need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // be
>>>>>> aborted.
>>>>>>>> For
>>>>>>>>>> the
>>>>>>>>>>>>>>>> idempotent
>>>>>>>>>>>>>>>>>>>>>>> producer,
>>>>>>>>>>>>>>>>>>>>>>>>> bump
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> avoid
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reusing (sequence,
>>>>>>> epoch)
>>>>>>>>>> pairs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              if
>>>>>>>>>> (isTransactional()) {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>> txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>> this.producerIdAndEpoch);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              } else {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> requestEpochBumpForPartition(batch.topicPartition);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          if
>>>>>>>>> (!isTransactional())
>>>>>>>>>> {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // For the
>>>>>>>>>> idempotent
>>>>>>>>>>>>>>> producer,
>>>>>>>>>>>>>>>>>>>> always
>>>>>>>>>>>>>>>>>>>>>>> retry
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors. If the batch
>>>>>> has
>>>>>>>> the
>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              // producer
>>>>>> ID
>>>>>>>> and
>>>>>>>>>>>> epoch,
>>>>>>>>>>>>>>>>> request a
>>>>>>>>>>>>>>>>>>>> bump
>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Otherwise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the
>>>>>> produce.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>> requestEpochBumpForPartition(batch.topicPartition);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              return true;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I was considering
>>>>>>> keeping
>>>>>>>>> this
>>>>>>>>>>>>>>> behavior —
>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>> am
>>>>>>>>>>>>>>>>>>>>>>> open
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simplifying
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We are leaving changes
>>>>>>> to
>>>>>>>>>> older
>>>>>>>>>>>>>>> clients
>>>>>>>>>>>>>>>> off
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> table
>>>>>>>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> caused
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many issues for
>>>>>> clients
>>>>>>> in
>>>>>>>>> the
>>>>>>>>>>>> past.
>>>>>>>>>>>>>>>>>> Previously
>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we didn't have the
>>>>>>>>> mechanisms
>>>>>>>>>> in
>>>>>>>>>>>>>>> place to
>>>>>>>>>>>>>>>>>>>> detect
>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> legitimate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case vs some bug or
>>>>>> gap
>>>>>>> in
>>>>>>>>> the
>>>>>>>>>>>>>>> protocol.
>>>>>>>>>>>>>>>>>>>> Ensuring
>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> own epoch should close
>>>>>>>> this
>>>>>>>>>> gap.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> And to address Jeff's
>>>>>>>> second
>>>>>>>>>>>> point:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *does the typical
>>>>>>> produce
>>>>>>>>>> request
>>>>>>>>>>>> path
>>>>>>>>>>>>>>>>> append
>>>>>>>>>>>>>>>>>>>>>>> records
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>> local
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along*
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *with the
>>>>>>>>>> currentTxnFirstOffset
>>>>>>>>>>>>>>>>> information?
>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand*
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *when the field is
>>>>>>> written
>>>>>>>>> to
>>>>>>>>>>>> disk.*
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, the first produce
>>>>>>>>> request
>>>>>>>>>>>>>>> populates
>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>> field
>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>> writes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> offset
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as part of the record
>>>>>>>> batch
>>>>>>>>>> and
>>>>>>>>>>>> also
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> snapshot.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we reload the records
>>>>>> on
>>>>>>>>>> restart
>>>>>>>>>>>>>>> and/or
>>>>>>>>>>>>>>>>>>>>>>> reassignment,
>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> repopulate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> field with the
>>>>>> snapshot
>>>>>>>> from
>>>>>>>>>> disk
>>>>>>>>>>>>>>> along
>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> rest
>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if there
>>>>>> are
>>>>>>>>>> further
>>>>>>>>>>>>>>> comments
>>>>>>>>>>>>>>>>>>>> and/or
>>>>>>>>>>>>>>>>>>>>>>>>> questions.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
>>>>>> at
>>>>>>>> 9:00
>>>>>>>>>> PM
>>>>>>>>>>>> Jeff
>>>>>>>>>>>>>>> Kim
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>> <jeff.kim@confluent.io.invalid
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP! I
>>>>>>>> have
>>>>>>>>>> two
>>>>>>>>>>>>>>>> questions:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1) For new clients,
>>>>>> we
>>>>>>>> can
>>>>>>>>>> once
>>>>>>>>>>>> again
>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for sequences
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that are non-zero
>>>>>> when
>>>>>>>>> there
>>>>>>>>>> is
>>>>>>>>>>>> no
>>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>>>> present
>>>>>>>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This will indicate we
>>>>>>>>> missed
>>>>>>>>>> the
>>>>>>>>>>>> 0
>>>>>>>>>>>>>>>>> sequence
>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>> yet
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the log.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would like to
>>>>>>>> understand
>>>>>>>>>> the
>>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>> behavior
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> handle
>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and if there are any
>>>>>>>>> changes
>>>>>>>>>> we
>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>> making.
>>>>>>>>>>>>>>>>>>>> Maybe
>>>>>>>>>>>>>>>>>>>>>>> I'm
>>>>>>>>>>>>>>>>>>>>>>>>>>> missing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but we would want to
>>>>>>>>> identify
>>>>>>>>>>>>>>> whether we
>>>>>>>>>>>>>>>>>>>> missed
>>>>>>>>>>>>>>>>>>>>>>> the 0
>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients, no?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2) Upon returning
>>>>>> from
>>>>>>>> the
>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>> coordinator, we
>>>>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>> set
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as ongoing on the
>>>>>>> leader
>>>>>>>> by
>>>>>>>>>>>>>>> populating
>>>>>>>>>>>>>>>>>>>>>>>>>> currentTxnFirstOffset
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through the typical
>>>>>>>> produce
>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>>> handling.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does the typical
>>>>>>> produce
>>>>>>>>>> request
>>>>>>>>>>>> path
>>>>>>>>>>>>>>>>> append
>>>>>>>>>>>>>>>>>>>>>>> records
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>> local
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>> currentTxnFirstOffset
>>>>>>>>>>>>>>>>> information?
>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when the field is
>>>>>>> written
>>>>>>>>> to
>>>>>>>>>>>> disk.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeff
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
>>>>>> at
>>>>>>>>> 4:44
>>>>>>>>>> PM
>>>>>>>>>>>> Artem
>>>>>>>>>>>>>>>>>> Livshits
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
>>>>>>> alivshits@confluent.io
>>>>>>>>>> .invalid>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for the
>>>>>> KIP.
>>>>>>>> I
>>>>>>>>>> have
>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>> question.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 5) For new clients,
>>>>>> we
>>>>>>>> can
>>>>>>>>>> once
>>>>>>>>>>>>>>> again
>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe we had
>>>>>>>> problems
>>>>>>>>>> in the
>>>>>>>>>>>>>>> past
>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>> returning
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because it was
>>>>>>>> considered
>>>>>>>>>> fatal
>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>>>> client
>>>>>>>>>>>>>>>>>>>>>>>>>>> restart.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good to spell out
>>>>>> the
>>>>>>>> new
>>>>>>>>>> client
>>>>>>>>>>>>>>>> behavior
>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>> receives
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
>>>>>>> at
>>>>>>>>>> 10:00 AM
>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>> Olshan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>> <jo...@confluent.io.invalid>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
>>>>>>>> look
>>>>>>>>>>>> Matthias.
>>>>>>>>>>>>>>>> I've
>>>>>>>>>>>>>>>>>>>> tried
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> answer
>>>>>>>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> questions
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 10)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Right — so the
>>>>>>> hanging
>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>> occurs
>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> come in and the
>>>>>>>> partition
>>>>>>>>>> is
>>>>>>>>>>>> never
>>>>>>>>>>>>>>>> added
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> again.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never add the
>>>>>>> partition
>>>>>>>>> to
>>>>>>>>>> a
>>>>>>>>>>>>>>>>> transaction,
>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>> never
>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> marker
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never advance the
>>>>>>> LSO.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do end up
>>>>>>> adding
>>>>>>>>> the
>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>> (I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suppose
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happen before or
>>>>>>> after
>>>>>>>>> the
>>>>>>>>>> late
>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>> comes
>>>>>>>>>>>>>>>>>>>>> in)
>>>>>>>>>>>>>>>>>>>>>>>> then
>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> include
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late message in the
>>>>>>>> next
>>>>>>>>>>>>>>> (incorrect)
>>>>>>>>>>>>>>>>>>>>> transaction.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> So perhaps it is
>>>>>>>> clearer
>>>>>>>>> to
>>>>>>>>>>>> make
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> distinction
>>>>>>>>>>>>>>>>>>>>>>>>> between
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> eventually get
>>>>>> added
>>>>>>> to
>>>>>>>>> the
>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>> (but
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> wrong
>>>>>>>>>>>>>>>>>>>>>>>>>>> one)
>>>>>>>>>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that never get
>>>>>> added
>>>>>>>> and
>>>>>>>>>> become
>>>>>>>>>>>>>>>> hanging.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side
>>>>>>> change
>>>>>>>>> for
>>>>>>>>>> 2 is
>>>>>>>>>>>>>>>> removing
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitions
>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need
>>>>>>> to
>>>>>>>>> make
>>>>>>>>>>>> this
>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> txn
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In my opinion, the
>>>>>>>> issue
>>>>>>>>>> with
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
>>>>>>>>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is that we don't
>>>>>> have
>>>>>>>> the
>>>>>>>>>> epoch
>>>>>>>>>>>>>>> bump,
>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> belongs to the
>>>>>>> previous
>>>>>>>>>>>>>>> transaction or
>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>> one.
>>>>>>>>>>>>>>>>>>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition has been
>>>>>>>> added
>>>>>>>>> to
>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>> transaction.
>>>>>>>>>>>>>>>>>>>>> Of
>>>>>>>>>>>>>>>>>>>>>>>>> course,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> means
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't completely
>>>>>>> cover
>>>>>>>>> the
>>>>>>>>>> case
>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>>>>>>>>>> late
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have added the
>>>>>>>> partition
>>>>>>>>> to
>>>>>>>>>>>> the new
>>>>>>>>>>>>>>>>>>>>> transaction,
>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>> that's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something we will
>>>>>>> need
>>>>>>>>> the
>>>>>>>>>> new
>>>>>>>>>>>>>>> clients
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> cover.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is
>>>>>>> ongoing
>>>>>>>> =
>>>>>>>>>>>> partition
>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>> added to
>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> via
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn.
>>>>>>> We
>>>>>>>>>> check
>>>>>>>>>>>> this
>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this
>>>>>>> wasn't
>>>>>>>>>>>> sufficiently
>>>>>>>>>>>>>>>>>>>> explained
>>>>>>>>>>>>>>>>>>>>>>> here:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is
>>>>>> that
>>>>>>>> if
>>>>>>>>>> any
>>>>>>>>>>>>>>> messages
>>>>>>>>>>>>>>>>>>>> somehow
>>>>>>>>>>>>>>>>>>>>>>> come
>>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the
>>>>>>> producer,
>>>>>>>>> they
>>>>>>>>>>>> will be
>>>>>>>>>>>>>>>>>> fenced.
>>>>>>>>>>>>>>>>>>>>>>> However,
>>>>>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it
>>>>>> can
>>>>>>> be
>>>>>>>>>>>> discussed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 50)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It should be
>>>>>>>> synchronous
>>>>>>>>>>>> because
>>>>>>>>>>>>>>> if we
>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>> event
>>>>>>>>>>>>>>>>>>>>>>>>>> (ie,
>>>>>>>>>>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes us to need
>>>>>> to
>>>>>>>>> abort
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> transaction,
>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions to send
>>>>>>>>>> transaction
>>>>>>>>>>>>>>> markers
>>>>>>>>>>>>>>>>> to.
>>>>>>>>>>>>>>>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we added them to
>>>>>> the
>>>>>>>>>>>> coordinator
>>>>>>>>>>>>>>> via
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Previously we have
>>>>>>> had
>>>>>>>>>>>> asynchronous
>>>>>>>>>>>>>>>>> calls
>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> past
>>>>>>>>>>>>>>>>>>>>>>>>>> (ie,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit markers when
>>>>>>> the
>>>>>>>>>>>>>>> transaction is
>>>>>>>>>>>>>>>>>>>>> completed)
>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>> often
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes confusion as
>>>>>>> we
>>>>>>>>>> need to
>>>>>>>>>>>> wait
>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>> operations
>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complete.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing commit
>>>>>>> markers
>>>>>>>>>> case,
>>>>>>>>>>>>>>> clients
>>>>>>>>>>>>>>>>> often
>>>>>>>>>>>>>>>>>>>> see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>> CONCURRENT_TRANSACTIONs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error messages and
>>>>>>> that
>>>>>>>>>> can be
>>>>>>>>>>>>>>>>> confusing.
>>>>>>>>>>>>>>>>>>>> For
>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>> reason,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> may
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simpler to just
>>>>>> have
>>>>>>>>>>>> synchronous
>>>>>>>>>>>>>>>> calls —
>>>>>>>>>>>>>>>>>>>>>>> especially
>>>>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> block
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some operation's
>>>>>>>>> completion
>>>>>>>>>>>> anyway
>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>> start
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. And
>>>>>>> yes, I
>>>>>>>>>> meant
>>>>>>>>>>>>>>>>>> coordinator. I
>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>> fix
>>>>>>>>>>>>>>>>>>>>>>>>>>> that.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are
>>>>>> checking
>>>>>>> if
>>>>>>>>> the
>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>> ongoing,
>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the
>>>>>> leader
>>>>>>>>>> partition
>>>>>>>>>>>> to
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for
>>>>>>> this
>>>>>>>>>>>> message to
>>>>>>>>>>>>>>>> come
>>>>>>>>>>>>>>>>>>>> back,
>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>> theory
>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
>>>>>>>> that
>>>>>>>>>> would
>>>>>>>>>>>> make
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why
>>>>>> we
>>>>>>>> can
>>>>>>>>>> check
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> leader
>>>>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm happy to update
>>>>>>> the
>>>>>>>>>> KIP if
>>>>>>>>>>>>>>> some of
>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>>>>> things
>>>>>>>>>>>>>>>>>>>>>>>>> were
>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Nov 21,
>>>>>> 2022
>>>>>>> at
>>>>>>>>>> 7:11 PM
>>>>>>>>>>>>>>>> Matthias
>>>>>>>>>>>>>>>>>> J.
>>>>>>>>>>>>>>>>>>>>> Sax <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the
>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Couple of
>>>>>>>> clarification
>>>>>>>>>>>> questions
>>>>>>>>>>>>>>> (I
>>>>>>>>>>>>>>>> am
>>>>>>>>>>>>>>>>>>>> not a
>>>>>>>>>>>>>>>>>>>>>>>> broker
>>>>>>>>>>>>>>>>>>>>>>>>>>> expert
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some question are
>>>>>>>>> obvious
>>>>>>>>>> for
>>>>>>>>>>>>>>> others,
>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>> me
>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lack
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker knowledge).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (10)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What happens if
>>>>>> the
>>>>>>>>>> message
>>>>>>>>>>>> come
>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request? It seems
>>>>>>> the
>>>>>>>>>> broker
>>>>>>>>>>>>>>> hosting
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> data
>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anything about it
>>>>>>> and
>>>>>>>>>> append
>>>>>>>>>>>> it to
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> partition,
>>>>>>>>>>>>>>>>>>>>>>>>> too?
>>>>>>>>>>>>>>>>>>>>>>>>>>> What
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> difference between
>>>>>>>> both
>>>>>>>>>> cases?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, it seems a
>>>>>> TX
>>>>>>>>> would
>>>>>>>>>> only
>>>>>>>>>>>>>>> hang,
>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>>>>>> is no
>>>>>>>>>>>>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> either committer
>>>>>> or
>>>>>>>>>> aborted?
>>>>>>>>>>>> Thus,
>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>> above,
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> might
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually not hang
>>>>>>> (of
>>>>>>>>>> course,
>>>>>>>>>>>> we
>>>>>>>>>>>>>>>> might
>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>> EOS
>>>>>>>>>>>>>>>>>>>>>>>>>>> violation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX was aborted and
>>>>>>> the
>>>>>>>>>> second
>>>>>>>>>>>>>>>>> committed,
>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (20)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Of course, 1 and
>>>>>> 2
>>>>>>>>>> require
>>>>>>>>>>>>>>>> client-side
>>>>>>>>>>>>>>>>>>>>>>> changes, so
>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> those approaches
>>>>>>> won’t
>>>>>>>>>> apply.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For (1) I
>>>>>> understand
>>>>>>>>> why a
>>>>>>>>>>>> client
>>>>>>>>>>>>>>>>> change
>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>> necessary,
>>>>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we need a client
>>>>>>>> change
>>>>>>>>>> for
>>>>>>>>>>>> (2).
>>>>>>>>>>>>>>> Can
>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>> elaborate?
>>>>>>>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> explain
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that we should
>>>>>> send
>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactionRequest,
>>>>>>>>>>>>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>>>>>>>>>>>> am
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Can't we not just
>>>>>> do
>>>>>>>> an
>>>>>>>>>>>> implicit
>>>>>>>>>>>>>>>>>>>>>>> AddPartiitonToTx,
>>>>>>>>>>>>>>>>>>>>>>>>> too?
>>>>>>>>>>>>>>>>>>>>>>>>>>> If
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> old
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer correctly
>>>>>>>>>> registered
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>>>>>> already,
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can just ignore it
>>>>>>> as
>>>>>>>>>> it's an
>>>>>>>>>>>>>>>>> idempotent
>>>>>>>>>>>>>>>>>>>>>>> operation?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (30)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To cover older
>>>>>>>> clients,
>>>>>>>>>> we
>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>> ensure a
>>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ongoing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we write to a
>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Not sure what you
>>>>>>> mean
>>>>>>>>> by
>>>>>>>>>>>> this?
>>>>>>>>>>>>>>> Can
>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>> elaborate?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (40)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [the
>>>>>>> TX-coordinator]
>>>>>>>>> will
>>>>>>>>>>>> write
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> prepare
>>>>>>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch and send
>>>>>>>>>>>>>>> WriteTxnMarkerRequests
>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> bumped
>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why do we use the
>>>>>>>> bumped
>>>>>>>>>>>> epoch for
>>>>>>>>>>>>>>>>> both?
>>>>>>>>>>>>>>>>>> It
>>>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> intuitive
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the current epoch,
>>>>>>> and
>>>>>>>>>> only
>>>>>>>>>>>> return
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> bumped
>>>>>>>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (50) "Implicit
>>>>>>>>>>>>>>>>> AddPartitionToTransaction"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why does the
>>>>>>>> implicitly
>>>>>>>>>> sent
>>>>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>> synchronous?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also says
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in case we need
>>>>>> to
>>>>>>>>> abort
>>>>>>>>>> and
>>>>>>>>>>>>>>> need to
>>>>>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What do you mean
>>>>>> by
>>>>>>>>> this?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we don’t want to
>>>>>>>> write
>>>>>>>>>> to it
>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> manager
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Do you mean
>>>>>>>>> TX-coordinator
>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>>>> "manager"?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (60)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For older clients
>>>>>>> and
>>>>>>>>>> ensuring
>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> TX
>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>> ongoing,
>>>>>>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> describe a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> race condition. I
>>>>>> am
>>>>>>>> not
>>>>>>>>>> sure
>>>>>>>>>>>> if I
>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>> follow
>>>>>>>>>>>>>>>>>>>>>>> here.
>>>>>>>>>>>>>>>>>>>>>>>>> Can
>>>>>>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> elaborate?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/18/22 1:21
>>>>>> PM,
>>>>>>>>>> Justine
>>>>>>>>>>>>>>> Olshan
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey all!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to
>>>>>> start a
>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>> on my
>>>>>>>>>>>>>>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> add
>>>>>>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checks on
>>>>>>>> transactions
>>>>>>>>> to
>>>>>>>>>>>> avoid
>>>>>>>>>>>>>>>>> hanging
>>>>>>>>>>>>>>>>>>>>>>>>> transactions.
>>>>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an issue for some
>>>>>>>> time,
>>>>>>>>>> so I
>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>> hope
>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> helpful
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users of EOS.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The KIP includes
>>>>>>>>> changes
>>>>>>>>>> that
>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>> compatible
>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>> old
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> changes to
>>>>>> improve
>>>>>>>>>>>> performance
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> correctness
>>>>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Please take a
>>>>>> look
>>>>>>>> and
>>>>>>>>>> leave
>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>> comments
>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>> may
>>>>>>>>>>>>>>>>>>>>>>>>>> have!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> JIRA:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-14402
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
> 

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
Hey Matthias,
Let me put it this way, if a producer is checking if a transaction is
ongoing, then no writes to the partition from the producer will go through
until the transaction is confirmed ongoing.
From then, I think I can apply the writes in the order they came in. Does
that make sense?

Let me know if I'm missing something.
Justine

On Wed, Jan 25, 2023 at 4:57 PM Matthias J. Sax <mj...@apache.org> wrote:

> > would it build an offset map with just the latest timestamp for a key?
>
> Cannot remember the details without reading the KIP, but yes, something
> like this (I believe it actually needs to track both, offset and
> timestamp per key).
>
> > I wonder if ordering assumptions are baked in there, why not use
> offset-based compaction.
>
> The use case is a compacted topic that does contain out-of-order data.
> If you set key k1=v1 @ 5 offset 100 and later key1 = v0 @ 3 at offset
> 200 we want to cleanup v0 with higher offset because it's out-of-order
> based on time, but keep v1 what is the actual latest version of k1.
>
>
> > I was also not aware of this "guarantee" with regards to broker side
> time.
>
> As already said: I am not sure if it's a public contract, but based on
> my experience, people might reply on it as "implicit contract". -- Maybe
> somebody else knows if it's public or not, and if it would be ok to
> "break" it.
>
> > Let me know if you have any concerns here.
>
> My understanding is: While we cannot make an offset-order guarantee for
> interleaved writes of different producer, if the topic is configures
> with "append_time", we "guarantee" (cf. my comment above") timestamp
> order... If that's the case, it would be an issue if we break this
> "guarantee".
>
> I am not sure when the broker sets the timestamp for "append_time"
> config? If we do it before putting the request into purgatory, we have a
> problem. However, if we set the timestamp when we actually process the
> request and do the actual append, it seems there is no issue, as the
> request that was waiting in purgatory get the "newest" timestamp and
> thus cannot introduce out-of-order data.
>
>
> -Matthias
>
>
> On 1/24/23 10:44 AM, Justine Olshan wrote:
> > Hey Matthias,
> >
> > I have actually never heard of KIP-280 so thanks for bringing it up. That
> > seems interesting. I wonder how it would work though -- would it build an
> > offset map with just the latest timestamp for a key? I wonder if ordering
> > assumptions are baked in there, why not use offset-based compaction.
> >
> > I was also not aware of this "guarantee" with regards to broker side
> time.
> > I think that we can do in order handling for a given producer, but not
> > across all producers. However, we can't guarantee that anyway.
> >
> > Let me know if you have any concerns here.
> >
> > Thanks,
> > Justine
> >
> > On Mon, Jan 23, 2023 at 6:33 PM Matthias J. Sax <mj...@apache.org>
> wrote:
> >
> >> Just a side note about Guozhang comments about timestamps.
> >>
> >> If the producer sets the timestamp, putting the record into purgatory
> >> seems not to be an issue (as already said: for this case we don't
> >> guarantee timestamp order between writes of different producers anyway).
> >> However, if the broker sets the timestamp, the expectation is that there
> >> is no out-of-order data in the partition ever; if we would introduce
> >> out-of-order data for this case (for interleaved writes of different
> >> producers), it seems we would violate the current contract? (To be fair:
> >> I don't know if that's an official contract, but I assume people rely on
> >> this behavior -- and it "advertised" in many public talks...)
> >>
> >> About compaction: there is actually KIP-280 that adds timestamp based
> >> compaction what is a very useful feature for Kafka Streams with regard
> >> to out-of-order data handling. So the impact if we introduce
> >> out-of-order data could be larger scoped.
> >>
> >>
> >> -Matthias
> >>
> >>
> >> On 1/20/23 4:48 PM, Justine Olshan wrote:
> >>> Hey Artem,
> >>>
> >>> I see there is a check for transactional producers. I'm wondering if we
> >>> don't handle the epoch overflow case. I'm also not sure it will be a
> huge
> >>> issue to extend to transactional producers, but maybe I'm missing
> >> something.
> >>>
> >>> As for the recovery path -- I think Guozhang's point was if we have a
> bad
> >>> client that repeatedly tries to produce without adding to the
> transaction
> >>> we would do the following:
> >>> a) if not fatal, we just fail the produce request over and over
> >>> b) if fatal, we fence the producer
> >>>
> >>> Here with B, the issue with the client would be made clear more
> quickly.
> >> I
> >>> suppose there are some intermediate cases where the issue only occurs
> >>> sometimes, but I wonder if we should consider how to recover with
> clients
> >>> who don't behave as expected anyway.
> >>>
> >>> I think there is a place for the abortable error that we are adding --
> >> just
> >>> abort and try again. But I think there are also some cases where trying
> >> to
> >>> recover overcomplicates some logic. Especially if we are considering
> >> older
> >>> clients -- there I'm not sure if there's a ton we can do besides fail
> the
> >>> batch or fence the producer. With newer clients, we can consider more
> >>> options for what can just be recovered after aborting. But epochs might
> >> be
> >>> a hard one unless we also want to reset producer ID.
> >>>
> >>> Thanks,
> >>> Justine
> >>>
> >>>
> >>>
> >>> On Fri, Jan 20, 2023 at 3:59 PM Artem Livshits
> >>> <al...@confluent.io.invalid> wrote:
> >>>
> >>>>>    besides the poorly written client case
> >>>>
> >>>> A poorly written client could create a lot of grief to people who run
> >> Kafka
> >>>> brokers :-), so when deciding to make an error fatal I would see if
> >> there
> >>>> is a reasonable recovery path rather than how often it could happen.
> >> If we
> >>>> have solid implementation of transactions (which I hope we'll do as a
> >>>> result of this KIP), it would help to recover from a large class of
> >> errors
> >>>> by just aborting a transaction, even if the cause of error is a race
> >>>> condition or etc.
> >>>>
> >>>> -Artem
> >>>>
> >>>> On Fri, Jan 20, 2023 at 3:26 PM Justine Olshan
> >>>> <jo...@confluent.io.invalid>
> >>>> wrote:
> >>>>
> >>>>> Artem --
> >>>>> I guess the discussion path we were going down is when we expect to
> see
> >>>>> this error. I mentioned that it was hard to come up with cases for
> when
> >>>> the
> >>>>> producer would still be around to receive the error besides the
> poorly
> >>>>> written client case.
> >>>>> If we don't expect to have a producer to receive the response, it
> sort
> >> of
> >>>>> makes sense for it to be fatal.
> >>>>>
> >>>>> I had some discussion with Jason offline about the epoch being off
> >> cases
> >>>>> and I'm not sure we could find a ton (outside of produce requests)
> >> where
> >>>> we
> >>>>> could/should recover. I'd be happy to hear some examples though,
> maybe
> >>>> I'm
> >>>>> missing something.
> >>>>>
> >>>>> Thanks,
> >>>>> Justine
> >>>>>
> >>>>> On Fri, Jan 20, 2023 at 3:19 PM Artem Livshits
> >>>>> <al...@confluent.io.invalid> wrote:
> >>>>>
> >>>>>> In general, I'd like to avoid fatal errors as much as possible, in
> >> some
> >>>>>> sense fatal errors just push out recovery logic to the application
> >>>> which
> >>>>>> either complicates the application or leads to disruption (we've
> seen
> >>>>> cases
> >>>>>> when a transient broker error could lead to work stoppage when
> >>>>> applications
> >>>>>> need to be manually restarted).  I think we should strive to define
> >>>>>> recovery logic for most errors (and/or encapsulate it in the Kafka
> >>>> client
> >>>>>> as much as possible).
> >>>>>>
> >>>>>> One benefit of transactions is that they simplify recovery from
> >> errors,
> >>>>>> pretty much any error (that's not handled transparently by retries
> in
> >>>>> Kafka
> >>>>>> client) can be handled by the application via aborting the
> transaction
> >>>>> and
> >>>>>> repeating the transactional logic again.  One tricky error is an
> error
> >>>>>> during commit, because we don't know the outcome.  For commit
> errors,
> >>>> the
> >>>>>> recommendation should be to retry the commit until it returns the
> >>>>> specific
> >>>>>> result (committed or aborted).
> >>>>>>
> >>>>>> -Artem
> >>>>>>
> >>>>>> On Fri, Jan 20, 2023 at 2:52 PM Justine Olshan
> >>>>>> <jo...@confluent.io.invalid>
> >>>>>> wrote:
> >>>>>>
> >>>>>>> That's a fair point about other clients.
> >>>>>>>
> >>>>>>> I think the abortable error case is interesting because I'm curious
> >>>> how
> >>>>>>> other clients would handle this. I assume they would need to
> >>>> implement
> >>>>>>> handling for the error code unless they did something like "any
> >>>> unknown
> >>>>>>> error codes/any codes that aren't x,y,z are retriable." I would
> hope
> >>>>> that
> >>>>>>> unknown error codes were fatal, and if the code was implemented it
> >>>>> would
> >>>>>>> abort the transaction. But I will think on this too.
> >>>>>>>
> >>>>>>> As for InvalidRecord -- you mentioned it was not fatal, but I'm
> >>>> taking
> >>>>> a
> >>>>>>> look through the code. We would see this on handling the produce
> >>>>>> response.
> >>>>>>> If I recall correctly, we check if errors are retriable. I think
> this
> >>>>>> error
> >>>>>>> would not be retriable. But I guess the concern here is that it is
> >>>> not
> >>>>>>> enough for just that batch to fail. I guess I hadn't considered
> fully
> >>>>>>> fencing the old producer but there are valid arguments here why we
> >>>>> would
> >>>>>>> want to.
> >>>>>>>
> >>>>>>> Thanks,
> >>>>>>> Justine
> >>>>>>>
> >>>>>>> On Fri, Jan 20, 2023 at 2:35 PM Guozhang Wang <
> >>>>>> guozhang.wang.us@gmail.com>
> >>>>>>> wrote:
> >>>>>>>
> >>>>>>>> Thanks Justine for the replies! I agree with most of your
> thoughts.
> >>>>>>>>
> >>>>>>>> Just for 3/7), though I agree for our own AK producer, since we do
> >>>>>>>> "nextRequest(boolean hasIncompleteBatches)", we guarantee the
> >>>> end-txn
> >>>>>>>> would not be sent until we've effectively flushed, but I was
> >>>>> referring
> >>>>>>>> to any future bugs or other buggy clients that the same client may
> >>>>> get
> >>>>>>>> into this situation, in which case we should give the client a
> >>>> clear
> >>>>>>>> msg that "you did something wrong, and hence now you should
> fatally
> >>>>>>>> close yourself". What I'm concerned about is that, by seeing an
> >>>>>>>> "abortable error" or in some rare cases an "invalid record", the
> >>>>>>>> client could not realize "something that's really bad happened".
> So
> >>>>>>>> it's not about adding a new error, it's mainly about those real
> >>>> buggy
> >>>>>>>> situations causing such "should never happen" cases, the errors
> >>>>> return
> >>>>>>>> would not be informative enough.
> >>>>>>>>
> >>>>>>>> Thinking in other ways, if we believe that for most cases such
> >>>> error
> >>>>>>>> codes would not reach the original clients since they would be
> >>>>>>>> disconnected or even gone by that time, and only in some rare
> cases
> >>>>>>>> they would still be seen by the sending clients, then why not make
> >>>>>>>> them more fatal and more specific than generic.
> >>>>>>>>
> >>>>>>>> Guozhang
> >>>>>>>>
> >>>>>>>> On Fri, Jan 20, 2023 at 1:59 PM Justine Olshan
> >>>>>>>> <jo...@confluent.io.invalid> wrote:
> >>>>>>>>>
> >>>>>>>>> Hey Guozhang. Thanks for taking a look and for the detailed
> >>>>> comments!
> >>>>>>>> I'll
> >>>>>>>>> do my best to address below.
> >>>>>>>>>
> >>>>>>>>> 1. I see what you are saying here, but I think I need to look
> >>>>> through
> >>>>>>> the
> >>>>>>>>> sequence of events you mention. Typically we've seen this issue
> >>>> in
> >>>>> a
> >>>>>>> few
> >>>>>>>>> cases.
> >>>>>>>>>
> >>>>>>>>>    One is when we have a producer disconnect when trying to
> >>>> produce.
> >>>>>>>>> Typically in these cases, we abort the transaction. We've seen
> >>>> that
> >>>>>>> after
> >>>>>>>>> the markers are written, the disconnection can sometimes cause
> >>>> the
> >>>>>>>> request
> >>>>>>>>> to get flushed to the broker. In this case, we don't need client
> >>>>>>> handling
> >>>>>>>>> because the producer we are responding to is gone. We just needed
> >>>>> to
> >>>>>>> make
> >>>>>>>>> sure we didn't write to the log on the broker side. I'm trying to
> >>>>>> think
> >>>>>>>> of
> >>>>>>>>> a case where we do have the client to return to. I'd think the
> >>>> same
> >>>>>>>> client
> >>>>>>>>> couldn't progress to committing the transaction unless the
> >>>> produce
> >>>>>>>> request
> >>>>>>>>> returned right? Of course, there is the incorrectly written
> >>>> clients
> >>>>>>> case.
> >>>>>>>>> I'll think on this a bit more and let you know if I come up with
> >>>>>>> another
> >>>>>>>>> scenario when we would return to an active client when the
> >>>>>> transaction
> >>>>>>> is
> >>>>>>>>> no longer ongoing.
> >>>>>>>>>
> >>>>>>>>> I was not aware that we checked the result of a send after we
> >>>>> commit
> >>>>>>>>> though. I'll need to look into that a bit more.
> >>>>>>>>>
> >>>>>>>>> 2. There were some questions about this in the discussion. The
> >>>> plan
> >>>>>> is
> >>>>>>> to
> >>>>>>>>> handle overflow with the mechanism we currently have in the
> >>>>> producer.
> >>>>>>> If
> >>>>>>>> we
> >>>>>>>>> try to bump and the epoch will overflow, we actually allocate a
> >>>> new
> >>>>>>>>> producer ID. I need to confirm the fencing logic on the last
> >>>> epoch
> >>>>>> (ie,
> >>>>>>>> we
> >>>>>>>>> probably shouldn't allow any records to be produced with the
> >>>> final
> >>>>>>> epoch
> >>>>>>>>> since we can never properly fence that one).
> >>>>>>>>>
> >>>>>>>>> 3. I can agree with you that the current error handling is
> >>>> messy. I
> >>>>>>>> recall
> >>>>>>>>> taking a look at your KIP a while back, but I think I mostly saw
> >>>>> the
> >>>>>>>>> section about how the errors were wrapped. Maybe I need to take
> >>>>>> another
> >>>>>>>>> look. As for abortable error, the idea was that the handling
> >>>> would
> >>>>> be
> >>>>>>>>> simple -- if this error is seen, the transaction should be
> >>>> aborted
> >>>>> --
> >>>>>>> no
> >>>>>>>>> other logic about previous state or requests necessary. Is your
> >>>>>> concern
> >>>>>>>>> simply about adding new errors? We were hoping to have an error
> >>>>> that
> >>>>>>>> would
> >>>>>>>>> have one meaning and many of the current errors have a history of
> >>>>>>> meaning
> >>>>>>>>> different things on different client versions. That was the main
> >>>>>>>> motivation
> >>>>>>>>> for adding a new error.
> >>>>>>>>>
> >>>>>>>>> 4. This is a good point about record timestamp reordering.
> >>>>> Timestamps
> >>>>>>>> don't
> >>>>>>>>> affect compaction, but they do affect retention deletion. For
> >>>> that,
> >>>>>>> kafka
> >>>>>>>>> considers the largest timestamp in the segment, so I think a
> >>>> small
> >>>>>>> amount
> >>>>>>>>> of reordering (hopefully on the order of milliseconds or even
> >>>>>> seconds)
> >>>>>>>> will
> >>>>>>>>> be ok. We take timestamps from clients so there is already a
> >>>>>>> possibility
> >>>>>>>>> for some drift and non-monotonically increasing timestamps.
> >>>>>>>>>
> >>>>>>>>> 5. Thanks for catching. The error is there, but it's actually
> >>>> that
> >>>>>>> those
> >>>>>>>>> fields should be 4+! Due to how the message generator works, I
> >>>>>> actually
> >>>>>>>>> have to redefine those fields inside the
> >>>>>>> `"AddPartitionsToTxnTransaction`
> >>>>>>>>> block for it to build correctly. I'll fix it to be correct.
> >>>>>>>>>
> >>>>>>>>> 6. Correct -- we will only add the request to purgatory if the
> >>>>> cache
> >>>>>>> has
> >>>>>>>> no
> >>>>>>>>> ongoing transaction. I can change the wording to make that
> >>>> clearer
> >>>>>> that
> >>>>>>>> we
> >>>>>>>>> only place the request in purgatory if we need to contact the
> >>>>>>> transaction
> >>>>>>>>> coordinator.
> >>>>>>>>>
> >>>>>>>>> 7. We did take a look at some of the errors and it was hard to
> >>>> come
> >>>>>> up
> >>>>>>>> with
> >>>>>>>>> a good one. I agree that InvalidTxnStateException is ideal except
> >>>>> for
> >>>>>>> the
> >>>>>>>>> fact that it hasn't been returned on Produce requests before. The
> >>>>>> error
> >>>>>>>>> handling for clients is a bit vague (which is why I opened
> >>>>>> KAFKA-14439
> >>>>>>>>> <https://issues.apache.org/jira/browse/KAFKA-14439>), but the
> >>>>>> decision
> >>>>>>>> we
> >>>>>>>>> made here was to only return errors that have been previously
> >>>>>> returned
> >>>>>>> to
> >>>>>>>>> producers. As for not being fatal, I think part of the theory was
> >>>>>> that
> >>>>>>> in
> >>>>>>>>> many cases, the producer would be disconnected. (See point 1) and
> >>>>>> this
> >>>>>>>>> would just be an error to return from the server. I did plan to
> >>>>> think
> >>>>>>>> about
> >>>>>>>>> other cases, so let me know if you think of any as well!
> >>>>>>>>>
> >>>>>>>>> Lots to say! Let me know if you have further thoughts!
> >>>>>>>>> Justine
> >>>>>>>>>
> >>>>>>>>> On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang <
> >>>>>>>> guozhang.wang.us@gmail.com>
> >>>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Hello Justine,
> >>>>>>>>>>
> >>>>>>>>>> Thanks for the great write-up! I made a quick pass through it
> >>>> and
> >>>>>>> here
> >>>>>>>>>> are some thoughts (I have not been able to read through this
> >>>>> thread
> >>>>>>> so
> >>>>>>>>>> pardon me if they have overlapped or subsumed by previous
> >>>>>> comments):
> >>>>>>>>>>
> >>>>>>>>>> First are some meta ones:
> >>>>>>>>>>
> >>>>>>>>>> 1. I think we need to also improve the client's experience once
> >>>>> we
> >>>>>>>>>> have this defence in place. More concretely, say a user's
> >>>>> producer
> >>>>>>>>>> code is like following:
> >>>>>>>>>>
> >>>>>>>>>> future = producer.send();
> >>>>>>>>>> // producer.flush();
> >>>>>>>>>> producer.commitTransaction();
> >>>>>>>>>> future.get();
> >>>>>>>>>>
> >>>>>>>>>> Which resulted in the order of a) produce-request sent by
> >>>>> producer,
> >>>>>>> b)
> >>>>>>>>>> end-txn-request sent by producer, c) end-txn-response sent
> >>>> back,
> >>>>> d)
> >>>>>>>>>> txn-marker-request sent from coordinator to partition leader,
> >>>> e)
> >>>>>>>>>> produce-request finally received by the partition leader,
> >>>> before
> >>>>>> this
> >>>>>>>>>> KIP e) step would be accepted causing a dangling txn; now it
> >>>>> would
> >>>>>> be
> >>>>>>>>>> rejected in step e) which is good. But from the client's point
> >>>> of
> >>>>>>> view
> >>>>>>>>>> now it becomes confusing since the `commitTransaction()`
> >>>> returns
> >>>>>>>>>> successfully, but the "future" throws an invalid-epoch error,
> >>>> and
> >>>>>>> they
> >>>>>>>>>> are not sure if the transaction did succeed or not. In fact, it
> >>>>>>>>>> "partially succeeded" with some msgs being rejected but others
> >>>>>>>>>> committed successfully.
> >>>>>>>>>>
> >>>>>>>>>> Of course the easy way to avoid this is, always call
> >>>>>>>>>> "producer.flush()" before commitTxn and that's what we do
> >>>>>> ourselves,
> >>>>>>>>>> and what we recommend users do. But I suspect not everyone does
> >>>>> it.
> >>>>>>> In
> >>>>>>>>>> fact I just checked the javadoc in KafkaProducer and our code
> >>>>>> snippet
> >>>>>>>>>> does not include a `flush()` call. So I'm thinking maybe we can
> >>>>> in
> >>>>>>>>>> side the `commitTxn` code to enforce flushing before sending
> >>>> the
> >>>>>>>>>> end-txn request.
> >>>>>>>>>>
> >>>>>>>>>> 2. I'd like to clarify a bit details on "just add partitions to
> >>>>> the
> >>>>>>>>>> transaction on the first produce request during a transaction".
> >>>>> My
> >>>>>>>>>> understanding is that the partition leader's cache has the
> >>>>> producer
> >>>>>>> id
> >>>>>>>>>> / sequence / epoch for the latest txn, either on-going or is
> >>>>>>> completed
> >>>>>>>>>> (upon receiving the marker request from coordinator). When a
> >>>>>> produce
> >>>>>>>>>> request is received, if
> >>>>>>>>>>
> >>>>>>>>>> * producer's epoch < cached epoch, or producer's epoch ==
> >>>> cached
> >>>>>>> epoch
> >>>>>>>>>> but the latest txn is completed, leader directly reject with
> >>>>>>>>>> invalid-epoch.
> >>>>>>>>>> * producer's epoch > cached epoch, park the the request and
> >>>> send
> >>>>>>>>>> add-partitions request to coordinator.
> >>>>>>>>>>
> >>>>>>>>>> In order to do it, does the coordinator need to bump the
> >>>> sequence
> >>>>>> and
> >>>>>>>>>> reset epoch to 0 when the next epoch is going to overflow? If
> >>>> no
> >>>>>> need
> >>>>>>>>>> to do so, then how we handle the (admittedly rare, but still
> >>>> may
> >>>>>>>>>> happen) epoch overflow situation?
> >>>>>>>>>>
> >>>>>>>>>> 3. I'm a bit concerned about adding a generic "ABORTABLE_ERROR"
> >>>>>> given
> >>>>>>>>>> we already have a pretty messy error classification and error
> >>>>>>> handling
> >>>>>>>>>> on the producer clients side --- I have a summary about the
> >>>>> issues
> >>>>>>> and
> >>>>>>>>>> a proposal to address this in
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
> >>>>>>>>>> -- I understand we do not want to use "UNKNOWN_PRODUCER_ID"
> >>>>> anymore
> >>>>>>>>>> and in fact we intend to deprecate it in KIP-360 and eventually
> >>>>>>> remove
> >>>>>>>>>> it; but I'm wondering can we still use specific error codes.
> >>>> E.g.
> >>>>>>> what
> >>>>>>>>>> about "InvalidProducerEpochException" since for new clients,
> >>>> the
> >>>>>>>>>> actual reason this would actually be rejected is indeed because
> >>>>> the
> >>>>>>>>>> epoch on the coordinator caused the add-partitions-request from
> >>>>> the
> >>>>>>>>>> brokers to be rejected anyways?
> >>>>>>>>>>
> >>>>>>>>>> 4. It seems we put the producer request into purgatory before
> >>>> we
> >>>>>> ever
> >>>>>>>>>> append the records, while other producer's records may still be
> >>>>>>>>>> appended during the time; and that potentially may result in
> >>>> some
> >>>>>>>>>> re-ordering compared with reception order. I'm not super
> >>>>> concerned
> >>>>>>>>>> about it since Kafka does not guarantee reception ordering
> >>>> across
> >>>>>>>>>> producers anyways, but it may make the timestamps of records
> >>>>>> inside a
> >>>>>>>>>> partition to be more out-of-ordered. Are we aware of any
> >>>>> scenarios
> >>>>>>>>>> such as future enhancements on log compactions that may be
> >>>>> affected
> >>>>>>> by
> >>>>>>>>>> this effect?
> >>>>>>>>>>
> >>>>>>>>>> Below are just minor comments:
> >>>>>>>>>>
> >>>>>>>>>> 5. In "AddPartitionsToTxnTransaction" field of
> >>>>>>>>>> "AddPartitionsToTxnRequest" RPC, the versions of those inner
> >>>>> fields
> >>>>>>>>>> are "0-3" while I thought they should be "0+" still?
> >>>>>>>>>>
> >>>>>>>>>> 6. Regarding "we can place the request in a purgatory of sorts
> >>>>> and
> >>>>>>>>>> check if there is any state for the transaction on the
> >>>> broker": i
> >>>>>>>>>> think at this time when we just do the checks against the
> >>>> cached
> >>>>>>>>>> state, we do not need to put the request to purgatory yet?
> >>>>>>>>>>
> >>>>>>>>>> 7. This is related to 3) above. I feel using
> >>>>>> "InvalidRecordException"
> >>>>>>>>>> for older clients may also be a bit confusing, and also it is
> >>>> not
> >>>>>>>>>> fatal -- for old clients, it better to be fatal since this
> >>>>>> indicates
> >>>>>>>>>> the clients is doing something wrong and hence it should be
> >>>>> closed.
> >>>>>>>>>> And in general I'd prefer to use slightly more specific meaning
> >>>>>> error
> >>>>>>>>>> codes for clients. That being said, I also feel
> >>>>>>>>>> "InvalidProducerEpochException" is not suitable for old
> >>>> versioned
> >>>>>>>>>> clients, and we'd have to pick one that old clients recognize.
> >>>>> I'd
> >>>>>>>>>> prefer "InvalidTxnStateException" but that one is supposed to
> >>>> be
> >>>>>>>>>> returned from txn coordinators only today. I'd suggest we do a
> >>>>>> quick
> >>>>>>>>>> check in the current client's code path and see if that one
> >>>> would
> >>>>>> be
> >>>>>>>>>> handled if it's from a produce-response, and if yes, use this
> >>>>> one;
> >>>>>>>>>> otherwise, use "ProducerFencedException" which is much less
> >>>>>>> meaningful
> >>>>>>>>>> but it's still a fatal error.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Thanks,
> >>>>>>>>>> Guozhang
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
> >>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>> Yeah -- looks like we already have code to handle bumping the
> >>>>>> epoch
> >>>>>>>> and
> >>>>>>>>>>> when the epoch is Short.MAX_VALUE, we get a new producer ID.
> >>>>>> Since
> >>>>>>>> this
> >>>>>>>>>> is
> >>>>>>>>>>> already the behavior, do we want to change it further?
> >>>>>>>>>>>
> >>>>>>>>>>> Justine
> >>>>>>>>>>>
> >>>>>>>>>>> On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <
> >>>>>>> jolshan@confluent.io
> >>>>>>>>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> Hey all, just wanted to quickly update and say I've
> >>>> modified
> >>>>>> the
> >>>>>>>> KIP to
> >>>>>>>>>>>> explicitly mention that AddOffsetCommitsToTxnRequest will
> >>>> be
> >>>>>>>> replaced
> >>>>>>>>>> by
> >>>>>>>>>>>> a coordinator-side (inter-broker) AddPartitionsToTxn
> >>>> implicit
> >>>>>>>> request.
> >>>>>>>>>> This
> >>>>>>>>>>>> mirrors the user partitions and will implicitly add offset
> >>>>>>>> partitions
> >>>>>>>>>> to
> >>>>>>>>>>>> transactions when we commit offsets on them. We will
> >>>>> deprecate
> >>>>>>>>>> AddOffsetCommitsToTxnRequest
> >>>>>>>>>>>> for new clients.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Also to address Artem's comments --
> >>>>>>>>>>>> I'm a bit unsure if the changes here will change the
> >>>> previous
> >>>>>>>> behavior
> >>>>>>>>>> for
> >>>>>>>>>>>> fencing producers. In the case you mention in the first
> >>>>>>> paragraph,
> >>>>>>>> are
> >>>>>>>>>> you
> >>>>>>>>>>>> saying we bump the epoch before we try to abort the
> >>>>>> transaction?
> >>>>>>> I
> >>>>>>>>>> think I
> >>>>>>>>>>>> need to understand the scenarios you mention a bit better.
> >>>>>>>>>>>>
> >>>>>>>>>>>> As for the second part -- I think it makes sense to have
> >>>> some
> >>>>>>> sort
> >>>>>>>> of
> >>>>>>>>>>>> "sentinel" epoch to signal epoch is about to overflow (I
> >>>>> think
> >>>>>> we
> >>>>>>>> sort
> >>>>>>>>>> of
> >>>>>>>>>>>> have this value in place in some ways) so we can codify it
> >>>> in
> >>>>>> the
> >>>>>>>> KIP.
> >>>>>>>>>> I'll
> >>>>>>>>>>>> look into that and try to update soon.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thanks,
> >>>>>>>>>>>> Justine.
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
> >>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> It's good to know that KIP-588 addressed some of the
> >>>> issues.
> >>>>>>>> Looking
> >>>>>>>>>> at
> >>>>>>>>>>>>> the code, it still looks like there are some cases that
> >>>>> would
> >>>>>>>> result
> >>>>>>>>>> in
> >>>>>>>>>>>>> fatal error, e.g. PRODUCER_FENCED is issued by the
> >>>>> transaction
> >>>>>>>>>> coordinator
> >>>>>>>>>>>>> if epoch doesn't match, and the client treats it as a
> >>>> fatal
> >>>>>>> error
> >>>>>>>>>> (code in
> >>>>>>>>>>>>> TransactionManager request handling).  If we consider, for
> >>>>>>>> example,
> >>>>>>>>>>>>> committing a transaction that returns a timeout, but
> >>>>> actually
> >>>>>>>>>> succeeds,
> >>>>>>>>>>>>> trying to abort it or re-commit may result in
> >>>>> PRODUCER_FENCED
> >>>>>>>> error
> >>>>>>>>>>>>> (because of epoch bump).
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> For failed commits, specifically, we need to know the
> >>>> actual
> >>>>>>>> outcome,
> >>>>>>>>>>>>> because if we return an error the application may think
> >>>> that
> >>>>>> the
> >>>>>>>>>>>>> transaction is aborted and redo the work, leading to
> >>>>>> duplicates.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Re: overflowing epoch.  We could either do it on the TC
> >>>> and
> >>>>>>> return
> >>>>>>>>>> both
> >>>>>>>>>>>>> producer id and epoch (e.g. change the protocol), or
> >>>> signal
> >>>>>> the
> >>>>>>>> client
> >>>>>>>>>>>>> that
> >>>>>>>>>>>>> it needs to get a new producer id.  Checking for max epoch
> >>>>>> could
> >>>>>>>> be a
> >>>>>>>>>>>>> reasonable signal, the value to check should probably be
> >>>>>> present
> >>>>>>>> in
> >>>>>>>>>> the
> >>>>>>>>>>>>> KIP
> >>>>>>>>>>>>> as this is effectively a part of the contract.  Also, the
> >>>> TC
> >>>>>>>> should
> >>>>>>>>>>>>> probably return an error if the client didn't change
> >>>>> producer
> >>>>>> id
> >>>>>>>> after
> >>>>>>>>>>>>> hitting max epoch.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> -Artem
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
> >>>>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thanks for the discussion Artem.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> With respect to the handling of fenced producers, we
> >>>> have
> >>>>>> some
> >>>>>>>>>> behavior
> >>>>>>>>>>>>>> already in place. As of KIP-588:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> >>>>>>>>>>>>>> ,
> >>>>>>>>>>>>>> we handle timeouts more gracefully. The producer can
> >>>>>> recover.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Produce requests can also recover from epoch fencing by
> >>>>>>>> aborting the
> >>>>>>>>>>>>>> transaction and starting over.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> What other cases were you considering that would cause
> >>>> us
> >>>>> to
> >>>>>>>> have a
> >>>>>>>>>>>>> fenced
> >>>>>>>>>>>>>> epoch but we'd want to recover?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> The first point about handling epoch overflows is fair.
> >>>> I
> >>>>>>> think
> >>>>>>>>>> there is
> >>>>>>>>>>>>>> some logic we'd need to consider. (ie, if we are one
> >>>> away
> >>>>>> from
> >>>>>>>> the
> >>>>>>>>>> max
> >>>>>>>>>>>>>> epoch, we need to reset the producer ID.) I'm still
> >>>>>> wondering
> >>>>>>> if
> >>>>>>>>>> there
> >>>>>>>>>>>>> is a
> >>>>>>>>>>>>>> way to direct this from the response, or if everything
> >>>>>> should
> >>>>>>> be
> >>>>>>>>>> done on
> >>>>>>>>>>>>>> the client side. Let me know if you have any thoughts
> >>>>> here.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
> >>>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> There are some workflows in the client that are
> >>>> implied
> >>>>> by
> >>>>>>>>>> protocol
> >>>>>>>>>>>>>>> changes, e.g.:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> - for new clients, epoch changes with every
> >>>> transaction
> >>>>>> and
> >>>>>>>> can
> >>>>>>>>>>>>> overflow,
> >>>>>>>>>>>>>>> in old clients this condition was handled
> >>>> transparently,
> >>>>>>>> because
> >>>>>>>>>> epoch
> >>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>> bumped in InitProducerId and it would return a new
> >>>>>> producer
> >>>>>>>> id if
> >>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>> overflows, the new clients would need to implement
> >>>> some
> >>>>>>>> workflow
> >>>>>>>>>> to
> >>>>>>>>>>>>>> refresh
> >>>>>>>>>>>>>>> producer id
> >>>>>>>>>>>>>>> - how to handle fenced producers, for new clients
> >>>> epoch
> >>>>>>>> changes
> >>>>>>>>>> with
> >>>>>>>>>>>>>> every
> >>>>>>>>>>>>>>> transaction, so in presence of failures during
> >>>> commits /
> >>>>>>>> aborts,
> >>>>>>>>>> the
> >>>>>>>>>>>>>>> producer could get easily fenced, old clients would
> >>>>> pretty
> >>>>>>>> much
> >>>>>>>>>> would
> >>>>>>>>>>>>> get
> >>>>>>>>>>>>>>> fenced when a new incarnation of the producer was
> >>>>>>> initialized
> >>>>>>>> with
> >>>>>>>>>>>>>>> InitProducerId so it's ok to treat as a fatal error,
> >>>> the
> >>>>>> new
> >>>>>>>>>> clients
> >>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>> need to implement some workflow to handle that error,
> >>>>>>>> otherwise
> >>>>>>>>>> they
> >>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>> get fenced by themselves
> >>>>>>>>>>>>>>> - in particular (as a subset of the previous issue),
> >>>>> what
> >>>>>>>> would
> >>>>>>>>>> the
> >>>>>>>>>>>>>> client
> >>>>>>>>>>>>>>> do if it got a timeout during commit?  commit could've
> >>>>>>>> succeeded
> >>>>>>>>>> or
> >>>>>>>>>>>>>> failed
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Not sure if this has to be defined in the KIP as
> >>>>>>> implementing
> >>>>>>>>>> those
> >>>>>>>>>>>>>>> probably wouldn't require protocol changes, but we
> >>>> have
> >>>>>>>> multiple
> >>>>>>>>>>>>>>> implementations of Kafka clients, so probably would be
> >>>>>> good
> >>>>>>> to
> >>>>>>>>>> have
> >>>>>>>>>>>>> some
> >>>>>>>>>>>>>>> client implementation guidance.  Could also be done
> >>>> as a
> >>>>>>>> separate
> >>>>>>>>>> doc.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> -Artem
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
> >>>>>>>>>>>>>> <jolshan@confluent.io.invalid
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Hey all, I've updated the KIP to incorporate Jason's
> >>>>>>>>>> suggestions.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> 1. Use AddPartitionsToTxn + verify flag to check on
> >>>>> old
> >>>>>>>> clients
> >>>>>>>>>>>>>>>> 2. Updated AddPartitionsToTxn API to support
> >>>>> transaction
> >>>>>>>>>> batching
> >>>>>>>>>>>>>>>> 3. Mention IBP bump
> >>>>>>>>>>>>>>>> 4. Mention auth change on new AddPartitionsToTxn
> >>>>>> version.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I'm planning on opening a vote soon.
> >>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <
> >>>>>>>>>> jolshan@confluent.io
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thanks Jason. Those changes make sense to me. I
> >>>> will
> >>>>>>>> update
> >>>>>>>>>> the
> >>>>>>>>>>>>> KIP.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
> >>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Hey Justine,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> I was wondering about compatibility here. When
> >>>> we
> >>>>>>> send
> >>>>>>>>>> requests
> >>>>>>>>>>>>>>>>>> between brokers, we want to ensure that the
> >>>>> receiving
> >>>>>>>> broker
> >>>>>>>>>>>>>>> understands
> >>>>>>>>>>>>>>>>>> the request (specifically the new fields).
> >>>>> Typically
> >>>>>>>> this is
> >>>>>>>>>> done
> >>>>>>>>>>>>>> via
> >>>>>>>>>>>>>>>>>> IBP/metadata version.
> >>>>>>>>>>>>>>>>>> I'm trying to think if there is a way around it
> >>>> but
> >>>>>> I'm
> >>>>>>>> not
> >>>>>>>>>> sure
> >>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>> is.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Yes. I think we would gate usage of this behind
> >>>> an
> >>>>>> IBP
> >>>>>>>> bump.
> >>>>>>>>>> Does
> >>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>> seem
> >>>>>>>>>>>>>>>>>> reasonable?
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> As for the improvements -- can you clarify how
> >>>>> the
> >>>>>>>> multiple
> >>>>>>>>>>>>>>>>>> transactional
> >>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking of a case
> >>>>>> where
> >>>>>>> we
> >>>>>>>>>>>>> wait/batch
> >>>>>>>>>>>>>>>>>> multiple produce requests together? My
> >>>>> understanding
> >>>>>>> for
> >>>>>>>> now
> >>>>>>>>>> was
> >>>>>>>>>>>>> 1
> >>>>>>>>>>>>>>>>>> transactional ID and one validation per 1 produce
> >>>>>>>> request.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Each call to `AddPartitionsToTxn` is essentially
> >>>> a
> >>>>>>> write
> >>>>>>>> to
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>> log and must block on replication. The more we
> >>>> can
> >>>>>> fit
> >>>>>>>> into a
> >>>>>>>>>>>>> single
> >>>>>>>>>>>>>>>>>> request, the more writes we can do in parallel.
> >>>> The
> >>>>>>>>>> alternative
> >>>>>>>>>>>>> is
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>> use of more connections, but usually we prefer
> >>>>>> batching
> >>>>>>>>>> since the
> >>>>>>>>>>>>>>>> network
> >>>>>>>>>>>>>>>>>> stack is not really optimized for high
> >>>>>>> connection/request
> >>>>>>>>>> loads.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Finally with respect to the authorizations, I
> >>>>> think
> >>>>>>> it
> >>>>>>>>>> makes
> >>>>>>>>>>>>> sense
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> skip
> >>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit confused by
> >>>> the
> >>>>>>>> "leader
> >>>>>>>>>> ID"
> >>>>>>>>>>>>>> field.
> >>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the request as
> >>>> from a
> >>>>>>>> broker
> >>>>>>>>>> (does
> >>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>> matter
> >>>>>>>>>>>>>>>>>> which one?).
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> We could also make it version-based. For the next
> >>>>>>>> version, we
> >>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>> require
> >>>>>>>>>>>>>>>>>> CLUSTER auth. So clients would not be able to use
> >>>>> the
> >>>>>>> API
> >>>>>>>>>>>>> anymore,
> >>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>> probably what we want.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> -Jason
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> >>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> As a follow up, I was just thinking about the
> >>>>>>> batching
> >>>>>>>> a
> >>>>>>>>>> bit
> >>>>>>>>>>>>> more.
> >>>>>>>>>>>>>>>>>>> I suppose if we have one request in flight and
> >>>> we
> >>>>>>>> queue up
> >>>>>>>>>> the
> >>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>> produce requests in some sort of purgatory, we
> >>>>>> could
> >>>>>>>> send
> >>>>>>>>>>>>>>> information
> >>>>>>>>>>>>>>>>>> out
> >>>>>>>>>>>>>>>>>>> for all of them rather than one by one. So that
> >>>>>> would
> >>>>>>>> be a
> >>>>>>>>>>>>> benefit
> >>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>> batching partitions to add per transaction.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> I'll need to think a bit more on the design of
> >>>>> this
> >>>>>>>> part
> >>>>>>>>>> of the
> >>>>>>>>>>>>>> KIP,
> >>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>> will update the KIP in the next few days.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan
> >>>> <
> >>>>>>>>>>>>>>> jolshan@confluent.io>
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Hey Jason -- thanks for the input -- I was
> >>>> just
> >>>>>>>> digging
> >>>>>>>>>> a bit
> >>>>>>>>>>>>>>> deeper
> >>>>>>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>>>>> the design + implementation of the validation
> >>>>>> calls
> >>>>>>>> here
> >>>>>>>>>> and
> >>>>>>>>>>>>>> what
> >>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>> say
> >>>>>>>>>>>>>>>>>>>> makes sense.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I was wondering about compatibility here.
> >>>> When
> >>>>> we
> >>>>>>>> send
> >>>>>>>>>>>>> requests
> >>>>>>>>>>>>>>>>>>>> between brokers, we want to ensure that the
> >>>>>>> receiving
> >>>>>>>>>> broker
> >>>>>>>>>>>>>>>>>> understands
> >>>>>>>>>>>>>>>>>>>> the request (specifically the new fields).
> >>>>>>> Typically
> >>>>>>>>>> this is
> >>>>>>>>>>>>>> done
> >>>>>>>>>>>>>>>> via
> >>>>>>>>>>>>>>>>>>>> IBP/metadata version.
> >>>>>>>>>>>>>>>>>>>> I'm trying to think if there is a way around
> >>>> it
> >>>>>> but
> >>>>>>>> I'm
> >>>>>>>>>> not
> >>>>>>>>>>>>> sure
> >>>>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>>>> is.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> As for the improvements -- can you clarify
> >>>> how
> >>>>>> the
> >>>>>>>>>> multiple
> >>>>>>>>>>>>>>>>>> transactional
> >>>>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking of a
> >>>>> case
> >>>>>>>> where we
> >>>>>>>>>>>>>>> wait/batch
> >>>>>>>>>>>>>>>>>>>> multiple produce requests together? My
> >>>>>>> understanding
> >>>>>>>> for
> >>>>>>>>>> now
> >>>>>>>>>>>>>> was 1
> >>>>>>>>>>>>>>>>>>>> transactional ID and one validation per 1
> >>>>> produce
> >>>>>>>>>> request.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Finally with respect to the authorizations, I
> >>>>>> think
> >>>>>>>> it
> >>>>>>>>>> makes
> >>>>>>>>>>>>>> sense
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> skip
> >>>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit confused
> >>>> by
> >>>>>> the
> >>>>>>>>>> "leader
> >>>>>>>>>>>>> ID"
> >>>>>>>>>>>>>>>> field.
> >>>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the request as
> >>>>>> from a
> >>>>>>>>>> broker
> >>>>>>>>>>>>> (does
> >>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>> matter
> >>>>>>>>>>>>>>>>>>>> which one?).
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I think I want to adopt these suggestions,
> >>>> just
> >>>>>> had
> >>>>>>>> a few
> >>>>>>>>>>>>>>> questions
> >>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> details.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On Thu, Jan 5, 2023 at 5:05 PM Jason
> >>>> Gustafson
> >>>>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
> >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Hi Justine,
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Thanks for the proposal.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> I was thinking about the implementation a
> >>>>> little
> >>>>>>>> bit.
> >>>>>>>>>> In the
> >>>>>>>>>>>>>>>> current
> >>>>>>>>>>>>>>>>>>>>> proposal, the behavior depends on whether we
> >>>>>> have
> >>>>>>> an
> >>>>>>>>>> old or
> >>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>> client.
> >>>>>>>>>>>>>>>>>>>>> For
> >>>>>>>>>>>>>>>>>>>>> old clients, we send `DescribeTransactions`
> >>>>> and
> >>>>>>>> verify
> >>>>>>>>>> the
> >>>>>>>>>>>>>> result
> >>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>> new clients, we send `AddPartitionsToTxn`.
> >>>> We
> >>>>>>> might
> >>>>>>>> be
> >>>>>>>>>> able
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> simplify
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> implementation if we can use the same
> >>>> request
> >>>>>>> type.
> >>>>>>>> For
> >>>>>>>>>>>>>> example,
> >>>>>>>>>>>>>>>>>> what if
> >>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>> bump the protocol version for
> >>>>>> `AddPartitionsToTxn`
> >>>>>>>> and
> >>>>>>>>>> add a
> >>>>>>>>>>>>>>>>>>>>> `validateOnly`
> >>>>>>>>>>>>>>>>>>>>> flag? For older versions, we can set
> >>>>>>>>>> `validateOnly=true` so
> >>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> request only returns successfully if the
> >>>>>> partition
> >>>>>>>> had
> >>>>>>>>>>>>> already
> >>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>>> added.
> >>>>>>>>>>>>>>>>>>>>> For new versions, we can set
> >>>>>> `validateOnly=false`
> >>>>>>>> and
> >>>>>>>>>> the
> >>>>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>> added to the transaction. The other slightly
> >>>>>>>> annoying
> >>>>>>>>>> thing
> >>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>> get around is the need to collect the
> >>>>>> transaction
> >>>>>>>> state
> >>>>>>>>>> for
> >>>>>>>>>>>>> all
> >>>>>>>>>>>>>>>>>>> partitions
> >>>>>>>>>>>>>>>>>>>>> even when we only care about a subset.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Some additional improvements to consider:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> - We can give `AddPartitionsToTxn` better
> >>>>> batch
> >>>>>>>> support
> >>>>>>>>>> for
> >>>>>>>>>>>>>>>>>> inter-broker
> >>>>>>>>>>>>>>>>>>>>> usage. Currently we only allow one
> >>>>>>>> `TransactionalId` to
> >>>>>>>>>> be
> >>>>>>>>>>>>>>>> specified,
> >>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>> the broker may get some benefit being able
> >>>> to
> >>>>>>> batch
> >>>>>>>>>> across
> >>>>>>>>>>>>>>> multiple
> >>>>>>>>>>>>>>>>>>>>> transactions.
> >>>>>>>>>>>>>>>>>>>>> - Another small improvement is skipping
> >>>> topic
> >>>>>>>>>> authorization
> >>>>>>>>>>>>>>> checks
> >>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>> `AddPartitionsToTxn` when the request is
> >>>> from
> >>>>> a
> >>>>>>>> broker.
> >>>>>>>>>>>>> Perhaps
> >>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>> add
> >>>>>>>>>>>>>>>>>>>>> a field for the `LeaderId` or something like
> >>>>>> that
> >>>>>>>> and
> >>>>>>>>>>>>> require
> >>>>>>>>>>>>>>>> CLUSTER
> >>>>>>>>>>>>>>>>>>>>> permission when set.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>>> Jason
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
> >>>>>>>>>>>>>> <jun@confluent.io.invalid
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation. It makes sense
> >>>>> to
> >>>>>> me
> >>>>>>>> now.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Jun
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 1:42 PM Justine
> >>>>> Olshan
> >>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> My understanding of the mechanism is
> >>>> that
> >>>>>> when
> >>>>>>>> we
> >>>>>>>>>> get to
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> last
> >>>>>>>>>>>>>>>>>>>>> epoch,
> >>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>> increment to the fencing/last epoch and
> >>>> if
> >>>>>> any
> >>>>>>>>>> further
> >>>>>>>>>>>>>>> requests
> >>>>>>>>>>>>>>>>>> come
> >>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>> this producer ID they are fenced. Then
> >>>> the
> >>>>>>>> producer
> >>>>>>>>>>>>> gets a
> >>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>> ID
> >>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>> restarts with epoch/sequence 0. The
> >>>> fenced
> >>>>>>> epoch
> >>>>>>>>>> sticks
> >>>>>>>>>>>>>>> around
> >>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> duration of producer.id.expiration.ms
> >>>> and
> >>>>>>>> blocks
> >>>>>>>>>> any
> >>>>>>>>>>>>> late
> >>>>>>>>>>>>>>>>>> messages
> >>>>>>>>>>>>>>>>>>>>>> there.
> >>>>>>>>>>>>>>>>>>>>>>> The new ID will get to take advantage of
> >>>>> the
> >>>>>>>>>> improved
> >>>>>>>>>>>>>>> semantics
> >>>>>>>>>>>>>>>>>>> around
> >>>>>>>>>>>>>>>>>>>>>>> non-zero start sequences. So I think we
> >>>>> are
> >>>>>>>> covered.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> The only potential issue is overloading
> >>>>> the
> >>>>>>>> cache,
> >>>>>>>>>> but
> >>>>>>>>>>>>>>>> hopefully
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> improvements (lowered
> >>>>>>> producer.id.expiration.ms
> >>>>>>>> )
> >>>>>>>>>> will
> >>>>>>>>>>>>> help
> >>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>> that.
> >>>>>>>>>>>>>>>>>>>>>> Let
> >>>>>>>>>>>>>>>>>>>>>>> me know if you still have concerns.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
> >>>>>>>>>>>>>>>>>> <ju...@confluent.io.invalid>
> >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 70. The proposed fencing logic doesn't
> >>>>>> apply
> >>>>>>>> when
> >>>>>>>>>> pid
> >>>>>>>>>>>>>>>> changes,
> >>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>> right? If so, I am not sure how
> >>>> complete
> >>>>>> we
> >>>>>>>> are
> >>>>>>>>>>>>>> addressing
> >>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>> issue
> >>>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>> the pid changes more frequently.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Jun
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 16, 2022 at 9:16 AM
> >>>> Justine
> >>>>>>> Olshan
> >>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Thanks for replying!
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> 70.We already do the overflow
> >>>>> mechanism,
> >>>>>>> so
> >>>>>>>> my
> >>>>>>>>>>>>> change
> >>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>> happen more often.
> >>>>>>>>>>>>>>>>>>>>>>>>> I was also not suggesting a new
> >>>> field
> >>>>> in
> >>>>>>> the
> >>>>>>>>>> log,
> >>>>>>>>>>>>> but
> >>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> response,
> >>>>>>>>>>>>>>>>>>>>>>>>> which would be gated by the client
> >>>>>>> version.
> >>>>>>>>>> Sorry if
> >>>>>>>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>> unclear. I think we are starting to
> >>>>>>> diverge.
> >>>>>>>>>>>>>>>>>>>>>>>>> The goal of this KIP is to not
> >>>> change
> >>>>> to
> >>>>>>> the
> >>>>>>>>>> marker
> >>>>>>>>>>>>>>> format
> >>>>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>>>> all.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> 71. Yes, I guess I was going under
> >>>> the
> >>>>>>>>>> assumption
> >>>>>>>>>>>>> that
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> log
> >>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>>>> look at its last epoch and treat it
> >>>> as
> >>>>>> the
> >>>>>>>>>> current
> >>>>>>>>>>>>>>> epoch. I
> >>>>>>>>>>>>>>>>>>>>> suppose
> >>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>> have some special logic that if the
> >>>>> last
> >>>>>>>> epoch
> >>>>>>>>>> was
> >>>>>>>>>>>>> on a
> >>>>>>>>>>>>>>>>>> marker
> >>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>>>>>>> expect the next epoch or something
> >>>>> like
> >>>>>>>> that. We
> >>>>>>>>>>>>> just
> >>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>> distinguish
> >>>>>>>>>>>>>>>>>>>>>>>>> based on whether we had a
> >>>> commit/abort
> >>>>>>>> marker.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> 72.
> >>>>>>>>>>>>>>>>>>>>>>>>>> if the producer epoch hasn't been
> >>>>>> bumped
> >>>>>>>> on
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
> >>>>>> message
> >>>>>>>> will
> >>>>>>>>>> fail
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> sequence
> >>>>>>>>>>>>>>>>>>>>>>>> validation
> >>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the producer
> >>>>>> epoch
> >>>>>>>> has
> >>>>>>>>>> been
> >>>>>>>>>>>>>>> bumped,
> >>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>> ignore
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck message
> >>>>>> could
> >>>>>>>> be
> >>>>>>>>>>>>> appended
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> log.
> >>>>>>>>>>>>>>>>>>>>>> So,
> >>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
> >>>> guard?
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I follow that "the
> >>>>> message
> >>>>>>> will
> >>>>>>>>>> fail
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> sequence
> >>>>>>>>>>>>>>>>>>>>>>>> validation".
> >>>>>>>>>>>>>>>>>>>>>>>>> In some of these cases, we had an
> >>>>> abort
> >>>>>>>> marker
> >>>>>>>>>> (due
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>> error)
> >>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>> then
> >>>>>>>>>>>>>>>>>>>>>>>>> the late message comes in with the
> >>>>>> correct
> >>>>>>>>>> sequence
> >>>>>>>>>>>>>>> number.
> >>>>>>>>>>>>>>>>>> This
> >>>>>>>>>>>>>>>>>>>>> is a
> >>>>>>>>>>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>>>>>> covered by the KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>> The latter case is actually not
> >>>>>> something
> >>>>>>>> we've
> >>>>>>>>>>>>>>> considered
> >>>>>>>>>>>>>>>>>>> here. I
> >>>>>>>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>>>> generally when we bump the epoch, we
> >>>>> are
> >>>>>>>>>> accepting
> >>>>>>>>>>>>> that
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> sequence
> >>>>>>>>>>>>>>>>>>>>>>> does
> >>>>>>>>>>>>>>>>>>>>>>>>> not need to be checked anymore. My
> >>>>>>>>>> understanding is
> >>>>>>>>>>>>>> also
> >>>>>>>>>>>>>>>>>> that we
> >>>>>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>>>> typically bump epoch mid transaction
> >>>>>>> (based
> >>>>>>>> on a
> >>>>>>>>>>>>> quick
> >>>>>>>>>>>>>>> look
> >>>>>>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> code)
> >>>>>>>>>>>>>>>>>>>>>>>>> but let me know if that is the case.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 15, 2022 at 12:23 PM Jun
> >>>>> Rao
> >>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the reply.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> 70. Assigning a new pid on int
> >>>>>> overflow
> >>>>>>>> seems
> >>>>>>>>>> a
> >>>>>>>>>>>>> bit
> >>>>>>>>>>>>>>>> hacky.
> >>>>>>>>>>>>>>>>>> If
> >>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>> need a
> >>>>>>>>>>>>>>>>>>>>>>>>> txn
> >>>>>>>>>>>>>>>>>>>>>>>>>> level id, it will be better to
> >>>> model
> >>>>>>> this
> >>>>>>>>>>>>> explicitly.
> >>>>>>>>>>>>>>>>>> Adding a
> >>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>>> field
> >>>>>>>>>>>>>>>>>>>>>>>>>> would require a bit more work
> >>>> since
> >>>>> it
> >>>>>>>>>> requires a
> >>>>>>>>>>>>> new
> >>>>>>>>>>>>>>> txn
> >>>>>>>>>>>>>>>>>>> marker
> >>>>>>>>>>>>>>>>>>>>>>> format
> >>>>>>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>> the log. So, we probably need to
> >>>>> guard
> >>>>>>> it
> >>>>>>>>>> with an
> >>>>>>>>>>>>> IBP
> >>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>> metadata
> >>>>>>>>>>>>>>>>>>>>>>>> version
> >>>>>>>>>>>>>>>>>>>>>>>>>> and document the impact on
> >>>> downgrade
> >>>>>>> once
> >>>>>>>> the
> >>>>>>>>>> new
> >>>>>>>>>>>>>>> format
> >>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>> written
> >>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> log.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> 71. Hmm, once the marker is
> >>>> written,
> >>>>>> the
> >>>>>>>>>> partition
> >>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>> expect
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> next
> >>>>>>>>>>>>>>>>>>>>>>>>>> append to be on the next epoch.
> >>>> Does
> >>>>>>> that
> >>>>>>>>>> cover
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>> mentioned?
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> 72. Also, just to be clear on the
> >>>>>>> stucked
> >>>>>>>>>> message
> >>>>>>>>>>>>>> issue
> >>>>>>>>>>>>>>>>>>>>> described
> >>>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> motivation. With EoS, we also
> >>>>> validate
> >>>>>>> the
> >>>>>>>>>>>>> sequence
> >>>>>>>>>>>>>> id
> >>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>> idempotency.
> >>>>>>>>>>>>>>>>>>>>>>>>> So,
> >>>>>>>>>>>>>>>>>>>>>>>>>> with the current logic, if the
> >>>>>> producer
> >>>>>>>> epoch
> >>>>>>>>>>>>> hasn't
> >>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>>>>> bumped on
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
> >>>>>>> message
> >>>>>>>> will
> >>>>>>>>>>>>> fail
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> sequence
> >>>>>>>>>>>>>>>>>>>>>>>>> validation
> >>>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the
> >>>> producer
> >>>>>>>> epoch has
> >>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>> bumped, we
> >>>>>>>>>>>>>>>>>>>>>> ignore
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck
> >>>> message
> >>>>>>>> could be
> >>>>>>>>>>>>>> appended
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> log.
> >>>>>>>>>>>>>>>>>>>>>>> So,
> >>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
> >>>>> guard?
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 10:44 AM
> >>>>>> Justine
> >>>>>>>>>> Olshan
> >>>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> >>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias — thanks again for
> >>>> taking
> >>>>>>> time
> >>>>>>>> to
> >>>>>>>>>> look
> >>>>>>>>>>>>> a
> >>>>>>>>>>>>>>> this.
> >>>>>>>>>>>>>>>>>> You
> >>>>>>>>>>>>>>>>>>>>> said:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> My proposal was only focusing
> >>>> to
> >>>>>>> avoid
> >>>>>>>>>>>>> dangling
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
> >>>> added
> >>>>>>>> without
> >>>>>>>>>>>>>> registered
> >>>>>>>>>>>>>>>>>>>>> partition.
> >>>>>>>>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>>>>>>>>>> Maybe
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more details
> >>>> to
> >>>>>> the
> >>>>>>>> KIP
> >>>>>>>>>> about
> >>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>> scenario
> >>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>> better
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I understand what
> >>>> you
> >>>>>>> mean
> >>>>>>>>>> here.
> >>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>> motivation
> >>>>>>>>>>>>>>>>>>>>>>> section
> >>>>>>>>>>>>>>>>>>>>>>>>>>> describes two scenarios about
> >>>> how
> >>>>>> the
> >>>>>>>> record
> >>>>>>>>>>>>> can be
> >>>>>>>>>>>>>>>> added
> >>>>>>>>>>>>>>>>>>>>>> without a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> registered partition:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Another way hanging
> >>>> transactions
> >>>>>> can
> >>>>>>>>>> occur is
> >>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>> client
> >>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>> buggy
> >>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>> may somehow try to write to a
> >>>>>>> partition
> >>>>>>>>>> before
> >>>>>>>>>>>>> it
> >>>>>>>>>>>>>>> adds
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> transaction.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> For the first example of this
> >>>>> would
> >>>>>> it
> >>>>>>>> be
> >>>>>>>>>>>>> helpful
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> say
> >>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>> message
> >>>>>>>>>>>>>>>>>>>>>>>>>>> comes in after the abort, but
> >>>>> before
> >>>>>>> the
> >>>>>>>>>>>>> partition
> >>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>> added
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> next
> >>>>>>>>>>>>>>>>>>>>>>>>>>> transaction so it becomes
> >>>>> "hanging."
> >>>>>>>>>> Perhaps the
> >>>>>>>>>>>>>> next
> >>>>>>>>>>>>>>>>>>> sentence
> >>>>>>>>>>>>>>>>>>>>>>>>> describing
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the message becoming part of the
> >>>>>> next
> >>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>> (a
> >>>>>>>>>>>>>>>>>>> different
> >>>>>>>>>>>>>>>>>>>>>>> case)
> >>>>>>>>>>>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>>>>>>>>> not properly differentiated.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Jun — thanks for reading the
> >>>> KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> 70. The int typing was a
> >>>> concern.
> >>>>>>>> Currently
> >>>>>>>>>> we
> >>>>>>>>>>>>>> have a
> >>>>>>>>>>>>>>>>>>>>> mechanism
> >>>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>> place
> >>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> fence the final epoch when the
> >>>>> epoch
> >>>>>>> is
> >>>>>>>>>> about to
> >>>>>>>>>>>>>>>> overflow
> >>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>> assign
> >>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>>>>>> producer ID with epoch 0. Of
> >>>>> course,
> >>>>>>>> this
> >>>>>>>>>> is a
> >>>>>>>>>>>>> bit
> >>>>>>>>>>>>>>>> tricky
> >>>>>>>>>>>>>>>>>>>>> when it
> >>>>>>>>>>>>>>>>>>>>>>>> comes
> >>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the response back to the client.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Making this a long could be
> >>>>> another
> >>>>>>>> option,
> >>>>>>>>>> but
> >>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>> wonder
> >>>>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>>>>>>> implications on changing this
> >>>>> field
> >>>>>> if
> >>>>>>>> the
> >>>>>>>>>>>>> epoch is
> >>>>>>>>>>>>>>>>>>> persisted
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>> disk?
> >>>>>>>>>>>>>>>>>>>>>>>>>> I'd
> >>>>>>>>>>>>>>>>>>>>>>>>>>> need to check the usages.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> 71.This was something Matthias
> >>>>> asked
> >>>>>>>> about
> >>>>>>>>>> as
> >>>>>>>>>>>>>> well. I
> >>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>>>>>> considering a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> possible edge case where a
> >>>> produce
> >>>>>>>> request
> >>>>>>>>>> from
> >>>>>>>>>>>>> a
> >>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>> somehow
> >>>>>>>>>>>>>>>>>>>>>>>>>>> gets sent right after the marker
> >>>>> is
> >>>>>>>>>> written, but
> >>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>> alerted of the newly bumped
> >>>> epoch.
> >>>>>> In
> >>>>>>>> this
> >>>>>>>>>>>>> case, we
> >>>>>>>>>>>>>>> may
> >>>>>>>>>>>>>>>>>>>>> include
> >>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>> record
> >>>>>>>>>>>>>>>>>>>>>>>>>>> when we don't want to. I suppose
> >>>>> we
> >>>>>>>> could
> >>>>>>>>>> try
> >>>>>>>>>>>>> to do
> >>>>>>>>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>>>>>> client
> >>>>>>>>>>>>>>>>>>>>>>>>> side
> >>>>>>>>>>>>>>>>>>>>>>>>>>> to bump the epoch after sending
> >>>> an
> >>>>>>>> endTxn as
> >>>>>>>>>>>>> well
> >>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>> scenario
> >>>>>>>>>>>>>>>>>>>>>>> —
> >>>>>>>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>> wonder how it would work when
> >>>> the
> >>>>>>>> server is
> >>>>>>>>>>>>>> aborting
> >>>>>>>>>>>>>>>>>> based
> >>>>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>> server-side
> >>>>>>>>>>>>>>>>>>>>>>>>>>> error. I could also be missing
> >>>>>>>> something and
> >>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>> scenario
> >>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>> possible.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again to everyone reading
> >>>>> and
> >>>>>>>>>> commenting.
> >>>>>>>>>>>>>> Let
> >>>>>>>>>>>>>>> me
> >>>>>>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>>>>>>> further questions or comments.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 9:41 AM
> >>>>> Jun
> >>>>>>> Rao
> >>>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP. A couple
> >>>> of
> >>>>>>>> comments.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Currently, the producer
> >>>>> epoch
> >>>>>> is
> >>>>>>>> an
> >>>>>>>>>> int.
> >>>>>>>>>>>>> I am
> >>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>> sure
> >>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>>>> enough
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> to accommodate all
> >>>> transactions
> >>>>> in
> >>>>>>> the
> >>>>>>>>>>>>> lifetime
> >>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>> producer.
> >>>>>>>>>>>>>>>>>>>>>>>> Should
> >>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> change that to a long or add a
> >>>>> new
> >>>>>>>> long
> >>>>>>>>>> field
> >>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>> txnId?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> 71. "it will write the prepare
> >>>>>>> commit
> >>>>>>>>>> message
> >>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>> bumped
> >>>>>>>>>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>> send
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> WriteTxnMarkerRequests with
> >>>> the
> >>>>>>> bumped
> >>>>>>>>>> epoch."
> >>>>>>>>>>>>>> Hmm,
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>> associated
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> with the current txn right?
> >>>> So,
> >>>>> it
> >>>>>>>> seems
> >>>>>>>>>>>>> weird to
> >>>>>>>>>>>>>>>>>> write a
> >>>>>>>>>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>>>>>>>>> message
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> with a bumped epoch. Should we
> >>>>>> only
> >>>>>>>> bump
> >>>>>>>>>> up
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>> EndTxnResponse
> >>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> rename the field to sth like
> >>>>>>>>>>>>> nextProducerEpoch?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 12, 2022 at 8:54
> >>>> PM
> >>>>>>>> Matthias
> >>>>>>>>>> J.
> >>>>>>>>>>>>> Sax <
> >>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the background.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30: SGTM. My proposal was
> >>>>>> only
> >>>>>>>>>> focusing
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>> avoid
> >>>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
> >>>>>> added
> >>>>>>>>>> without
> >>>>>>>>>>>>>>>> registered
> >>>>>>>>>>>>>>>>>>>>>> partition.
> >>>>>>>>>>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more
> >>>> details
> >>>>>> to
> >>>>>>>> the
> >>>>>>>>>> KIP
> >>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>> scenario
> >>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>> better
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40: I think you hit a fair
> >>>>> point
> >>>>>>>> about
> >>>>>>>>>> race
> >>>>>>>>>>>>>>>>>> conditions
> >>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>> client
> >>>>>>>>>>>>>>>>>>>>>>>>> bugs
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> (incorrectly not bumping the
> >>>>>>>> epoch). The
> >>>>>>>>>>>>>>>>>>>>> complexity/confusion
> >>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>> using
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the bumped epoch I see, is
> >>>>>> mainly
> >>>>>>>> for
> >>>>>>>>>>>>> internal
> >>>>>>>>>>>>>>>>>>> debugging,
> >>>>>>>>>>>>>>>>>>>>> ie,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> inspecting
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> log segment dumps -- it
> >>>> seems
> >>>>>>>> harder to
> >>>>>>>>>>>>> reason
> >>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> system
> >>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>> us
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> humans. But if we get better
> >>>>>>>>>> guarantees, it
> >>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>> worth to
> >>>>>>>>>>>>>>>>>>>>>>> use
> >>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped epoch.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60: as I mentioned already,
> >>>> I
> >>>>>>> don't
> >>>>>>>>>> know the
> >>>>>>>>>>>>>>> broker
> >>>>>>>>>>>>>>>>>>>>> internals
> >>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> provide
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> more input. So if nobody
> >>>> else
> >>>>>>> chimes
> >>>>>>>>>> in, we
> >>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>> move
> >>>>>>>>>>>>>>>>>>>>>>>>> forward
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> with your proposal.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 12/6/22 4:22 PM, Justine
> >>>>>> Olshan
> >>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi all,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> After Artem's questions
> >>>>> about
> >>>>>>>> error
> >>>>>>>>>>>>> behavior,
> >>>>>>>>>>>>>>>> I've
> >>>>>>>>>>>>>>>>>>>>>>> re-evaluated
> >>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unknown producer ID
> >>>>> exception
> >>>>>>> and
> >>>>>>>> had
> >>>>>>>>>> some
> >>>>>>>>>>>>>>>>>> discussions
> >>>>>>>>>>>>>>>>>>>>>>> offline.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think generally it makes
> >>>>>> sense
> >>>>>>>> to
> >>>>>>>>>>>>> simplify
> >>>>>>>>>>>>>>>> error
> >>>>>>>>>>>>>>>>>>>>> handling
> >>>>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>> cases
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this and the
> >>>>>> UNKNOWN_PRODUCER_ID
> >>>>>>>> error
> >>>>>>>>>>>>> has a
> >>>>>>>>>>>>>>>> pretty
> >>>>>>>>>>>>>>>>>>> long
> >>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> complicated
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> history. Because of this,
> >>>> I
> >>>>>>>> propose
> >>>>>>>>>>>>> adding a
> >>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>> error
> >>>>>>>>>>>>>>>>>>>>> code
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> ABORTABLE_ERROR
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that when encountered by
> >>>> new
> >>>>>>>> clients
> >>>>>>>>>>>>> (gated
> >>>>>>>>>>>>>> by
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> produce
> >>>>>>>>>>>>>>>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> version)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will simply abort the
> >>>>>>> transaction.
> >>>>>>>>>> This
> >>>>>>>>>>>>>> allows
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> server
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> say
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in whether the client
> >>>> aborts
> >>>>>> and
> >>>>>>>> makes
> >>>>>>>>>>>>>> handling
> >>>>>>>>>>>>>>>>>> much
> >>>>>>>>>>>>>>>>>>>>>> simpler.
> >>>>>>>>>>>>>>>>>>>>>>>> In
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> future, we can also use
> >>>> this
> >>>>>>>> error in
> >>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>> situations
> >>>>>>>>>>>>>>>>>>>>>> where
> >>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>> want
> >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> abort the transactions. We
> >>>>> can
> >>>>>>>> even
> >>>>>>>>>> use on
> >>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>> apis.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've added this to the
> >>>> KIP.
> >>>>>> Let
> >>>>>>> me
> >>>>>>>>>> know if
> >>>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>>>>> questions
> >>>>>>>>>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 2, 2022 at
> >>>> 10:22
> >>>>>> AM
> >>>>>>>>>> Justine
> >>>>>>>>>>>>>> Olshan
> >>>>>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>>>>> jolshan@confluent.io
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey Matthias,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30 — Maybe I also
> >>>> didn't
> >>>>>>>> express
> >>>>>>>>>>>>> myself
> >>>>>>>>>>>>>>>>>> clearly.
> >>>>>>>>>>>>>>>>>>> For
> >>>>>>>>>>>>>>>>>>>>>>> older
> >>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't have a way to
> >>>>>> distinguish
> >>>>>>>>>> between a
> >>>>>>>>>>>>>>>> previous
> >>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> current
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction since we
> >>>> don't
> >>>>>> have
> >>>>>>>> the
> >>>>>>>>>> epoch
> >>>>>>>>>>>>>>> bump.
> >>>>>>>>>>>>>>>>>> This
> >>>>>>>>>>>>>>>>>>>>> means
> >>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> late
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message from the previous
> >>>>>>>> transaction
> >>>>>>>>>>>>> may be
> >>>>>>>>>>>>>>>>>> added to
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>>>> one.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> With
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older clients — we can't
> >>>>>>>> guarantee
> >>>>>>>>>> this
> >>>>>>>>>>>>>> won't
> >>>>>>>>>>>>>>>>>> happen
> >>>>>>>>>>>>>>>>>>>>> if we
> >>>>>>>>>>>>>>>>>>>>>>>>> already
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call
> >>>>> (why
> >>>>>> we
> >>>>>>>> make
> >>>>>>>>>>>>> changes
> >>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> newer
> >>>>>>>>>>>>>>>>>>>>>>>>> client)
> >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can at least gate some by
> >>>>>>>> ensuring
> >>>>>>>>>> that
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>>>>> has
> >>>>>>>>>>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>>>>>>>>>> added
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. The
> >>>> rationale
> >>>>>> here
> >>>>>>>> is
> >>>>>>>>>> that
> >>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>>>> likely
> >>>>>>>>>>>>>>>>>>>>>>> LESS
> >>>>>>>>>>>>>>>>>>>>>>>>> late
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrivals
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as time goes on, so
> >>>>> hopefully
> >>>>>>>> most
> >>>>>>>>>> late
> >>>>>>>>>>>>>>> arrivals
> >>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>> come
> >>>>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>> BEFORE
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call.
> >>>>>> Those
> >>>>>>>> that
> >>>>>>>>>>>>> arrive
> >>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>> properly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> gated
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> >>>>> describeTransactions
> >>>>>>>>>> approach.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we take the approach
> >>>> you
> >>>>>>>>>> suggested,
> >>>>>>>>>>>>> ANY
> >>>>>>>>>>>>>>> late
> >>>>>>>>>>>>>>>>>>> arrival
> >>>>>>>>>>>>>>>>>>>>>>> from a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> previous
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction will be
> >>>> added.
> >>>>>> And
> >>>>>>> we
> >>>>>>>>>> don't
> >>>>>>>>>>>>> want
> >>>>>>>>>>>>>>>>>> that. I
> >>>>>>>>>>>>>>>>>>>>> also
> >>>>>>>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>>>>> see
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> benefit in sending
> >>>>>>>> addPartitionsToTxn
> >>>>>>>>>>>>> over
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> describeTxns
> >>>>>>>>>>>>>>>>>>>>>>>>> call.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> They
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> both be one extra RPC to
> >>>>> the
> >>>>>>> Txn
> >>>>>>>>>>>>>> coordinator.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To be clear — newer
> >>>> clients
> >>>>>>> will
> >>>>>>>> use
> >>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> >>>>>>>>>>>>>>>>>>>>>>>> instead
> >>>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTxns.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that if we
> >>>>> have
> >>>>>>>> some
> >>>>>>>>>> delay
> >>>>>>>>>>>>> in
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> client
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>> bump
> >>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it could continue to send
> >>>>>> epoch
> >>>>>>>> 73
> >>>>>>>>>> and
> >>>>>>>>>>>>> those
> >>>>>>>>>>>>>>>>>> records
> >>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> fenced.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Perhaps this is not an
> >>>>> issue
> >>>>>> if
> >>>>>>>> we
> >>>>>>>>>> don't
> >>>>>>>>>>>>>> allow
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> next
> >>>>>>>>>>>>>>>>>>>>>>>> produce
> >>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> go
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through before the EndTxn
> >>>>>>> request
> >>>>>>>>>>>>> returns.
> >>>>>>>>>>>>>> I'm
> >>>>>>>>>>>>>>>>>> also
> >>>>>>>>>>>>>>>>>>>>>> thinking
> >>>>>>>>>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> cases of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure. I will need to
> >>>>> think
> >>>>>>> on
> >>>>>>>>>> this a
> >>>>>>>>>>>>> bit.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I wasn't sure if it was
> >>>>> that
> >>>>>>>>>> confusing.
> >>>>>>>>>>>>> But
> >>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>> think it
> >>>>>>>>>>>>>>>>>>>>>>> is,
> >>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> investigate other ways.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure these are
> >>>> the
> >>>>>> same
> >>>>>>>>>>>>> purgatories
> >>>>>>>>>>>>>>>> since
> >>>>>>>>>>>>>>>>>> one
> >>>>>>>>>>>>>>>>>>>>> is a
> >>>>>>>>>>>>>>>>>>>>>>>>> produce
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory (I was planning
> >>>>> on
> >>>>>>>> using a
> >>>>>>>>>>>>>> callback
> >>>>>>>>>>>>>>>>>> rather
> >>>>>>>>>>>>>>>>>>>>> than
> >>>>>>>>>>>>>>>>>>>>>>>>>> purgatory)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the other is simply a
> >>>>> request
> >>>>>>> to
> >>>>>>>>>> append
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> log.
> >>>>>>>>>>>>>>>>>>> Not
> >>>>>>>>>>>>>>>>>>>>>> sure
> >>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> structure here for
> >>>>> ordering,
> >>>>>>> but
> >>>>>>>> my
> >>>>>>>>>>>>>>>> understanding
> >>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> broker
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handle the write request
> >>>>>> before
> >>>>>>>> it
> >>>>>>>>>> hears
> >>>>>>>>>>>>>> back
> >>>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> Txn
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Coordinator.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if I
> >>>>>> misunderstood
> >>>>>>>>>> something
> >>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>>>>>>> unclear.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 1, 2022 at
> >>>>> 12:15
> >>>>>> PM
> >>>>>>>>>> Matthias
> >>>>>>>>>>>>> J.
> >>>>>>>>>>>>>>> Sax
> >>>>>>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the details
> >>>>>>> Justine!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side change
> >>>>> for
> >>>>>> 2
> >>>>>>> is
> >>>>>>>>>>>>> removing
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> addPartitions
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need to
> >>>>> make
> >>>>>>>> this
> >>>>>>>>>> from
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> txn
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think I did not
> >>>> express
> >>>>>>> myself
> >>>>>>>>>>>>> clearly. I
> >>>>>>>>>>>>>>>>>>> understand
> >>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> (and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should) change the
> >>>>> producer
> >>>>>> to
> >>>>>>>> not
> >>>>>>>>>> send
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> `addPartitions`
> >>>>>>>>>>>>>>>>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> longer. But I don't
> >>>> thinks
> >>>>>>> it's
> >>>>>>>>>>>>> requirement
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> change
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> broker?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What I am trying to say
> >>>>> is:
> >>>>>>> as a
> >>>>>>>>>>>>> safe-guard
> >>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>> improvement
> >>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>> older
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producers, the partition
> >>>>>>> leader
> >>>>>>>> can
> >>>>>>>>>> just
> >>>>>>>>>>>>>> send
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request to the
> >>>>>> TX-coordinator
> >>>>>>>> in any
> >>>>>>>>>>>>> case
> >>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> old
> >>>>>>>>>>>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> correctly did send the
> >>>>>>>>>> `addPartition`
> >>>>>>>>>>>>>> request
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> already, the
> >>>>> TX-coordinator
> >>>>>>> can
> >>>>>>>> just
> >>>>>>>>>>>>>> "ignore"
> >>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>>> idempotent.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> However,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if the old producer has
> >>>> a
> >>>>>> bug
> >>>>>>>> and
> >>>>>>>>>> did
> >>>>>>>>>>>>>> forget
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> sent
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> `addPartition`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request, we would now
> >>>>> ensure
> >>>>>>>> that
> >>>>>>>>>> the
> >>>>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>> indeed
> >>>>>>>>>>>>>>>>>>>>>>>> added
> >>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX and thus fix a
> >>>>> potential
> >>>>>>>>>> producer bug
> >>>>>>>>>>>>>>> (even
> >>>>>>>>>>>>>>>>>> if we
> >>>>>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fencing via the bump
> >>>>> epoch).
> >>>>>>> --
> >>>>>>>> It
> >>>>>>>>>>>>> seems to
> >>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>> good
> >>>>>>>>>>>>>>>>>>>>>>>>>> improvement?
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Or
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there a reason to not do
> >>>>>> this?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is ongoing
> >>>> =
> >>>>>>>> partition
> >>>>>>>>>> was
> >>>>>>>>>>>>>> added
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>> via
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn. We
> >>>>>> check
> >>>>>>>> this
> >>>>>>>>>> with
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this wasn't
> >>>>>>>> sufficiently
> >>>>>>>>>>>>>>> explained
> >>>>>>>>>>>>>>>>>> here:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do what I propose
> >>>> in
> >>>>>>>> (20), we
> >>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>> really
> >>>>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `DescribeTransaction`
> >>>>> call,
> >>>>>> as
> >>>>>>>> the
> >>>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>> leader
> >>>>>>>>>>>>>>>>>>>>> adds
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for older clients and we
> >>>>> get
> >>>>>>>> this
> >>>>>>>>>> check
> >>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>> free.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is that
> >>>> if
> >>>>>> any
> >>>>>>>>>> messages
> >>>>>>>>>>>>>>> somehow
> >>>>>>>>>>>>>>>>>> come
> >>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the producer,
> >>>>> they
> >>>>>>>> will be
> >>>>>>>>>>>>>> fenced.
> >>>>>>>>>>>>>>>>>>> However,
> >>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it can be
> >>>>>>>> discussed
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree that we should
> >>>>> have
> >>>>>>>> epoch
> >>>>>>>>>>>>> fencing.
> >>>>>>>>>>>>>> My
> >>>>>>>>>>>>>>>>>>>>> question is
> >>>>>>>>>>>>>>>>>>>>>>>>>>> different:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Assume we are at epoch
> >>>> 73,
> >>>>>> and
> >>>>>>>> we
> >>>>>>>>>> have
> >>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>> ongoing
> >>>>>>>>>>>>>>>>>>>>>>>> transaction,
> >>>>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed. It seems
> >>>>> natural
> >>>>>> to
> >>>>>>>>>> write the
> >>>>>>>>>>>>>>>> "prepare
> >>>>>>>>>>>>>>>>>>>>> commit"
> >>>>>>>>>>>>>>>>>>>>>>>>> marker
> >>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
> >>>>> both
> >>>>>>> with
> >>>>>>>>>> epoch
> >>>>>>>>>>>>> 73,
> >>>>>>>>>>>>>>> too,
> >>>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>> belongs
> >>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current transaction. Of
> >>>>>>> course,
> >>>>>>>> we
> >>>>>>>>>> now
> >>>>>>>>>>>>> also
> >>>>>>>>>>>>>>>> bump
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>> expect
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the next requests to
> >>>> have
> >>>>>>> epoch
> >>>>>>>> 74,
> >>>>>>>>>> and
> >>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>> reject
> >>>>>>>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch 73, as the
> >>>>>> corresponding
> >>>>>>>> TX
> >>>>>>>>>> for
> >>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>> 73
> >>>>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>>>> already
> >>>>>>>>>>>>>>>>>>>>>>>>>>> committed.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems you propose to
> >>>>>> write
> >>>>>>>> the
> >>>>>>>>>>>>> "prepare
> >>>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>>>> marker"
> >>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
> >>>>> with
> >>>>>>>> epoch 74
> >>>>>>>>>>>>>> though,
> >>>>>>>>>>>>>>>> what
> >>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>> work,
> >>>>>>>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems confusing. Is
> >>>> there
> >>>>> a
> >>>>>>>> reason
> >>>>>>>>>> why
> >>>>>>>>>>>>> we
> >>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>> use
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> bumped
> >>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> 74
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of the current
> >>>>> epoch
> >>>>>>> 73?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are checking if
> >>>>> the
> >>>>>>>>>>>>> transaction is
> >>>>>>>>>>>>>>>>>> ongoing,
> >>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the leader
> >>>>>>> partition
> >>>>>>>> to
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>> coordinator.
> >>>>>>>>>>>>>>>>>>>>>>>>>> In
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for this
> >>>>>>>> message to
> >>>>>>>>>> come
> >>>>>>>>>>>>>>> back,
> >>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>> theory
> >>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
> >>>> that
> >>>>>>> would
> >>>>>>>>>> make the
> >>>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>>>>>>> result
> >>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> check
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why we
> >>>> can
> >>>>>>> check
> >>>>>>>> the
> >>>>>>>>>>>>> leader
> >>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>> write
> >>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks. Got it.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, is this really
> >>>> an
> >>>>>>>> issue?
> >>>>>>>>>> We put
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> produce
> >>>>>>>>>>>>>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory, so how could
> >>>> we
> >>>>>>>> process
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
> >>>>>>>>>>>>>>>>>>>>>>>>>>> first?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Don't we need to put the
> >>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
> >>>>>>>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>>>>>>>>>> purgatory,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> too,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for this case, and
> >>>> process
> >>>>>>> both
> >>>>>>>>>> request
> >>>>>>>>>>>>>>>> in-order?
> >>>>>>>>>>>>>>>>>>>>> (Again,
> >>>>>>>>>>>>>>>>>>>>>>> my
> >>>>>>>>>>>>>>>>>>>>>>>>>> broker
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> knowledge is limited and
> >>>>>> maybe
> >>>>>>>> we
> >>>>>>>>>> don't
> >>>>>>>>>>>>>>>> maintain
> >>>>>>>>>>>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>>>>>>>>> order
> >>>>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case, what seems to be
> >>>> an
> >>>>>>> issue
> >>>>>>>>>> IMHO,
> >>>>>>>>>>>>> and I
> >>>>>>>>>>>>>>> am
> >>>>>>>>>>>>>>>>>>>>> wondering
> >>>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>>>> changing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request handling to
> >>>>> preserve
> >>>>>>>> order
> >>>>>>>>>> for
> >>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>> might be
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> cleaner
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution?)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/30/22 3:28 PM,
> >>>> Artem
> >>>>>>>> Livshits
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think the interesting
> >>>>>> part
> >>>>>>> is
> >>>>>>>>>> not in
> >>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>> logic
> >>>>>>>>>>>>>>>>>>>>>> (because
> >>>>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>> tries
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> figure out when
> >>>>>>>>>> UNKNOWN_PRODUCER_ID is
> >>>>>>>>>>>>>>>> retriable
> >>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> retryable,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's definitely not
> >>>>> fatal),
> >>>>>>> but
> >>>>>>>>>> what
> >>>>>>>>>>>>>> happens
> >>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>> logic
> >>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'true' and falls
> >>>> through.
> >>>>>> In
> >>>>>>>> the
> >>>>>>>>>> old
> >>>>>>>>>>>>>>> clients
> >>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>> fatal,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> keep the behavior in
> >>>> the
> >>>>>> new
> >>>>>>>>>> clients,
> >>>>>>>>>>>>> I'd
> >>>>>>>>>>>>>>>>>> expect it
> >>>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>> fatal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> well.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 29, 2022 at
> >>>>>> 11:57
> >>>>>>>> AM
> >>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>> Olshan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>> <jolshan@confluent.io.invalid
> >>>>>>>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Artem and Jeff,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
> >>>> look
> >>>>>> and
> >>>>>>>>>> sorry for
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> slow
> >>>>>>>>>>>>>>>>>>>>>> response.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You both mentioned the
> >>>>>>> change
> >>>>>>>> to
> >>>>>>>>>>>>> handle
> >>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> >>>>>>>>>>>>>>>>>>>>>>>>>>> errors.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> To
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear — this error
> >>>> code
> >>>>>> will
> >>>>>>>> only
> >>>>>>>>>> be
> >>>>>>>>>>>>> sent
> >>>>>>>>>>>>>>>> again
> >>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> client's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version is high enough
> >>>>> to
> >>>>>>>> ensure
> >>>>>>>>>> we
> >>>>>>>>>>>>>> handle
> >>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>> correctly.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The current (Java)
> >>>>> client
> >>>>>>>> handles
> >>>>>>>>>>>>> this by
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> following
> >>>>>>>>>>>>>>>>>>>>>>>>>> (somewhat
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> long)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code snippet:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // An
> >>>>> UNKNOWN_PRODUCER_ID
> >>>>>>>> means
> >>>>>>>>>> that
> >>>>>>>>>>>>> we
> >>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>> lost
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker. Depending on
> >>>> the
> >>>>>> log
> >>>>>>>> start
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // offset, we may want
> >>>>> to
> >>>>>>>> retry
> >>>>>>>>>>>>> these, as
> >>>>>>>>>>>>>>>>>>> described
> >>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>> each
> >>>>>>>>>>>>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below. If
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> none of those apply,
> >>>>> then
> >>>>>>> for
> >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // idempotent
> >>>> producer,
> >>>>> we
> >>>>>>>> will
> >>>>>>>>>>>>> locally
> >>>>>>>>>>>>>>> bump
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>> reset
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence numbers of
> >>>>>>> in-flight
> >>>>>>>>>> batches
> >>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // sequence 0, then
> >>>>> retry
> >>>>>>> the
> >>>>>>>>>> failed
> >>>>>>>>>>>>>> batch,
> >>>>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>>>>>> now
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> succeed.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the transactional
> >>>>>> producer,
> >>>>>>>> allow
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // batch to fail. When
> >>>>>>>> processing
> >>>>>>>>>> the
> >>>>>>>>>>>>>>> failed
> >>>>>>>>>>>>>>>>>>> batch,
> >>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transition
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an abortable error and
> >>>>>> set a
> >>>>>>>> flag
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // indicating that we
> >>>>> need
> >>>>>>> to
> >>>>>>>>>> bump the
> >>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>> (if
> >>>>>>>>>>>>>>>>>>>>>>> supported
> >>>>>>>>>>>>>>>>>>>>>>>> by
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if (error ==
> >>>>>>>>>>>>>> Errors.*UNKNOWN_PRODUCER_ID*)
> >>>>>>>>>>>>>>> {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>         if
> >>>>>>>> (response.logStartOffset
> >>>>>>>>>> ==
> >>>>>>>>>>>>> -1)
> >>>>>>>>>>>>>> {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // We don't
> >>>>> know
> >>>>>>>> the log
> >>>>>>>>>>>>> start
> >>>>>>>>>>>>>>>> offset
> >>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>> response.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> We
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the request
> >>>>>> until
> >>>>>>>> we
> >>>>>>>>>> get
> >>>>>>>>>>>>> it.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // The
> >>>>>>>>>> UNKNOWN_PRODUCER_ID
> >>>>>>>>>>>>>> error
> >>>>>>>>>>>>>>>> code
> >>>>>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>>>> added
> >>>>>>>>>>>>>>>>>>>>>>>>> along
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProduceResponse which
> >>>>>>>> includes the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             //
> >>>>>> logStartOffset.
> >>>>>>>> So
> >>>>>>>>>> the
> >>>>>>>>>>>>> '-1'
> >>>>>>>>>>>>>>>>>> sentinel
> >>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> backward
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatibility.
> >>>> Instead,
> >>>>> it
> >>>>>>> is
> >>>>>>>>>> possible
> >>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // a broker
> >>>> to
> >>>>>> not
> >>>>>>>> know
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> logStartOffset at
> >>>>>>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returning
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response because
> >>>> the
> >>>>>>>> partition
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // may have
> >>>>>> moved
> >>>>>>>> away
> >>>>>>>>>> from
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> broker
> >>>>>>>>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> time
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error was
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initially raised to
> >>>> the
> >>>>>> time
> >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // response
> >>>>> was
> >>>>>>>> being
> >>>>>>>>>>>>>>> constructed.
> >>>>>>>>>>>>>>>> In
> >>>>>>>>>>>>>>>>>>>>> these
> >>>>>>>>>>>>>>>>>>>>>>>> cases,
> >>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retry the request: we
> >>>>> are
> >>>>>>>>>> guaranteed
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // to
> >>>>> eventually
> >>>>>>>> get a
> >>>>>>>>>>>>>>>> logStartOffset
> >>>>>>>>>>>>>>>>>>> once
> >>>>>>>>>>>>>>>>>>>>>>> things
> >>>>>>>>>>>>>>>>>>>>>>>>>>> settle
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> down.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             return true;
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>         }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>         if
> >>>>>>>>>>>>> (batch.sequenceHasBeenReset()) {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // When the
> >>>>>> first
> >>>>>>>>>> inflight
> >>>>>>>>>>>>>> batch
> >>>>>>>>>>>>>>>>>> fails
> >>>>>>>>>>>>>>>>>>>>> due to
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> truncation
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> then the sequences of
> >>>>> all
> >>>>>>> the
> >>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // in flight
> >>>>>>> batches
> >>>>>>>>>> would
> >>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>>>>>> restarted
> >>>>>>>>>>>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> beginning.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, when those
> >>>>>>> responses
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // come back
> >>>>>> from
> >>>>>>>> the
> >>>>>>>>>>>>> broker,
> >>>>>>>>>>>>>>> they
> >>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>> also
> >>>>>>>>>>>>>>>>>>>>>>>> come
> >>>>>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> >>>>> error.
> >>>>>>> In
> >>>>>>>> this
> >>>>>>>>>>>>> case,
> >>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // reset the
> >>>>>>>> sequence
> >>>>>>>>>>>>> numbers
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> beginning.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             return true;
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>         } else if
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>> (lastAckedOffset(batch.topicPartition).orElse(
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> >>>>>>>>>>>>>>>>>>>>>>>> response.logStartOffset) {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // The head
> >>>> of
> >>>>>> the
> >>>>>>>> log
> >>>>>>>>>> has
> >>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>> removed,
> >>>>>>>>>>>>>>>>>>>>>>> probably
> >>>>>>>>>>>>>>>>>>>>>>>>> due
> >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retention time
> >>>> elapsing.
> >>>>>> In
> >>>>>>>> this
> >>>>>>>>>> case,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // we expect
> >>>>> to
> >>>>>>>> lose the
> >>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>> state.
> >>>>>>>>>>>>>>>>>>>>> For
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer, reset the
> >>>>>>> sequences
> >>>>>>>> of
> >>>>>>>>>> all
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // inflight
> >>>>>>> batches
> >>>>>>>> to
> >>>>>>>>>> be
> >>>>>>>>>>>>> from
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> beginning
> >>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>> retry
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> them,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the transaction
> >>>>> does
> >>>>>>> not
> >>>>>>>>>> need to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // be
> >>>> aborted.
> >>>>>> For
> >>>>>>>> the
> >>>>>>>>>>>>>> idempotent
> >>>>>>>>>>>>>>>>>>>>> producer,
> >>>>>>>>>>>>>>>>>>>>>>> bump
> >>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> avoid
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reusing (sequence,
> >>>>> epoch)
> >>>>>>>> pairs
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             if
> >>>>>>>> (isTransactional()) {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>> txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>> this.producerIdAndEpoch);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             } else {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>> requestEpochBumpForPartition(batch.topicPartition);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             return true;
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>         }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>         if
> >>>>>>> (!isTransactional())
> >>>>>>>> {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // For the
> >>>>>>>> idempotent
> >>>>>>>>>>>>> producer,
> >>>>>>>>>>>>>>>>>> always
> >>>>>>>>>>>>>>>>>>>>> retry
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors. If the batch
> >>>> has
> >>>>>> the
> >>>>>>>>>> current
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // producer
> >>>> ID
> >>>>>> and
> >>>>>>>>>> epoch,
> >>>>>>>>>>>>>>> request a
> >>>>>>>>>>>>>>>>>> bump
> >>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Otherwise
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the
> >>>> produce.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>> requestEpochBumpForPartition(batch.topicPartition);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             return true;
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>         }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I was considering
> >>>>> keeping
> >>>>>>> this
> >>>>>>>>>>>>> behavior —
> >>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>> am
> >>>>>>>>>>>>>>>>>>>>> open
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> simplifying
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We are leaving changes
> >>>>> to
> >>>>>>>> older
> >>>>>>>>>>>>> clients
> >>>>>>>>>>>>>> off
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> table
> >>>>>>>>>>>>>>>>>>>>>>> here
> >>>>>>>>>>>>>>>>>>>>>>>>>> since
> >>>>>>>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> caused
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many issues for
> >>>> clients
> >>>>> in
> >>>>>>> the
> >>>>>>>>>> past.
> >>>>>>>>>>>>>>>> Previously
> >>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>> fatal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> error
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we didn't have the
> >>>>>>> mechanisms
> >>>>>>>> in
> >>>>>>>>>>>>> place to
> >>>>>>>>>>>>>>>>>> detect
> >>>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> legitimate
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case vs some bug or
> >>>> gap
> >>>>> in
> >>>>>>> the
> >>>>>>>>>>>>> protocol.
> >>>>>>>>>>>>>>>>>> Ensuring
> >>>>>>>>>>>>>>>>>>>>> each
> >>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> its
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> own epoch should close
> >>>>>> this
> >>>>>>>> gap.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> And to address Jeff's
> >>>>>> second
> >>>>>>>>>> point:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *does the typical
> >>>>> produce
> >>>>>>>> request
> >>>>>>>>>> path
> >>>>>>>>>>>>>>> append
> >>>>>>>>>>>>>>>>>>>>> records
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>> local
> >>>>>>>>>>>>>>>>>>>>>>>>>>> log
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along*
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *with the
> >>>>>>>> currentTxnFirstOffset
> >>>>>>>>>>>>>>> information?
> >>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand*
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *when the field is
> >>>>> written
> >>>>>>> to
> >>>>>>>>>> disk.*
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, the first produce
> >>>>>>> request
> >>>>>>>>>>>>> populates
> >>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>> field
> >>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>> writes
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> offset
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as part of the record
> >>>>>> batch
> >>>>>>>> and
> >>>>>>>>>> also
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> snapshot.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we reload the records
> >>>> on
> >>>>>>>> restart
> >>>>>>>>>>>>> and/or
> >>>>>>>>>>>>>>>>>>>>> reassignment,
> >>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>> repopulate
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> field with the
> >>>> snapshot
> >>>>>> from
> >>>>>>>> disk
> >>>>>>>>>>>>> along
> >>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> rest
> >>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if there
> >>>> are
> >>>>>>>> further
> >>>>>>>>>>>>> comments
> >>>>>>>>>>>>>>>>>> and/or
> >>>>>>>>>>>>>>>>>>>>>>> questions.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> >>>> at
> >>>>>> 9:00
> >>>>>>>> PM
> >>>>>>>>>> Jeff
> >>>>>>>>>>>>> Kim
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>> <jeff.kim@confluent.io.invalid
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP! I
> >>>>>> have
> >>>>>>>> two
> >>>>>>>>>>>>>> questions:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1) For new clients,
> >>>> we
> >>>>>> can
> >>>>>>>> once
> >>>>>>>>>> again
> >>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>> error
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for sequences
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that are non-zero
> >>>> when
> >>>>>>> there
> >>>>>>>> is
> >>>>>>>>>> no
> >>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>>>> present
> >>>>>>>>>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This will indicate we
> >>>>>>> missed
> >>>>>>>> the
> >>>>>>>>>> 0
> >>>>>>>>>>>>>>> sequence
> >>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>>> yet
> >>>>>>>>>>>>>>>>>>>>>>>>>>> want
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the log.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would like to
> >>>>>> understand
> >>>>>>>> the
> >>>>>>>>>>>>> current
> >>>>>>>>>>>>>>>>>> behavior
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>> handle
> >>>>>>>>>>>>>>>>>>>>>>>>>> older
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and if there are any
> >>>>>>> changes
> >>>>>>>> we
> >>>>>>>>>> are
> >>>>>>>>>>>>>>> making.
> >>>>>>>>>>>>>>>>>> Maybe
> >>>>>>>>>>>>>>>>>>>>> I'm
> >>>>>>>>>>>>>>>>>>>>>>>>> missing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but we would want to
> >>>>>>> identify
> >>>>>>>>>>>>> whether we
> >>>>>>>>>>>>>>>>>> missed
> >>>>>>>>>>>>>>>>>>>>> the 0
> >>>>>>>>>>>>>>>>>>>>>>>>> sequence
> >>>>>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients, no?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2) Upon returning
> >>>> from
> >>>>>> the
> >>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>> coordinator, we
> >>>>>>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>> set
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as ongoing on the
> >>>>> leader
> >>>>>> by
> >>>>>>>>>>>>> populating
> >>>>>>>>>>>>>>>>>>>>>>>> currentTxnFirstOffset
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through the typical
> >>>>>> produce
> >>>>>>>>>> request
> >>>>>>>>>>>>>>>> handling.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does the typical
> >>>>> produce
> >>>>>>>> request
> >>>>>>>>>> path
> >>>>>>>>>>>>>>> append
> >>>>>>>>>>>>>>>>>>>>> records
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>> local
> >>>>>>>>>>>>>>>>>>>>>>>>>>> log
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> >>>>>>>> currentTxnFirstOffset
> >>>>>>>>>>>>>>> information?
> >>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when the field is
> >>>>> written
> >>>>>>> to
> >>>>>>>>>> disk.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeff
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> >>>> at
> >>>>>>> 4:44
> >>>>>>>> PM
> >>>>>>>>>> Artem
> >>>>>>>>>>>>>>>> Livshits
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
> >>>>> alivshits@confluent.io
> >>>>>>>> .invalid>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for the
> >>>> KIP.
> >>>>>> I
> >>>>>>>> have
> >>>>>>>>>> one
> >>>>>>>>>>>>>>>> question.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 5) For new clients,
> >>>> we
> >>>>>> can
> >>>>>>>> once
> >>>>>>>>>>>>> again
> >>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>> error
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe we had
> >>>>>> problems
> >>>>>>>> in the
> >>>>>>>>>>>>> past
> >>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>> returning
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because it was
> >>>>>> considered
> >>>>>>>> fatal
> >>>>>>>>>> and
> >>>>>>>>>>>>>>>> required
> >>>>>>>>>>>>>>>>>>>>> client
> >>>>>>>>>>>>>>>>>>>>>>>>> restart.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> It
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good to spell out
> >>>> the
> >>>>>> new
> >>>>>>>> client
> >>>>>>>>>>>>>> behavior
> >>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>> receives
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> error.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> >>>>> at
> >>>>>>>> 10:00 AM
> >>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>> Olshan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>> <jo...@confluent.io.invalid>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
> >>>>>> look
> >>>>>>>>>> Matthias.
> >>>>>>>>>>>>>> I've
> >>>>>>>>>>>>>>>>>> tried
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>> answer
> >>>>>>>>>>>>>>>>>>>>>>>>> your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> questions
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 10)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Right — so the
> >>>>> hanging
> >>>>>>>>>> transaction
> >>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>> occurs
> >>>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> late
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> come in and the
> >>>>>> partition
> >>>>>>>> is
> >>>>>>>>>> never
> >>>>>>>>>>>>>> added
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>> again.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> If
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never add the
> >>>>> partition
> >>>>>>> to
> >>>>>>>> a
> >>>>>>>>>>>>>>> transaction,
> >>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>> never
> >>>>>>>>>>>>>>>>>>>>>>>>>> write
> >>>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> marker
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never advance the
> >>>>> LSO.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do end up
> >>>>> adding
> >>>>>>> the
> >>>>>>>>>>>>> partition
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>> (I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> suppose
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happen before or
> >>>>> after
> >>>>>>> the
> >>>>>>>> late
> >>>>>>>>>>>>>> message
> >>>>>>>>>>>>>>>>>> comes
> >>>>>>>>>>>>>>>>>>> in)
> >>>>>>>>>>>>>>>>>>>>>> then
> >>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> include
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late message in the
> >>>>>> next
> >>>>>>>>>>>>> (incorrect)
> >>>>>>>>>>>>>>>>>>> transaction.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> So perhaps it is
> >>>>>> clearer
> >>>>>>> to
> >>>>>>>>>> make
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> distinction
> >>>>>>>>>>>>>>>>>>>>>>> between
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> eventually get
> >>>> added
> >>>>> to
> >>>>>>> the
> >>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>> (but
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> wrong
> >>>>>>>>>>>>>>>>>>>>>>>>> one)
> >>>>>>>>>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that never get
> >>>> added
> >>>>>> and
> >>>>>>>> become
> >>>>>>>>>>>>>> hanging.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side
> >>>>> change
> >>>>>>> for
> >>>>>>>> 2 is
> >>>>>>>>>>>>>> removing
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> addPartitions
> >>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need
> >>>>> to
> >>>>>>> make
> >>>>>>>>>> this
> >>>>>>>>>>>>> from
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> txn
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In my opinion, the
> >>>>>> issue
> >>>>>>>> with
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> >>>>>>>>>>>>>>>>>>>>>>>> call
> >>>>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is that we don't
> >>>> have
> >>>>>> the
> >>>>>>>> epoch
> >>>>>>>>>>>>> bump,
> >>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> belongs to the
> >>>>> previous
> >>>>>>>>>>>>> transaction or
> >>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>> one.
> >>>>>>>>>>>>>>>>>>>>> We
> >>>>>>>>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition has been
> >>>>>> added
> >>>>>>> to
> >>>>>>>>>> this
> >>>>>>>>>>>>>>>>>> transaction.
> >>>>>>>>>>>>>>>>>>> Of
> >>>>>>>>>>>>>>>>>>>>>>> course,
> >>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> means
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't completely
> >>>>> cover
> >>>>>>> the
> >>>>>>>> case
> >>>>>>>>>>>>> where
> >>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>> have a
> >>>>>>>>>>>>>>>>>>>>>> really
> >>>>>>>>>>>>>>>>>>>>>>>>> late
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have added the
> >>>>>> partition
> >>>>>>> to
> >>>>>>>>>> the new
> >>>>>>>>>>>>>>>>>>> transaction,
> >>>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>> that's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something we will
> >>>>> need
> >>>>>>> the
> >>>>>>>> new
> >>>>>>>>>>>>> clients
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> cover.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is
> >>>>> ongoing
> >>>>>> =
> >>>>>>>>>> partition
> >>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>> added to
> >>>>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>> via
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn.
> >>>>> We
> >>>>>>>> check
> >>>>>>>>>> this
> >>>>>>>>>>>>> with
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this
> >>>>> wasn't
> >>>>>>>>>> sufficiently
> >>>>>>>>>>>>>>>>>> explained
> >>>>>>>>>>>>>>>>>>>>> here:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is
> >>>> that
> >>>>>> if
> >>>>>>>> any
> >>>>>>>>>>>>> messages
> >>>>>>>>>>>>>>>>>> somehow
> >>>>>>>>>>>>>>>>>>>>> come
> >>>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the
> >>>>> producer,
> >>>>>>> they
> >>>>>>>>>> will be
> >>>>>>>>>>>>>>>> fenced.
> >>>>>>>>>>>>>>>>>>>>> However,
> >>>>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it
> >>>> can
> >>>>> be
> >>>>>>>>>> discussed
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 50)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It should be
> >>>>>> synchronous
> >>>>>>>>>> because
> >>>>>>>>>>>>> if we
> >>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>> event
> >>>>>>>>>>>>>>>>>>>>>>>> (ie,
> >>>>>>>>>>>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> error)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes us to need
> >>>> to
> >>>>>>> abort
> >>>>>>>> the
> >>>>>>>>>>>>>>>> transaction,
> >>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions to send
> >>>>>>>> transaction
> >>>>>>>>>>>>> markers
> >>>>>>>>>>>>>>> to.
> >>>>>>>>>>>>>>>>>> We
> >>>>>>>>>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we added them to
> >>>> the
> >>>>>>>>>> coordinator
> >>>>>>>>>>>>> via
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Previously we have
> >>>>> had
> >>>>>>>>>> asynchronous
> >>>>>>>>>>>>>>> calls
> >>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> past
> >>>>>>>>>>>>>>>>>>>>>>>> (ie,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> writing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit markers when
> >>>>> the
> >>>>>>>>>>>>> transaction is
> >>>>>>>>>>>>>>>>>>> completed)
> >>>>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>> often
> >>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes confusion as
> >>>>> we
> >>>>>>>> need to
> >>>>>>>>>> wait
> >>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>>> operations
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> complete.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing commit
> >>>>> markers
> >>>>>>>> case,
> >>>>>>>>>>>>> clients
> >>>>>>>>>>>>>>> often
> >>>>>>>>>>>>>>>>>> see
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>> CONCURRENT_TRANSACTIONs
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error messages and
> >>>>> that
> >>>>>>>> can be
> >>>>>>>>>>>>>>> confusing.
> >>>>>>>>>>>>>>>>>> For
> >>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>> reason,
> >>>>>>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> may
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simpler to just
> >>>> have
> >>>>>>>>>> synchronous
> >>>>>>>>>>>>>> calls —
> >>>>>>>>>>>>>>>>>>>>> especially
> >>>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> block
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some operation's
> >>>>>>> completion
> >>>>>>>>>> anyway
> >>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>> start
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> next
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. And
> >>>>> yes, I
> >>>>>>>> meant
> >>>>>>>>>>>>>>>> coordinator. I
> >>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>> fix
> >>>>>>>>>>>>>>>>>>>>>>>>> that.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are
> >>>> checking
> >>>>> if
> >>>>>>> the
> >>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>> ongoing,
> >>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the
> >>>> leader
> >>>>>>>> partition
> >>>>>>>>>> to
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for
> >>>>> this
> >>>>>>>>>> message to
> >>>>>>>>>>>>>> come
> >>>>>>>>>>>>>>>>>> back,
> >>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>> theory
> >>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
> >>>>>> that
> >>>>>>>> would
> >>>>>>>>>> make
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>>>>>>>>> result
> >>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why
> >>>> we
> >>>>>> can
> >>>>>>>> check
> >>>>>>>>>> the
> >>>>>>>>>>>>>>> leader
> >>>>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>> write
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm happy to update
> >>>>> the
> >>>>>>>> KIP if
> >>>>>>>>>>>>> some of
> >>>>>>>>>>>>>>>> these
> >>>>>>>>>>>>>>>>>>>>> things
> >>>>>>>>>>>>>>>>>>>>>>> were
> >>>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Nov 21,
> >>>> 2022
> >>>>> at
> >>>>>>>> 7:11 PM
> >>>>>>>>>>>>>> Matthias
> >>>>>>>>>>>>>>>> J.
> >>>>>>>>>>>>>>>>>>> Sax <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the
> >>>> KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Couple of
> >>>>>> clarification
> >>>>>>>>>> questions
> >>>>>>>>>>>>> (I
> >>>>>>>>>>>>>> am
> >>>>>>>>>>>>>>>>>> not a
> >>>>>>>>>>>>>>>>>>>>>> broker
> >>>>>>>>>>>>>>>>>>>>>>>>> expert
> >>>>>>>>>>>>>>>>>>>>>>>>>>> do
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some question are
> >>>>>>> obvious
> >>>>>>>> for
> >>>>>>>>>>>>> others,
> >>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>> me
> >>>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>>> my
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> lack
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker knowledge).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (10)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What happens if
> >>>> the
> >>>>>>>> message
> >>>>>>>>>> come
> >>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> next
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request? It seems
> >>>>> the
> >>>>>>>> broker
> >>>>>>>>>>>>> hosting
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> data
> >>>>>>>>>>>>>>>>>>>>>>>> partitions
> >>>>>>>>>>>>>>>>>>>>>>>>>>> won't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anything about it
> >>>>> and
> >>>>>>>> append
> >>>>>>>>>> it to
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> partition,
> >>>>>>>>>>>>>>>>>>>>>>> too?
> >>>>>>>>>>>>>>>>>>>>>>>>> What
> >>>>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> difference between
> >>>>>> both
> >>>>>>>> cases?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, it seems a
> >>>> TX
> >>>>>>> would
> >>>>>>>> only
> >>>>>>>>>>>>> hang,
> >>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>>>>>> is no
> >>>>>>>>>>>>>>>>>>>>>>>>>> following
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> TX
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> either committer
> >>>> or
> >>>>>>>> aborted?
> >>>>>>>>>> Thus,
> >>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>>>> above,
> >>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> TX
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> might
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually not hang
> >>>>> (of
> >>>>>>>> course,
> >>>>>>>>>> we
> >>>>>>>>>>>>>> might
> >>>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>> EOS
> >>>>>>>>>>>>>>>>>>>>>>>>> violation
> >>>>>>>>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX was aborted and
> >>>>> the
> >>>>>>>> second
> >>>>>>>>>>>>>>> committed,
> >>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>>>>> way
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> around).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (20)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Of course, 1 and
> >>>> 2
> >>>>>>>> require
> >>>>>>>>>>>>>> client-side
> >>>>>>>>>>>>>>>>>>>>> changes, so
> >>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>> older
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> those approaches
> >>>>> won’t
> >>>>>>>> apply.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For (1) I
> >>>> understand
> >>>>>>> why a
> >>>>>>>>>> client
> >>>>>>>>>>>>>>> change
> >>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>> necessary,
> >>>>>>>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we need a client
> >>>>>> change
> >>>>>>>> for
> >>>>>>>>>> (2).
> >>>>>>>>>>>>> Can
> >>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>> elaborate?
> >>>>>>>>>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>>>>>>>>>>> Later
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> explain
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that we should
> >>>> send
> >>>>> a
> >>>>>>>>>>>>>>>>>>>>> DescribeTransactionRequest,
> >>>>>>>>>>>>>>>>>>>>>>> but I
> >>>>>>>>>>>>>>>>>>>>>>>>> am
> >>>>>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Can't we not just
> >>>> do
> >>>>>> an
> >>>>>>>>>> implicit
> >>>>>>>>>>>>>>>>>>>>> AddPartiitonToTx,
> >>>>>>>>>>>>>>>>>>>>>>> too?
> >>>>>>>>>>>>>>>>>>>>>>>>> If
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> old
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer correctly
> >>>>>>>> registered
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>>>>>> already,
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can just ignore it
> >>>>> as
> >>>>>>>> it's an
> >>>>>>>>>>>>>>> idempotent
> >>>>>>>>>>>>>>>>>>>>> operation?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (30)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To cover older
> >>>>>> clients,
> >>>>>>>> we
> >>>>>>>>>> will
> >>>>>>>>>>>>>>> ensure a
> >>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> ongoing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we write to a
> >>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Not sure what you
> >>>>> mean
> >>>>>>> by
> >>>>>>>>>> this?
> >>>>>>>>>>>>> Can
> >>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>> elaborate?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (40)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [the
> >>>>> TX-coordinator]
> >>>>>>> will
> >>>>>>>>>> write
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> prepare
> >>>>>>>>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>>>>>>>> message
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch and send
> >>>>>>>>>>>>> WriteTxnMarkerRequests
> >>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> bumped
> >>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why do we use the
> >>>>>> bumped
> >>>>>>>>>> epoch for
> >>>>>>>>>>>>>>> both?
> >>>>>>>>>>>>>>>> It
> >>>>>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> intuitive
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the current epoch,
> >>>>> and
> >>>>>>>> only
> >>>>>>>>>> return
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> bumped
> >>>>>>>>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (50) "Implicit
> >>>>>>>>>>>>>>> AddPartitionToTransaction"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why does the
> >>>>>> implicitly
> >>>>>>>> sent
> >>>>>>>>>>>>> request
> >>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>> synchronous?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also says
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in case we need
> >>>> to
> >>>>>>> abort
> >>>>>>>> and
> >>>>>>>>>>>>> need to
> >>>>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>>>>>>>> partitions
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What do you mean
> >>>> by
> >>>>>>> this?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we don’t want to
> >>>>>> write
> >>>>>>>> to it
> >>>>>>>>>>>>> before
> >>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> manager
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Do you mean
> >>>>>>> TX-coordinator
> >>>>>>>>>>>>> instead of
> >>>>>>>>>>>>>>>>>>> "manager"?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (60)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For older clients
> >>>>> and
> >>>>>>>> ensuring
> >>>>>>>>>>>>> that
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> TX
> >>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>> ongoing,
> >>>>>>>>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> describe a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> race condition. I
> >>>> am
> >>>>>> not
> >>>>>>>> sure
> >>>>>>>>>> if I
> >>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>> follow
> >>>>>>>>>>>>>>>>>>>>> here.
> >>>>>>>>>>>>>>>>>>>>>>> Can
> >>>>>>>>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> elaborate?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/18/22 1:21
> >>>> PM,
> >>>>>>>> Justine
> >>>>>>>>>>>>> Olshan
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey all!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to
> >>>> start a
> >>>>>>>>>> discussion
> >>>>>>>>>>>>> on my
> >>>>>>>>>>>>>>>>>> proposal
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>> add
> >>>>>>>>>>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checks on
> >>>>>> transactions
> >>>>>>> to
> >>>>>>>>>> avoid
> >>>>>>>>>>>>>>> hanging
> >>>>>>>>>>>>>>>>>>>>>>> transactions.
> >>>>>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an issue for some
> >>>>>> time,
> >>>>>>>> so I
> >>>>>>>>>>>>> really
> >>>>>>>>>>>>>>> hope
> >>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> helpful
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users of EOS.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The KIP includes
> >>>>>>> changes
> >>>>>>>> that
> >>>>>>>>>>>>> will
> >>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>> compatible
> >>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>> old
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> changes to
> >>>> improve
> >>>>>>>>>> performance
> >>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>> correctness
> >>>>>>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>>>>>> clients.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Please take a
> >>>> look
> >>>>>> and
> >>>>>>>> leave
> >>>>>>>>>> any
> >>>>>>>>>>>>>>>> comments
> >>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>> may
> >>>>>>>>>>>>>>>>>>>>>>>> have!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> JIRA:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>> https://issues.apache.org/jira/browse/KAFKA-14402
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by "Matthias J. Sax" <mj...@apache.org>.
> would it build an offset map with just the latest timestamp for a key?

Cannot remember the details without reading the KIP, but yes, something 
like this (I believe it actually needs to track both, offset and 
timestamp per key).

> I wonder if ordering assumptions are baked in there, why not use offset-based compaction.

The use case is a compacted topic that does contain out-of-order data. 
If you set key k1=v1 @ 5 offset 100 and later key1 = v0 @ 3 at offset 
200 we want to cleanup v0 with higher offset because it's out-of-order 
based on time, but keep v1 what is the actual latest version of k1.


> I was also not aware of this "guarantee" with regards to broker side time.

As already said: I am not sure if it's a public contract, but based on 
my experience, people might reply on it as "implicit contract". -- Maybe 
somebody else knows if it's public or not, and if it would be ok to 
"break" it.

> Let me know if you have any concerns here.

My understanding is: While we cannot make an offset-order guarantee for 
interleaved writes of different producer, if the topic is configures 
with "append_time", we "guarantee" (cf. my comment above") timestamp 
order... If that's the case, it would be an issue if we break this 
"guarantee".

I am not sure when the broker sets the timestamp for "append_time" 
config? If we do it before putting the request into purgatory, we have a 
problem. However, if we set the timestamp when we actually process the 
request and do the actual append, it seems there is no issue, as the 
request that was waiting in purgatory get the "newest" timestamp and 
thus cannot introduce out-of-order data.


-Matthias


On 1/24/23 10:44 AM, Justine Olshan wrote:
> Hey Matthias,
> 
> I have actually never heard of KIP-280 so thanks for bringing it up. That
> seems interesting. I wonder how it would work though -- would it build an
> offset map with just the latest timestamp for a key? I wonder if ordering
> assumptions are baked in there, why not use offset-based compaction.
> 
> I was also not aware of this "guarantee" with regards to broker side time.
> I think that we can do in order handling for a given producer, but not
> across all producers. However, we can't guarantee that anyway.
> 
> Let me know if you have any concerns here.
> 
> Thanks,
> Justine
> 
> On Mon, Jan 23, 2023 at 6:33 PM Matthias J. Sax <mj...@apache.org> wrote:
> 
>> Just a side note about Guozhang comments about timestamps.
>>
>> If the producer sets the timestamp, putting the record into purgatory
>> seems not to be an issue (as already said: for this case we don't
>> guarantee timestamp order between writes of different producers anyway).
>> However, if the broker sets the timestamp, the expectation is that there
>> is no out-of-order data in the partition ever; if we would introduce
>> out-of-order data for this case (for interleaved writes of different
>> producers), it seems we would violate the current contract? (To be fair:
>> I don't know if that's an official contract, but I assume people rely on
>> this behavior -- and it "advertised" in many public talks...)
>>
>> About compaction: there is actually KIP-280 that adds timestamp based
>> compaction what is a very useful feature for Kafka Streams with regard
>> to out-of-order data handling. So the impact if we introduce
>> out-of-order data could be larger scoped.
>>
>>
>> -Matthias
>>
>>
>> On 1/20/23 4:48 PM, Justine Olshan wrote:
>>> Hey Artem,
>>>
>>> I see there is a check for transactional producers. I'm wondering if we
>>> don't handle the epoch overflow case. I'm also not sure it will be a huge
>>> issue to extend to transactional producers, but maybe I'm missing
>> something.
>>>
>>> As for the recovery path -- I think Guozhang's point was if we have a bad
>>> client that repeatedly tries to produce without adding to the transaction
>>> we would do the following:
>>> a) if not fatal, we just fail the produce request over and over
>>> b) if fatal, we fence the producer
>>>
>>> Here with B, the issue with the client would be made clear more quickly.
>> I
>>> suppose there are some intermediate cases where the issue only occurs
>>> sometimes, but I wonder if we should consider how to recover with clients
>>> who don't behave as expected anyway.
>>>
>>> I think there is a place for the abortable error that we are adding --
>> just
>>> abort and try again. But I think there are also some cases where trying
>> to
>>> recover overcomplicates some logic. Especially if we are considering
>> older
>>> clients -- there I'm not sure if there's a ton we can do besides fail the
>>> batch or fence the producer. With newer clients, we can consider more
>>> options for what can just be recovered after aborting. But epochs might
>> be
>>> a hard one unless we also want to reset producer ID.
>>>
>>> Thanks,
>>> Justine
>>>
>>>
>>>
>>> On Fri, Jan 20, 2023 at 3:59 PM Artem Livshits
>>> <al...@confluent.io.invalid> wrote:
>>>
>>>>>    besides the poorly written client case
>>>>
>>>> A poorly written client could create a lot of grief to people who run
>> Kafka
>>>> brokers :-), so when deciding to make an error fatal I would see if
>> there
>>>> is a reasonable recovery path rather than how often it could happen.
>> If we
>>>> have solid implementation of transactions (which I hope we'll do as a
>>>> result of this KIP), it would help to recover from a large class of
>> errors
>>>> by just aborting a transaction, even if the cause of error is a race
>>>> condition or etc.
>>>>
>>>> -Artem
>>>>
>>>> On Fri, Jan 20, 2023 at 3:26 PM Justine Olshan
>>>> <jo...@confluent.io.invalid>
>>>> wrote:
>>>>
>>>>> Artem --
>>>>> I guess the discussion path we were going down is when we expect to see
>>>>> this error. I mentioned that it was hard to come up with cases for when
>>>> the
>>>>> producer would still be around to receive the error besides the poorly
>>>>> written client case.
>>>>> If we don't expect to have a producer to receive the response, it sort
>> of
>>>>> makes sense for it to be fatal.
>>>>>
>>>>> I had some discussion with Jason offline about the epoch being off
>> cases
>>>>> and I'm not sure we could find a ton (outside of produce requests)
>> where
>>>> we
>>>>> could/should recover. I'd be happy to hear some examples though, maybe
>>>> I'm
>>>>> missing something.
>>>>>
>>>>> Thanks,
>>>>> Justine
>>>>>
>>>>> On Fri, Jan 20, 2023 at 3:19 PM Artem Livshits
>>>>> <al...@confluent.io.invalid> wrote:
>>>>>
>>>>>> In general, I'd like to avoid fatal errors as much as possible, in
>> some
>>>>>> sense fatal errors just push out recovery logic to the application
>>>> which
>>>>>> either complicates the application or leads to disruption (we've seen
>>>>> cases
>>>>>> when a transient broker error could lead to work stoppage when
>>>>> applications
>>>>>> need to be manually restarted).  I think we should strive to define
>>>>>> recovery logic for most errors (and/or encapsulate it in the Kafka
>>>> client
>>>>>> as much as possible).
>>>>>>
>>>>>> One benefit of transactions is that they simplify recovery from
>> errors,
>>>>>> pretty much any error (that's not handled transparently by retries in
>>>>> Kafka
>>>>>> client) can be handled by the application via aborting the transaction
>>>>> and
>>>>>> repeating the transactional logic again.  One tricky error is an error
>>>>>> during commit, because we don't know the outcome.  For commit errors,
>>>> the
>>>>>> recommendation should be to retry the commit until it returns the
>>>>> specific
>>>>>> result (committed or aborted).
>>>>>>
>>>>>> -Artem
>>>>>>
>>>>>> On Fri, Jan 20, 2023 at 2:52 PM Justine Olshan
>>>>>> <jo...@confluent.io.invalid>
>>>>>> wrote:
>>>>>>
>>>>>>> That's a fair point about other clients.
>>>>>>>
>>>>>>> I think the abortable error case is interesting because I'm curious
>>>> how
>>>>>>> other clients would handle this. I assume they would need to
>>>> implement
>>>>>>> handling for the error code unless they did something like "any
>>>> unknown
>>>>>>> error codes/any codes that aren't x,y,z are retriable." I would hope
>>>>> that
>>>>>>> unknown error codes were fatal, and if the code was implemented it
>>>>> would
>>>>>>> abort the transaction. But I will think on this too.
>>>>>>>
>>>>>>> As for InvalidRecord -- you mentioned it was not fatal, but I'm
>>>> taking
>>>>> a
>>>>>>> look through the code. We would see this on handling the produce
>>>>>> response.
>>>>>>> If I recall correctly, we check if errors are retriable. I think this
>>>>>> error
>>>>>>> would not be retriable. But I guess the concern here is that it is
>>>> not
>>>>>>> enough for just that batch to fail. I guess I hadn't considered fully
>>>>>>> fencing the old producer but there are valid arguments here why we
>>>>> would
>>>>>>> want to.
>>>>>>>
>>>>>>> Thanks,
>>>>>>> Justine
>>>>>>>
>>>>>>> On Fri, Jan 20, 2023 at 2:35 PM Guozhang Wang <
>>>>>> guozhang.wang.us@gmail.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Thanks Justine for the replies! I agree with most of your thoughts.
>>>>>>>>
>>>>>>>> Just for 3/7), though I agree for our own AK producer, since we do
>>>>>>>> "nextRequest(boolean hasIncompleteBatches)", we guarantee the
>>>> end-txn
>>>>>>>> would not be sent until we've effectively flushed, but I was
>>>>> referring
>>>>>>>> to any future bugs or other buggy clients that the same client may
>>>>> get
>>>>>>>> into this situation, in which case we should give the client a
>>>> clear
>>>>>>>> msg that "you did something wrong, and hence now you should fatally
>>>>>>>> close yourself". What I'm concerned about is that, by seeing an
>>>>>>>> "abortable error" or in some rare cases an "invalid record", the
>>>>>>>> client could not realize "something that's really bad happened". So
>>>>>>>> it's not about adding a new error, it's mainly about those real
>>>> buggy
>>>>>>>> situations causing such "should never happen" cases, the errors
>>>>> return
>>>>>>>> would not be informative enough.
>>>>>>>>
>>>>>>>> Thinking in other ways, if we believe that for most cases such
>>>> error
>>>>>>>> codes would not reach the original clients since they would be
>>>>>>>> disconnected or even gone by that time, and only in some rare cases
>>>>>>>> they would still be seen by the sending clients, then why not make
>>>>>>>> them more fatal and more specific than generic.
>>>>>>>>
>>>>>>>> Guozhang
>>>>>>>>
>>>>>>>> On Fri, Jan 20, 2023 at 1:59 PM Justine Olshan
>>>>>>>> <jo...@confluent.io.invalid> wrote:
>>>>>>>>>
>>>>>>>>> Hey Guozhang. Thanks for taking a look and for the detailed
>>>>> comments!
>>>>>>>> I'll
>>>>>>>>> do my best to address below.
>>>>>>>>>
>>>>>>>>> 1. I see what you are saying here, but I think I need to look
>>>>> through
>>>>>>> the
>>>>>>>>> sequence of events you mention. Typically we've seen this issue
>>>> in
>>>>> a
>>>>>>> few
>>>>>>>>> cases.
>>>>>>>>>
>>>>>>>>>    One is when we have a producer disconnect when trying to
>>>> produce.
>>>>>>>>> Typically in these cases, we abort the transaction. We've seen
>>>> that
>>>>>>> after
>>>>>>>>> the markers are written, the disconnection can sometimes cause
>>>> the
>>>>>>>> request
>>>>>>>>> to get flushed to the broker. In this case, we don't need client
>>>>>>> handling
>>>>>>>>> because the producer we are responding to is gone. We just needed
>>>>> to
>>>>>>> make
>>>>>>>>> sure we didn't write to the log on the broker side. I'm trying to
>>>>>> think
>>>>>>>> of
>>>>>>>>> a case where we do have the client to return to. I'd think the
>>>> same
>>>>>>>> client
>>>>>>>>> couldn't progress to committing the transaction unless the
>>>> produce
>>>>>>>> request
>>>>>>>>> returned right? Of course, there is the incorrectly written
>>>> clients
>>>>>>> case.
>>>>>>>>> I'll think on this a bit more and let you know if I come up with
>>>>>>> another
>>>>>>>>> scenario when we would return to an active client when the
>>>>>> transaction
>>>>>>> is
>>>>>>>>> no longer ongoing.
>>>>>>>>>
>>>>>>>>> I was not aware that we checked the result of a send after we
>>>>> commit
>>>>>>>>> though. I'll need to look into that a bit more.
>>>>>>>>>
>>>>>>>>> 2. There were some questions about this in the discussion. The
>>>> plan
>>>>>> is
>>>>>>> to
>>>>>>>>> handle overflow with the mechanism we currently have in the
>>>>> producer.
>>>>>>> If
>>>>>>>> we
>>>>>>>>> try to bump and the epoch will overflow, we actually allocate a
>>>> new
>>>>>>>>> producer ID. I need to confirm the fencing logic on the last
>>>> epoch
>>>>>> (ie,
>>>>>>>> we
>>>>>>>>> probably shouldn't allow any records to be produced with the
>>>> final
>>>>>>> epoch
>>>>>>>>> since we can never properly fence that one).
>>>>>>>>>
>>>>>>>>> 3. I can agree with you that the current error handling is
>>>> messy. I
>>>>>>>> recall
>>>>>>>>> taking a look at your KIP a while back, but I think I mostly saw
>>>>> the
>>>>>>>>> section about how the errors were wrapped. Maybe I need to take
>>>>>> another
>>>>>>>>> look. As for abortable error, the idea was that the handling
>>>> would
>>>>> be
>>>>>>>>> simple -- if this error is seen, the transaction should be
>>>> aborted
>>>>> --
>>>>>>> no
>>>>>>>>> other logic about previous state or requests necessary. Is your
>>>>>> concern
>>>>>>>>> simply about adding new errors? We were hoping to have an error
>>>>> that
>>>>>>>> would
>>>>>>>>> have one meaning and many of the current errors have a history of
>>>>>>> meaning
>>>>>>>>> different things on different client versions. That was the main
>>>>>>>> motivation
>>>>>>>>> for adding a new error.
>>>>>>>>>
>>>>>>>>> 4. This is a good point about record timestamp reordering.
>>>>> Timestamps
>>>>>>>> don't
>>>>>>>>> affect compaction, but they do affect retention deletion. For
>>>> that,
>>>>>>> kafka
>>>>>>>>> considers the largest timestamp in the segment, so I think a
>>>> small
>>>>>>> amount
>>>>>>>>> of reordering (hopefully on the order of milliseconds or even
>>>>>> seconds)
>>>>>>>> will
>>>>>>>>> be ok. We take timestamps from clients so there is already a
>>>>>>> possibility
>>>>>>>>> for some drift and non-monotonically increasing timestamps.
>>>>>>>>>
>>>>>>>>> 5. Thanks for catching. The error is there, but it's actually
>>>> that
>>>>>>> those
>>>>>>>>> fields should be 4+! Due to how the message generator works, I
>>>>>> actually
>>>>>>>>> have to redefine those fields inside the
>>>>>>> `"AddPartitionsToTxnTransaction`
>>>>>>>>> block for it to build correctly. I'll fix it to be correct.
>>>>>>>>>
>>>>>>>>> 6. Correct -- we will only add the request to purgatory if the
>>>>> cache
>>>>>>> has
>>>>>>>> no
>>>>>>>>> ongoing transaction. I can change the wording to make that
>>>> clearer
>>>>>> that
>>>>>>>> we
>>>>>>>>> only place the request in purgatory if we need to contact the
>>>>>>> transaction
>>>>>>>>> coordinator.
>>>>>>>>>
>>>>>>>>> 7. We did take a look at some of the errors and it was hard to
>>>> come
>>>>>> up
>>>>>>>> with
>>>>>>>>> a good one. I agree that InvalidTxnStateException is ideal except
>>>>> for
>>>>>>> the
>>>>>>>>> fact that it hasn't been returned on Produce requests before. The
>>>>>> error
>>>>>>>>> handling for clients is a bit vague (which is why I opened
>>>>>> KAFKA-14439
>>>>>>>>> <https://issues.apache.org/jira/browse/KAFKA-14439>), but the
>>>>>> decision
>>>>>>>> we
>>>>>>>>> made here was to only return errors that have been previously
>>>>>> returned
>>>>>>> to
>>>>>>>>> producers. As for not being fatal, I think part of the theory was
>>>>>> that
>>>>>>> in
>>>>>>>>> many cases, the producer would be disconnected. (See point 1) and
>>>>>> this
>>>>>>>>> would just be an error to return from the server. I did plan to
>>>>> think
>>>>>>>> about
>>>>>>>>> other cases, so let me know if you think of any as well!
>>>>>>>>>
>>>>>>>>> Lots to say! Let me know if you have further thoughts!
>>>>>>>>> Justine
>>>>>>>>>
>>>>>>>>> On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang <
>>>>>>>> guozhang.wang.us@gmail.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Hello Justine,
>>>>>>>>>>
>>>>>>>>>> Thanks for the great write-up! I made a quick pass through it
>>>> and
>>>>>>> here
>>>>>>>>>> are some thoughts (I have not been able to read through this
>>>>> thread
>>>>>>> so
>>>>>>>>>> pardon me if they have overlapped or subsumed by previous
>>>>>> comments):
>>>>>>>>>>
>>>>>>>>>> First are some meta ones:
>>>>>>>>>>
>>>>>>>>>> 1. I think we need to also improve the client's experience once
>>>>> we
>>>>>>>>>> have this defence in place. More concretely, say a user's
>>>>> producer
>>>>>>>>>> code is like following:
>>>>>>>>>>
>>>>>>>>>> future = producer.send();
>>>>>>>>>> // producer.flush();
>>>>>>>>>> producer.commitTransaction();
>>>>>>>>>> future.get();
>>>>>>>>>>
>>>>>>>>>> Which resulted in the order of a) produce-request sent by
>>>>> producer,
>>>>>>> b)
>>>>>>>>>> end-txn-request sent by producer, c) end-txn-response sent
>>>> back,
>>>>> d)
>>>>>>>>>> txn-marker-request sent from coordinator to partition leader,
>>>> e)
>>>>>>>>>> produce-request finally received by the partition leader,
>>>> before
>>>>>> this
>>>>>>>>>> KIP e) step would be accepted causing a dangling txn; now it
>>>>> would
>>>>>> be
>>>>>>>>>> rejected in step e) which is good. But from the client's point
>>>> of
>>>>>>> view
>>>>>>>>>> now it becomes confusing since the `commitTransaction()`
>>>> returns
>>>>>>>>>> successfully, but the "future" throws an invalid-epoch error,
>>>> and
>>>>>>> they
>>>>>>>>>> are not sure if the transaction did succeed or not. In fact, it
>>>>>>>>>> "partially succeeded" with some msgs being rejected but others
>>>>>>>>>> committed successfully.
>>>>>>>>>>
>>>>>>>>>> Of course the easy way to avoid this is, always call
>>>>>>>>>> "producer.flush()" before commitTxn and that's what we do
>>>>>> ourselves,
>>>>>>>>>> and what we recommend users do. But I suspect not everyone does
>>>>> it.
>>>>>>> In
>>>>>>>>>> fact I just checked the javadoc in KafkaProducer and our code
>>>>>> snippet
>>>>>>>>>> does not include a `flush()` call. So I'm thinking maybe we can
>>>>> in
>>>>>>>>>> side the `commitTxn` code to enforce flushing before sending
>>>> the
>>>>>>>>>> end-txn request.
>>>>>>>>>>
>>>>>>>>>> 2. I'd like to clarify a bit details on "just add partitions to
>>>>> the
>>>>>>>>>> transaction on the first produce request during a transaction".
>>>>> My
>>>>>>>>>> understanding is that the partition leader's cache has the
>>>>> producer
>>>>>>> id
>>>>>>>>>> / sequence / epoch for the latest txn, either on-going or is
>>>>>>> completed
>>>>>>>>>> (upon receiving the marker request from coordinator). When a
>>>>>> produce
>>>>>>>>>> request is received, if
>>>>>>>>>>
>>>>>>>>>> * producer's epoch < cached epoch, or producer's epoch ==
>>>> cached
>>>>>>> epoch
>>>>>>>>>> but the latest txn is completed, leader directly reject with
>>>>>>>>>> invalid-epoch.
>>>>>>>>>> * producer's epoch > cached epoch, park the the request and
>>>> send
>>>>>>>>>> add-partitions request to coordinator.
>>>>>>>>>>
>>>>>>>>>> In order to do it, does the coordinator need to bump the
>>>> sequence
>>>>>> and
>>>>>>>>>> reset epoch to 0 when the next epoch is going to overflow? If
>>>> no
>>>>>> need
>>>>>>>>>> to do so, then how we handle the (admittedly rare, but still
>>>> may
>>>>>>>>>> happen) epoch overflow situation?
>>>>>>>>>>
>>>>>>>>>> 3. I'm a bit concerned about adding a generic "ABORTABLE_ERROR"
>>>>>> given
>>>>>>>>>> we already have a pretty messy error classification and error
>>>>>>> handling
>>>>>>>>>> on the producer clients side --- I have a summary about the
>>>>> issues
>>>>>>> and
>>>>>>>>>> a proposal to address this in
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
>>>>>>>>>> -- I understand we do not want to use "UNKNOWN_PRODUCER_ID"
>>>>> anymore
>>>>>>>>>> and in fact we intend to deprecate it in KIP-360 and eventually
>>>>>>> remove
>>>>>>>>>> it; but I'm wondering can we still use specific error codes.
>>>> E.g.
>>>>>>> what
>>>>>>>>>> about "InvalidProducerEpochException" since for new clients,
>>>> the
>>>>>>>>>> actual reason this would actually be rejected is indeed because
>>>>> the
>>>>>>>>>> epoch on the coordinator caused the add-partitions-request from
>>>>> the
>>>>>>>>>> brokers to be rejected anyways?
>>>>>>>>>>
>>>>>>>>>> 4. It seems we put the producer request into purgatory before
>>>> we
>>>>>> ever
>>>>>>>>>> append the records, while other producer's records may still be
>>>>>>>>>> appended during the time; and that potentially may result in
>>>> some
>>>>>>>>>> re-ordering compared with reception order. I'm not super
>>>>> concerned
>>>>>>>>>> about it since Kafka does not guarantee reception ordering
>>>> across
>>>>>>>>>> producers anyways, but it may make the timestamps of records
>>>>>> inside a
>>>>>>>>>> partition to be more out-of-ordered. Are we aware of any
>>>>> scenarios
>>>>>>>>>> such as future enhancements on log compactions that may be
>>>>> affected
>>>>>>> by
>>>>>>>>>> this effect?
>>>>>>>>>>
>>>>>>>>>> Below are just minor comments:
>>>>>>>>>>
>>>>>>>>>> 5. In "AddPartitionsToTxnTransaction" field of
>>>>>>>>>> "AddPartitionsToTxnRequest" RPC, the versions of those inner
>>>>> fields
>>>>>>>>>> are "0-3" while I thought they should be "0+" still?
>>>>>>>>>>
>>>>>>>>>> 6. Regarding "we can place the request in a purgatory of sorts
>>>>> and
>>>>>>>>>> check if there is any state for the transaction on the
>>>> broker": i
>>>>>>>>>> think at this time when we just do the checks against the
>>>> cached
>>>>>>>>>> state, we do not need to put the request to purgatory yet?
>>>>>>>>>>
>>>>>>>>>> 7. This is related to 3) above. I feel using
>>>>>> "InvalidRecordException"
>>>>>>>>>> for older clients may also be a bit confusing, and also it is
>>>> not
>>>>>>>>>> fatal -- for old clients, it better to be fatal since this
>>>>>> indicates
>>>>>>>>>> the clients is doing something wrong and hence it should be
>>>>> closed.
>>>>>>>>>> And in general I'd prefer to use slightly more specific meaning
>>>>>> error
>>>>>>>>>> codes for clients. That being said, I also feel
>>>>>>>>>> "InvalidProducerEpochException" is not suitable for old
>>>> versioned
>>>>>>>>>> clients, and we'd have to pick one that old clients recognize.
>>>>> I'd
>>>>>>>>>> prefer "InvalidTxnStateException" but that one is supposed to
>>>> be
>>>>>>>>>> returned from txn coordinators only today. I'd suggest we do a
>>>>>> quick
>>>>>>>>>> check in the current client's code path and see if that one
>>>> would
>>>>>> be
>>>>>>>>>> handled if it's from a produce-response, and if yes, use this
>>>>> one;
>>>>>>>>>> otherwise, use "ProducerFencedException" which is much less
>>>>>>> meaningful
>>>>>>>>>> but it's still a fatal error.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Thanks,
>>>>>>>>>> Guozhang
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
>>>>>>>>>>>
>>>>>>>>>>> Yeah -- looks like we already have code to handle bumping the
>>>>>> epoch
>>>>>>>> and
>>>>>>>>>>> when the epoch is Short.MAX_VALUE, we get a new producer ID.
>>>>>> Since
>>>>>>>> this
>>>>>>>>>> is
>>>>>>>>>>> already the behavior, do we want to change it further?
>>>>>>>>>>>
>>>>>>>>>>> Justine
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <
>>>>>>> jolshan@confluent.io
>>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Hey all, just wanted to quickly update and say I've
>>>> modified
>>>>>> the
>>>>>>>> KIP to
>>>>>>>>>>>> explicitly mention that AddOffsetCommitsToTxnRequest will
>>>> be
>>>>>>>> replaced
>>>>>>>>>> by
>>>>>>>>>>>> a coordinator-side (inter-broker) AddPartitionsToTxn
>>>> implicit
>>>>>>>> request.
>>>>>>>>>> This
>>>>>>>>>>>> mirrors the user partitions and will implicitly add offset
>>>>>>>> partitions
>>>>>>>>>> to
>>>>>>>>>>>> transactions when we commit offsets on them. We will
>>>>> deprecate
>>>>>>>>>> AddOffsetCommitsToTxnRequest
>>>>>>>>>>>> for new clients.
>>>>>>>>>>>>
>>>>>>>>>>>> Also to address Artem's comments --
>>>>>>>>>>>> I'm a bit unsure if the changes here will change the
>>>> previous
>>>>>>>> behavior
>>>>>>>>>> for
>>>>>>>>>>>> fencing producers. In the case you mention in the first
>>>>>>> paragraph,
>>>>>>>> are
>>>>>>>>>> you
>>>>>>>>>>>> saying we bump the epoch before we try to abort the
>>>>>> transaction?
>>>>>>> I
>>>>>>>>>> think I
>>>>>>>>>>>> need to understand the scenarios you mention a bit better.
>>>>>>>>>>>>
>>>>>>>>>>>> As for the second part -- I think it makes sense to have
>>>> some
>>>>>>> sort
>>>>>>>> of
>>>>>>>>>>>> "sentinel" epoch to signal epoch is about to overflow (I
>>>>> think
>>>>>> we
>>>>>>>> sort
>>>>>>>>>> of
>>>>>>>>>>>> have this value in place in some ways) so we can codify it
>>>> in
>>>>>> the
>>>>>>>> KIP.
>>>>>>>>>> I'll
>>>>>>>>>>>> look into that and try to update soon.
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks,
>>>>>>>>>>>> Justine.
>>>>>>>>>>>>
>>>>>>>>>>>> On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> It's good to know that KIP-588 addressed some of the
>>>> issues.
>>>>>>>> Looking
>>>>>>>>>> at
>>>>>>>>>>>>> the code, it still looks like there are some cases that
>>>>> would
>>>>>>>> result
>>>>>>>>>> in
>>>>>>>>>>>>> fatal error, e.g. PRODUCER_FENCED is issued by the
>>>>> transaction
>>>>>>>>>> coordinator
>>>>>>>>>>>>> if epoch doesn't match, and the client treats it as a
>>>> fatal
>>>>>>> error
>>>>>>>>>> (code in
>>>>>>>>>>>>> TransactionManager request handling).  If we consider, for
>>>>>>>> example,
>>>>>>>>>>>>> committing a transaction that returns a timeout, but
>>>>> actually
>>>>>>>>>> succeeds,
>>>>>>>>>>>>> trying to abort it or re-commit may result in
>>>>> PRODUCER_FENCED
>>>>>>>> error
>>>>>>>>>>>>> (because of epoch bump).
>>>>>>>>>>>>>
>>>>>>>>>>>>> For failed commits, specifically, we need to know the
>>>> actual
>>>>>>>> outcome,
>>>>>>>>>>>>> because if we return an error the application may think
>>>> that
>>>>>> the
>>>>>>>>>>>>> transaction is aborted and redo the work, leading to
>>>>>> duplicates.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Re: overflowing epoch.  We could either do it on the TC
>>>> and
>>>>>>> return
>>>>>>>>>> both
>>>>>>>>>>>>> producer id and epoch (e.g. change the protocol), or
>>>> signal
>>>>>> the
>>>>>>>> client
>>>>>>>>>>>>> that
>>>>>>>>>>>>> it needs to get a new producer id.  Checking for max epoch
>>>>>> could
>>>>>>>> be a
>>>>>>>>>>>>> reasonable signal, the value to check should probably be
>>>>>> present
>>>>>>>> in
>>>>>>>>>> the
>>>>>>>>>>>>> KIP
>>>>>>>>>>>>> as this is effectively a part of the contract.  Also, the
>>>> TC
>>>>>>>> should
>>>>>>>>>>>>> probably return an error if the client didn't change
>>>>> producer
>>>>>> id
>>>>>>>> after
>>>>>>>>>>>>> hitting max epoch.
>>>>>>>>>>>>>
>>>>>>>>>>>>> -Artem
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
>>>>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks for the discussion Artem.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> With respect to the handling of fenced producers, we
>>>> have
>>>>>> some
>>>>>>>>>> behavior
>>>>>>>>>>>>>> already in place. As of KIP-588:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
>>>>>>>>>>>>>> ,
>>>>>>>>>>>>>> we handle timeouts more gracefully. The producer can
>>>>>> recover.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Produce requests can also recover from epoch fencing by
>>>>>>>> aborting the
>>>>>>>>>>>>>> transaction and starting over.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> What other cases were you considering that would cause
>>>> us
>>>>> to
>>>>>>>> have a
>>>>>>>>>>>>> fenced
>>>>>>>>>>>>>> epoch but we'd want to recover?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> The first point about handling epoch overflows is fair.
>>>> I
>>>>>>> think
>>>>>>>>>> there is
>>>>>>>>>>>>>> some logic we'd need to consider. (ie, if we are one
>>>> away
>>>>>> from
>>>>>>>> the
>>>>>>>>>> max
>>>>>>>>>>>>>> epoch, we need to reset the producer ID.) I'm still
>>>>>> wondering
>>>>>>> if
>>>>>>>>>> there
>>>>>>>>>>>>> is a
>>>>>>>>>>>>>> way to direct this from the response, or if everything
>>>>>> should
>>>>>>> be
>>>>>>>>>> done on
>>>>>>>>>>>>>> the client side. Let me know if you have any thoughts
>>>>> here.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
>>>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> There are some workflows in the client that are
>>>> implied
>>>>> by
>>>>>>>>>> protocol
>>>>>>>>>>>>>>> changes, e.g.:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> - for new clients, epoch changes with every
>>>> transaction
>>>>>> and
>>>>>>>> can
>>>>>>>>>>>>> overflow,
>>>>>>>>>>>>>>> in old clients this condition was handled
>>>> transparently,
>>>>>>>> because
>>>>>>>>>> epoch
>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>> bumped in InitProducerId and it would return a new
>>>>>> producer
>>>>>>>> id if
>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>> overflows, the new clients would need to implement
>>>> some
>>>>>>>> workflow
>>>>>>>>>> to
>>>>>>>>>>>>>> refresh
>>>>>>>>>>>>>>> producer id
>>>>>>>>>>>>>>> - how to handle fenced producers, for new clients
>>>> epoch
>>>>>>>> changes
>>>>>>>>>> with
>>>>>>>>>>>>>> every
>>>>>>>>>>>>>>> transaction, so in presence of failures during
>>>> commits /
>>>>>>>> aborts,
>>>>>>>>>> the
>>>>>>>>>>>>>>> producer could get easily fenced, old clients would
>>>>> pretty
>>>>>>>> much
>>>>>>>>>> would
>>>>>>>>>>>>> get
>>>>>>>>>>>>>>> fenced when a new incarnation of the producer was
>>>>>>> initialized
>>>>>>>> with
>>>>>>>>>>>>>>> InitProducerId so it's ok to treat as a fatal error,
>>>> the
>>>>>> new
>>>>>>>>>> clients
>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>> need to implement some workflow to handle that error,
>>>>>>>> otherwise
>>>>>>>>>> they
>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>> get fenced by themselves
>>>>>>>>>>>>>>> - in particular (as a subset of the previous issue),
>>>>> what
>>>>>>>> would
>>>>>>>>>> the
>>>>>>>>>>>>>> client
>>>>>>>>>>>>>>> do if it got a timeout during commit?  commit could've
>>>>>>>> succeeded
>>>>>>>>>> or
>>>>>>>>>>>>>> failed
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Not sure if this has to be defined in the KIP as
>>>>>>> implementing
>>>>>>>>>> those
>>>>>>>>>>>>>>> probably wouldn't require protocol changes, but we
>>>> have
>>>>>>>> multiple
>>>>>>>>>>>>>>> implementations of Kafka clients, so probably would be
>>>>>> good
>>>>>>> to
>>>>>>>>>> have
>>>>>>>>>>>>> some
>>>>>>>>>>>>>>> client implementation guidance.  Could also be done
>>>> as a
>>>>>>>> separate
>>>>>>>>>> doc.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> -Artem
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
>>>>>>>>>>>>>> <jolshan@confluent.io.invalid
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hey all, I've updated the KIP to incorporate Jason's
>>>>>>>>>> suggestions.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 1. Use AddPartitionsToTxn + verify flag to check on
>>>>> old
>>>>>>>> clients
>>>>>>>>>>>>>>>> 2. Updated AddPartitionsToTxn API to support
>>>>> transaction
>>>>>>>>>> batching
>>>>>>>>>>>>>>>> 3. Mention IBP bump
>>>>>>>>>>>>>>>> 4. Mention auth change on new AddPartitionsToTxn
>>>>>> version.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I'm planning on opening a vote soon.
>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <
>>>>>>>>>> jolshan@confluent.io
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks Jason. Those changes make sense to me. I
>>>> will
>>>>>>>> update
>>>>>>>>>> the
>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hey Justine,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I was wondering about compatibility here. When
>>>> we
>>>>>>> send
>>>>>>>>>> requests
>>>>>>>>>>>>>>>>>> between brokers, we want to ensure that the
>>>>> receiving
>>>>>>>> broker
>>>>>>>>>>>>>>> understands
>>>>>>>>>>>>>>>>>> the request (specifically the new fields).
>>>>> Typically
>>>>>>>> this is
>>>>>>>>>> done
>>>>>>>>>>>>>> via
>>>>>>>>>>>>>>>>>> IBP/metadata version.
>>>>>>>>>>>>>>>>>> I'm trying to think if there is a way around it
>>>> but
>>>>>> I'm
>>>>>>>> not
>>>>>>>>>> sure
>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>> is.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Yes. I think we would gate usage of this behind
>>>> an
>>>>>> IBP
>>>>>>>> bump.
>>>>>>>>>> Does
>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>> seem
>>>>>>>>>>>>>>>>>> reasonable?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> As for the improvements -- can you clarify how
>>>>> the
>>>>>>>> multiple
>>>>>>>>>>>>>>>>>> transactional
>>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking of a case
>>>>>> where
>>>>>>> we
>>>>>>>>>>>>> wait/batch
>>>>>>>>>>>>>>>>>> multiple produce requests together? My
>>>>> understanding
>>>>>>> for
>>>>>>>> now
>>>>>>>>>> was
>>>>>>>>>>>>> 1
>>>>>>>>>>>>>>>>>> transactional ID and one validation per 1 produce
>>>>>>>> request.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Each call to `AddPartitionsToTxn` is essentially
>>>> a
>>>>>>> write
>>>>>>>> to
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>> log and must block on replication. The more we
>>>> can
>>>>>> fit
>>>>>>>> into a
>>>>>>>>>>>>> single
>>>>>>>>>>>>>>>>>> request, the more writes we can do in parallel.
>>>> The
>>>>>>>>>> alternative
>>>>>>>>>>>>> is
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>> use of more connections, but usually we prefer
>>>>>> batching
>>>>>>>>>> since the
>>>>>>>>>>>>>>>> network
>>>>>>>>>>>>>>>>>> stack is not really optimized for high
>>>>>>> connection/request
>>>>>>>>>> loads.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Finally with respect to the authorizations, I
>>>>> think
>>>>>>> it
>>>>>>>>>> makes
>>>>>>>>>>>>> sense
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> skip
>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit confused by
>>>> the
>>>>>>>> "leader
>>>>>>>>>> ID"
>>>>>>>>>>>>>> field.
>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the request as
>>>> from a
>>>>>>>> broker
>>>>>>>>>> (does
>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>> matter
>>>>>>>>>>>>>>>>>> which one?).
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> We could also make it version-based. For the next
>>>>>>>> version, we
>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>> CLUSTER auth. So clients would not be able to use
>>>>> the
>>>>>>> API
>>>>>>>>>>>>> anymore,
>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>> probably what we want.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> -Jason
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> As a follow up, I was just thinking about the
>>>>>>> batching
>>>>>>>> a
>>>>>>>>>> bit
>>>>>>>>>>>>> more.
>>>>>>>>>>>>>>>>>>> I suppose if we have one request in flight and
>>>> we
>>>>>>>> queue up
>>>>>>>>>> the
>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>> produce requests in some sort of purgatory, we
>>>>>> could
>>>>>>>> send
>>>>>>>>>>>>>>> information
>>>>>>>>>>>>>>>>>> out
>>>>>>>>>>>>>>>>>>> for all of them rather than one by one. So that
>>>>>> would
>>>>>>>> be a
>>>>>>>>>>>>> benefit
>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>> batching partitions to add per transaction.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I'll need to think a bit more on the design of
>>>>> this
>>>>>>>> part
>>>>>>>>>> of the
>>>>>>>>>>>>>> KIP,
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> will update the KIP in the next few days.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan
>>>> <
>>>>>>>>>>>>>>> jolshan@confluent.io>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Hey Jason -- thanks for the input -- I was
>>>> just
>>>>>>>> digging
>>>>>>>>>> a bit
>>>>>>>>>>>>>>> deeper
>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>> the design + implementation of the validation
>>>>>> calls
>>>>>>>> here
>>>>>>>>>> and
>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>> say
>>>>>>>>>>>>>>>>>>>> makes sense.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I was wondering about compatibility here.
>>>> When
>>>>> we
>>>>>>>> send
>>>>>>>>>>>>> requests
>>>>>>>>>>>>>>>>>>>> between brokers, we want to ensure that the
>>>>>>> receiving
>>>>>>>>>> broker
>>>>>>>>>>>>>>>>>> understands
>>>>>>>>>>>>>>>>>>>> the request (specifically the new fields).
>>>>>>> Typically
>>>>>>>>>> this is
>>>>>>>>>>>>>> done
>>>>>>>>>>>>>>>> via
>>>>>>>>>>>>>>>>>>>> IBP/metadata version.
>>>>>>>>>>>>>>>>>>>> I'm trying to think if there is a way around
>>>> it
>>>>>> but
>>>>>>>> I'm
>>>>>>>>>> not
>>>>>>>>>>>>> sure
>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>> is.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> As for the improvements -- can you clarify
>>>> how
>>>>>> the
>>>>>>>>>> multiple
>>>>>>>>>>>>>>>>>> transactional
>>>>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking of a
>>>>> case
>>>>>>>> where we
>>>>>>>>>>>>>>> wait/batch
>>>>>>>>>>>>>>>>>>>> multiple produce requests together? My
>>>>>>> understanding
>>>>>>>> for
>>>>>>>>>> now
>>>>>>>>>>>>>> was 1
>>>>>>>>>>>>>>>>>>>> transactional ID and one validation per 1
>>>>> produce
>>>>>>>>>> request.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Finally with respect to the authorizations, I
>>>>>> think
>>>>>>>> it
>>>>>>>>>> makes
>>>>>>>>>>>>>> sense
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> skip
>>>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit confused
>>>> by
>>>>>> the
>>>>>>>>>> "leader
>>>>>>>>>>>>> ID"
>>>>>>>>>>>>>>>> field.
>>>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the request as
>>>>>> from a
>>>>>>>>>> broker
>>>>>>>>>>>>> (does
>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>> matter
>>>>>>>>>>>>>>>>>>>> which one?).
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I think I want to adopt these suggestions,
>>>> just
>>>>>> had
>>>>>>>> a few
>>>>>>>>>>>>>>> questions
>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> details.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Thu, Jan 5, 2023 at 5:05 PM Jason
>>>> Gustafson
>>>>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Hi Justine,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thanks for the proposal.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I was thinking about the implementation a
>>>>> little
>>>>>>>> bit.
>>>>>>>>>> In the
>>>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>> proposal, the behavior depends on whether we
>>>>>> have
>>>>>>> an
>>>>>>>>>> old or
>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>> client.
>>>>>>>>>>>>>>>>>>>>> For
>>>>>>>>>>>>>>>>>>>>> old clients, we send `DescribeTransactions`
>>>>> and
>>>>>>>> verify
>>>>>>>>>> the
>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>> new clients, we send `AddPartitionsToTxn`.
>>>> We
>>>>>>> might
>>>>>>>> be
>>>>>>>>>> able
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> simplify
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> implementation if we can use the same
>>>> request
>>>>>>> type.
>>>>>>>> For
>>>>>>>>>>>>>> example,
>>>>>>>>>>>>>>>>>> what if
>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>> bump the protocol version for
>>>>>> `AddPartitionsToTxn`
>>>>>>>> and
>>>>>>>>>> add a
>>>>>>>>>>>>>>>>>>>>> `validateOnly`
>>>>>>>>>>>>>>>>>>>>> flag? For older versions, we can set
>>>>>>>>>> `validateOnly=true` so
>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> request only returns successfully if the
>>>>>> partition
>>>>>>>> had
>>>>>>>>>>>>> already
>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>> added.
>>>>>>>>>>>>>>>>>>>>> For new versions, we can set
>>>>>> `validateOnly=false`
>>>>>>>> and
>>>>>>>>>> the
>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>> added to the transaction. The other slightly
>>>>>>>> annoying
>>>>>>>>>> thing
>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>> get around is the need to collect the
>>>>>> transaction
>>>>>>>> state
>>>>>>>>>> for
>>>>>>>>>>>>> all
>>>>>>>>>>>>>>>>>>> partitions
>>>>>>>>>>>>>>>>>>>>> even when we only care about a subset.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Some additional improvements to consider:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> - We can give `AddPartitionsToTxn` better
>>>>> batch
>>>>>>>> support
>>>>>>>>>> for
>>>>>>>>>>>>>>>>>> inter-broker
>>>>>>>>>>>>>>>>>>>>> usage. Currently we only allow one
>>>>>>>> `TransactionalId` to
>>>>>>>>>> be
>>>>>>>>>>>>>>>> specified,
>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>> the broker may get some benefit being able
>>>> to
>>>>>>> batch
>>>>>>>>>> across
>>>>>>>>>>>>>>> multiple
>>>>>>>>>>>>>>>>>>>>> transactions.
>>>>>>>>>>>>>>>>>>>>> - Another small improvement is skipping
>>>> topic
>>>>>>>>>> authorization
>>>>>>>>>>>>>>> checks
>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>> `AddPartitionsToTxn` when the request is
>>>> from
>>>>> a
>>>>>>>> broker.
>>>>>>>>>>>>> Perhaps
>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>> add
>>>>>>>>>>>>>>>>>>>>> a field for the `LeaderId` or something like
>>>>>> that
>>>>>>>> and
>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>> CLUSTER
>>>>>>>>>>>>>>>>>>>>> permission when set.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>> Jason
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
>>>>>>>>>>>>>> <jun@confluent.io.invalid
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation. It makes sense
>>>>> to
>>>>>> me
>>>>>>>> now.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Jun
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 1:42 PM Justine
>>>>> Olshan
>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> My understanding of the mechanism is
>>>> that
>>>>>> when
>>>>>>>> we
>>>>>>>>>> get to
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> last
>>>>>>>>>>>>>>>>>>>>> epoch,
>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>> increment to the fencing/last epoch and
>>>> if
>>>>>> any
>>>>>>>>>> further
>>>>>>>>>>>>>>> requests
>>>>>>>>>>>>>>>>>> come
>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>> this producer ID they are fenced. Then
>>>> the
>>>>>>>> producer
>>>>>>>>>>>>> gets a
>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>> ID
>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>> restarts with epoch/sequence 0. The
>>>> fenced
>>>>>>> epoch
>>>>>>>>>> sticks
>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> duration of producer.id.expiration.ms
>>>> and
>>>>>>>> blocks
>>>>>>>>>> any
>>>>>>>>>>>>> late
>>>>>>>>>>>>>>>>>> messages
>>>>>>>>>>>>>>>>>>>>>> there.
>>>>>>>>>>>>>>>>>>>>>>> The new ID will get to take advantage of
>>>>> the
>>>>>>>>>> improved
>>>>>>>>>>>>>>> semantics
>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>>>> non-zero start sequences. So I think we
>>>>> are
>>>>>>>> covered.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> The only potential issue is overloading
>>>>> the
>>>>>>>> cache,
>>>>>>>>>> but
>>>>>>>>>>>>>>>> hopefully
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> improvements (lowered
>>>>>>> producer.id.expiration.ms
>>>>>>>> )
>>>>>>>>>> will
>>>>>>>>>>>>> help
>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>> that.
>>>>>>>>>>>>>>>>>>>>>> Let
>>>>>>>>>>>>>>>>>>>>>>> me know if you still have concerns.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
>>>>>>>>>>>>>>>>>> <ju...@confluent.io.invalid>
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 70. The proposed fencing logic doesn't
>>>>>> apply
>>>>>>>> when
>>>>>>>>>> pid
>>>>>>>>>>>>>>>> changes,
>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>> right? If so, I am not sure how
>>>> complete
>>>>>> we
>>>>>>>> are
>>>>>>>>>>>>>> addressing
>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>> issue
>>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>> the pid changes more frequently.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Jun
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 16, 2022 at 9:16 AM
>>>> Justine
>>>>>>> Olshan
>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for replying!
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> 70.We already do the overflow
>>>>> mechanism,
>>>>>>> so
>>>>>>>> my
>>>>>>>>>>>>> change
>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>> happen more often.
>>>>>>>>>>>>>>>>>>>>>>>>> I was also not suggesting a new
>>>> field
>>>>> in
>>>>>>> the
>>>>>>>>>> log,
>>>>>>>>>>>>> but
>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> response,
>>>>>>>>>>>>>>>>>>>>>>>>> which would be gated by the client
>>>>>>> version.
>>>>>>>>>> Sorry if
>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>> unclear. I think we are starting to
>>>>>>> diverge.
>>>>>>>>>>>>>>>>>>>>>>>>> The goal of this KIP is to not
>>>> change
>>>>> to
>>>>>>> the
>>>>>>>>>> marker
>>>>>>>>>>>>>>> format
>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> 71. Yes, I guess I was going under
>>>> the
>>>>>>>>>> assumption
>>>>>>>>>>>>> that
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> log
>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>> look at its last epoch and treat it
>>>> as
>>>>>> the
>>>>>>>>>> current
>>>>>>>>>>>>>>> epoch. I
>>>>>>>>>>>>>>>>>>>>> suppose
>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>> have some special logic that if the
>>>>> last
>>>>>>>> epoch
>>>>>>>>>> was
>>>>>>>>>>>>> on a
>>>>>>>>>>>>>>>>>> marker
>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>> expect the next epoch or something
>>>>> like
>>>>>>>> that. We
>>>>>>>>>>>>> just
>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> distinguish
>>>>>>>>>>>>>>>>>>>>>>>>> based on whether we had a
>>>> commit/abort
>>>>>>>> marker.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> 72.
>>>>>>>>>>>>>>>>>>>>>>>>>> if the producer epoch hasn't been
>>>>>> bumped
>>>>>>>> on
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
>>>>>> message
>>>>>>>> will
>>>>>>>>>> fail
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> sequence
>>>>>>>>>>>>>>>>>>>>>>>> validation
>>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the producer
>>>>>> epoch
>>>>>>>> has
>>>>>>>>>> been
>>>>>>>>>>>>>>> bumped,
>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>> ignore
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck message
>>>>>> could
>>>>>>>> be
>>>>>>>>>>>>> appended
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> log.
>>>>>>>>>>>>>>>>>>>>>> So,
>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
>>>> guard?
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I follow that "the
>>>>> message
>>>>>>> will
>>>>>>>>>> fail
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> sequence
>>>>>>>>>>>>>>>>>>>>>>>> validation".
>>>>>>>>>>>>>>>>>>>>>>>>> In some of these cases, we had an
>>>>> abort
>>>>>>>> marker
>>>>>>>>>> (due
>>>>>>>>>>>>> to
>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>> error)
>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>> then
>>>>>>>>>>>>>>>>>>>>>>>>> the late message comes in with the
>>>>>> correct
>>>>>>>>>> sequence
>>>>>>>>>>>>>>> number.
>>>>>>>>>>>>>>>>>> This
>>>>>>>>>>>>>>>>>>>>> is a
>>>>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>> covered by the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>> The latter case is actually not
>>>>>> something
>>>>>>>> we've
>>>>>>>>>>>>>>> considered
>>>>>>>>>>>>>>>>>>> here. I
>>>>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>> generally when we bump the epoch, we
>>>>> are
>>>>>>>>>> accepting
>>>>>>>>>>>>> that
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> sequence
>>>>>>>>>>>>>>>>>>>>>>> does
>>>>>>>>>>>>>>>>>>>>>>>>> not need to be checked anymore. My
>>>>>>>>>> understanding is
>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>> that we
>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>> typically bump epoch mid transaction
>>>>>>> (based
>>>>>>>> on a
>>>>>>>>>>>>> quick
>>>>>>>>>>>>>>> look
>>>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> code)
>>>>>>>>>>>>>>>>>>>>>>>>> but let me know if that is the case.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 15, 2022 at 12:23 PM Jun
>>>>> Rao
>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the reply.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Assigning a new pid on int
>>>>>> overflow
>>>>>>>> seems
>>>>>>>>>> a
>>>>>>>>>>>>> bit
>>>>>>>>>>>>>>>> hacky.
>>>>>>>>>>>>>>>>>> If
>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>> need a
>>>>>>>>>>>>>>>>>>>>>>>>> txn
>>>>>>>>>>>>>>>>>>>>>>>>>> level id, it will be better to
>>>> model
>>>>>>> this
>>>>>>>>>>>>> explicitly.
>>>>>>>>>>>>>>>>>> Adding a
>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>> field
>>>>>>>>>>>>>>>>>>>>>>>>>> would require a bit more work
>>>> since
>>>>> it
>>>>>>>>>> requires a
>>>>>>>>>>>>> new
>>>>>>>>>>>>>>> txn
>>>>>>>>>>>>>>>>>>> marker
>>>>>>>>>>>>>>>>>>>>>>> format
>>>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>> the log. So, we probably need to
>>>>> guard
>>>>>>> it
>>>>>>>>>> with an
>>>>>>>>>>>>> IBP
>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>> metadata
>>>>>>>>>>>>>>>>>>>>>>>> version
>>>>>>>>>>>>>>>>>>>>>>>>>> and document the impact on
>>>> downgrade
>>>>>>> once
>>>>>>>> the
>>>>>>>>>> new
>>>>>>>>>>>>>>> format
>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>> written
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> log.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 71. Hmm, once the marker is
>>>> written,
>>>>>> the
>>>>>>>>>> partition
>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>> expect
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>> append to be on the next epoch.
>>>> Does
>>>>>>> that
>>>>>>>>>> cover
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>> mentioned?
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 72. Also, just to be clear on the
>>>>>>> stucked
>>>>>>>>>> message
>>>>>>>>>>>>>> issue
>>>>>>>>>>>>>>>>>>>>> described
>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> motivation. With EoS, we also
>>>>> validate
>>>>>>> the
>>>>>>>>>>>>> sequence
>>>>>>>>>>>>>> id
>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>> idempotency.
>>>>>>>>>>>>>>>>>>>>>>>>> So,
>>>>>>>>>>>>>>>>>>>>>>>>>> with the current logic, if the
>>>>>> producer
>>>>>>>> epoch
>>>>>>>>>>>>> hasn't
>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>> bumped on
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
>>>>>>> message
>>>>>>>> will
>>>>>>>>>>>>> fail
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> sequence
>>>>>>>>>>>>>>>>>>>>>>>>> validation
>>>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the
>>>> producer
>>>>>>>> epoch has
>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>> bumped, we
>>>>>>>>>>>>>>>>>>>>>> ignore
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck
>>>> message
>>>>>>>> could be
>>>>>>>>>>>>>> appended
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> log.
>>>>>>>>>>>>>>>>>>>>>>> So,
>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
>>>>> guard?
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 10:44 AM
>>>>>> Justine
>>>>>>>>>> Olshan
>>>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias — thanks again for
>>>> taking
>>>>>>> time
>>>>>>>> to
>>>>>>>>>> look
>>>>>>>>>>>>> a
>>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>>>> You
>>>>>>>>>>>>>>>>>>>>> said:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> My proposal was only focusing
>>>> to
>>>>>>> avoid
>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
>>>> added
>>>>>>>> without
>>>>>>>>>>>>>> registered
>>>>>>>>>>>>>>>>>>>>> partition.
>>>>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more details
>>>> to
>>>>>> the
>>>>>>>> KIP
>>>>>>>>>> about
>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>> scenario
>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I understand what
>>>> you
>>>>>>> mean
>>>>>>>>>> here.
>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>> motivation
>>>>>>>>>>>>>>>>>>>>>>> section
>>>>>>>>>>>>>>>>>>>>>>>>>>> describes two scenarios about
>>>> how
>>>>>> the
>>>>>>>> record
>>>>>>>>>>>>> can be
>>>>>>>>>>>>>>>> added
>>>>>>>>>>>>>>>>>>>>>> without a
>>>>>>>>>>>>>>>>>>>>>>>>>>> registered partition:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Another way hanging
>>>> transactions
>>>>>> can
>>>>>>>>>> occur is
>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>> client
>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>> buggy
>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>> may somehow try to write to a
>>>>>>> partition
>>>>>>>>>> before
>>>>>>>>>>>>> it
>>>>>>>>>>>>>>> adds
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> For the first example of this
>>>>> would
>>>>>> it
>>>>>>>> be
>>>>>>>>>>>>> helpful
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> say
>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>>> comes in after the abort, but
>>>>> before
>>>>>>> the
>>>>>>>>>>>>> partition
>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>> added
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction so it becomes
>>>>> "hanging."
>>>>>>>>>> Perhaps the
>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>> sentence
>>>>>>>>>>>>>>>>>>>>>>>>> describing
>>>>>>>>>>>>>>>>>>>>>>>>>>> the message becoming part of the
>>>>>> next
>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>> (a
>>>>>>>>>>>>>>>>>>> different
>>>>>>>>>>>>>>>>>>>>>>> case)
>>>>>>>>>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>>>>>>>> not properly differentiated.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun — thanks for reading the
>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. The int typing was a
>>>> concern.
>>>>>>>> Currently
>>>>>>>>>> we
>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>> mechanism
>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>> place
>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>> fence the final epoch when the
>>>>> epoch
>>>>>>> is
>>>>>>>>>> about to
>>>>>>>>>>>>>>>> overflow
>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>> assign
>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>> producer ID with epoch 0. Of
>>>>> course,
>>>>>>>> this
>>>>>>>>>> is a
>>>>>>>>>>>>> bit
>>>>>>>>>>>>>>>> tricky
>>>>>>>>>>>>>>>>>>>>> when it
>>>>>>>>>>>>>>>>>>>>>>>> comes
>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>> the response back to the client.
>>>>>>>>>>>>>>>>>>>>>>>>>>> Making this a long could be
>>>>> another
>>>>>>>> option,
>>>>>>>>>> but
>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>> wonder
>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>>>>> implications on changing this
>>>>> field
>>>>>> if
>>>>>>>> the
>>>>>>>>>>>>> epoch is
>>>>>>>>>>>>>>>>>>> persisted
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> disk?
>>>>>>>>>>>>>>>>>>>>>>>>>> I'd
>>>>>>>>>>>>>>>>>>>>>>>>>>> need to check the usages.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> 71.This was something Matthias
>>>>> asked
>>>>>>>> about
>>>>>>>>>> as
>>>>>>>>>>>>>> well. I
>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>>>>> considering a
>>>>>>>>>>>>>>>>>>>>>>>>>>> possible edge case where a
>>>> produce
>>>>>>>> request
>>>>>>>>>> from
>>>>>>>>>>>>> a
>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>> somehow
>>>>>>>>>>>>>>>>>>>>>>>>>>> gets sent right after the marker
>>>>> is
>>>>>>>>>> written, but
>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>> alerted of the newly bumped
>>>> epoch.
>>>>>> In
>>>>>>>> this
>>>>>>>>>>>>> case, we
>>>>>>>>>>>>>>> may
>>>>>>>>>>>>>>>>>>>>> include
>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>>>>>>>>>>> when we don't want to. I suppose
>>>>> we
>>>>>>>> could
>>>>>>>>>> try
>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>> client
>>>>>>>>>>>>>>>>>>>>>>>>> side
>>>>>>>>>>>>>>>>>>>>>>>>>>> to bump the epoch after sending
>>>> an
>>>>>>>> endTxn as
>>>>>>>>>>>>> well
>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>> scenario
>>>>>>>>>>>>>>>>>>>>>>> —
>>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>> wonder how it would work when
>>>> the
>>>>>>>> server is
>>>>>>>>>>>>>> aborting
>>>>>>>>>>>>>>>>>> based
>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
>>>>>>>>>>>>>>>>>>>>>>>>>>> error. I could also be missing
>>>>>>>> something and
>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>> scenario
>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>> possible.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again to everyone reading
>>>>> and
>>>>>>>>>> commenting.
>>>>>>>>>>>>>> Let
>>>>>>>>>>>>>>> me
>>>>>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>>>>> further questions or comments.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 9:41 AM
>>>>> Jun
>>>>>>> Rao
>>>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP. A couple
>>>> of
>>>>>>>> comments.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Currently, the producer
>>>>> epoch
>>>>>> is
>>>>>>>> an
>>>>>>>>>> int.
>>>>>>>>>>>>> I am
>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>> sure
>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>> enough
>>>>>>>>>>>>>>>>>>>>>>>>>>>> to accommodate all
>>>> transactions
>>>>> in
>>>>>>> the
>>>>>>>>>>>>> lifetime
>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>> producer.
>>>>>>>>>>>>>>>>>>>>>>>> Should
>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>> change that to a long or add a
>>>>> new
>>>>>>>> long
>>>>>>>>>> field
>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>> txnId?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 71. "it will write the prepare
>>>>>>> commit
>>>>>>>>>> message
>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>> bumped
>>>>>>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>> send
>>>>>>>>>>>>>>>>>>>>>>>>>>>> WriteTxnMarkerRequests with
>>>> the
>>>>>>> bumped
>>>>>>>>>> epoch."
>>>>>>>>>>>>>> Hmm,
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>> associated
>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the current txn right?
>>>> So,
>>>>> it
>>>>>>>> seems
>>>>>>>>>>>>> weird to
>>>>>>>>>>>>>>>>>> write a
>>>>>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>>>> with a bumped epoch. Should we
>>>>>> only
>>>>>>>> bump
>>>>>>>>>> up
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>> EndTxnResponse
>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>> rename the field to sth like
>>>>>>>>>>>>> nextProducerEpoch?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 12, 2022 at 8:54
>>>> PM
>>>>>>>> Matthias
>>>>>>>>>> J.
>>>>>>>>>>>>> Sax <
>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org>
>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the background.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30: SGTM. My proposal was
>>>>>> only
>>>>>>>>>> focusing
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>> avoid
>>>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
>>>>>> added
>>>>>>>>>> without
>>>>>>>>>>>>>>>> registered
>>>>>>>>>>>>>>>>>>>>>> partition.
>>>>>>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more
>>>> details
>>>>>> to
>>>>>>>> the
>>>>>>>>>> KIP
>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>> scenario
>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40: I think you hit a fair
>>>>> point
>>>>>>>> about
>>>>>>>>>> race
>>>>>>>>>>>>>>>>>> conditions
>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>> client
>>>>>>>>>>>>>>>>>>>>>>>>> bugs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (incorrectly not bumping the
>>>>>>>> epoch). The
>>>>>>>>>>>>>>>>>>>>> complexity/confusion
>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the bumped epoch I see, is
>>>>>> mainly
>>>>>>>> for
>>>>>>>>>>>>> internal
>>>>>>>>>>>>>>>>>>> debugging,
>>>>>>>>>>>>>>>>>>>>> ie,
>>>>>>>>>>>>>>>>>>>>>>>>>>> inspecting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log segment dumps -- it
>>>> seems
>>>>>>>> harder to
>>>>>>>>>>>>> reason
>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> system
>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>> us
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> humans. But if we get better
>>>>>>>>>> guarantees, it
>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>> worth to
>>>>>>>>>>>>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped epoch.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60: as I mentioned already,
>>>> I
>>>>>>> don't
>>>>>>>>>> know the
>>>>>>>>>>>>>>> broker
>>>>>>>>>>>>>>>>>>>>> internals
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>> provide
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more input. So if nobody
>>>> else
>>>>>>> chimes
>>>>>>>>>> in, we
>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>> move
>>>>>>>>>>>>>>>>>>>>>>>>> forward
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with your proposal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 12/6/22 4:22 PM, Justine
>>>>>> Olshan
>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi all,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> After Artem's questions
>>>>> about
>>>>>>>> error
>>>>>>>>>>>>> behavior,
>>>>>>>>>>>>>>>> I've
>>>>>>>>>>>>>>>>>>>>>>> re-evaluated
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unknown producer ID
>>>>> exception
>>>>>>> and
>>>>>>>> had
>>>>>>>>>> some
>>>>>>>>>>>>>>>>>> discussions
>>>>>>>>>>>>>>>>>>>>>>> offline.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think generally it makes
>>>>>> sense
>>>>>>>> to
>>>>>>>>>>>>> simplify
>>>>>>>>>>>>>>>> error
>>>>>>>>>>>>>>>>>>>>> handling
>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>> cases
>>>>>>>>>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this and the
>>>>>> UNKNOWN_PRODUCER_ID
>>>>>>>> error
>>>>>>>>>>>>> has a
>>>>>>>>>>>>>>>> pretty
>>>>>>>>>>>>>>>>>>> long
>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>> complicated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> history. Because of this,
>>>> I
>>>>>>>> propose
>>>>>>>>>>>>> adding a
>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>> error
>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ABORTABLE_ERROR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that when encountered by
>>>> new
>>>>>>>> clients
>>>>>>>>>>>>> (gated
>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> produce
>>>>>>>>>>>>>>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will simply abort the
>>>>>>> transaction.
>>>>>>>>>> This
>>>>>>>>>>>>>> allows
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> server
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> say
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in whether the client
>>>> aborts
>>>>>> and
>>>>>>>> makes
>>>>>>>>>>>>>> handling
>>>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>>>> simpler.
>>>>>>>>>>>>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> future, we can also use
>>>> this
>>>>>>>> error in
>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>> situations
>>>>>>>>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>> want
>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> abort the transactions. We
>>>>> can
>>>>>>>> even
>>>>>>>>>> use on
>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>> apis.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've added this to the
>>>> KIP.
>>>>>> Let
>>>>>>> me
>>>>>>>>>> know if
>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>>> questions
>>>>>>>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 2, 2022 at
>>>> 10:22
>>>>>> AM
>>>>>>>>>> Justine
>>>>>>>>>>>>>> Olshan
>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>> jolshan@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey Matthias,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30 — Maybe I also
>>>> didn't
>>>>>>>> express
>>>>>>>>>>>>> myself
>>>>>>>>>>>>>>>>>> clearly.
>>>>>>>>>>>>>>>>>>> For
>>>>>>>>>>>>>>>>>>>>>>> older
>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't have a way to
>>>>>> distinguish
>>>>>>>>>> between a
>>>>>>>>>>>>>>>> previous
>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction since we
>>>> don't
>>>>>> have
>>>>>>>> the
>>>>>>>>>> epoch
>>>>>>>>>>>>>>> bump.
>>>>>>>>>>>>>>>>>> This
>>>>>>>>>>>>>>>>>>>>> means
>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>> late
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message from the previous
>>>>>>>> transaction
>>>>>>>>>>>>> may be
>>>>>>>>>>>>>>>>>> added to
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>> one.
>>>>>>>>>>>>>>>>>>>>>>>>>>>> With
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older clients — we can't
>>>>>>>> guarantee
>>>>>>>>>> this
>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>> happen
>>>>>>>>>>>>>>>>>>>>> if we
>>>>>>>>>>>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call
>>>>> (why
>>>>>> we
>>>>>>>> make
>>>>>>>>>>>>> changes
>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> newer
>>>>>>>>>>>>>>>>>>>>>>>>> client)
>>>>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can at least gate some by
>>>>>>>> ensuring
>>>>>>>>>> that
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>>> has
>>>>>>>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>>>>>>> added
>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. The
>>>> rationale
>>>>>> here
>>>>>>>> is
>>>>>>>>>> that
>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>> likely
>>>>>>>>>>>>>>>>>>>>>>> LESS
>>>>>>>>>>>>>>>>>>>>>>>>> late
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrivals
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as time goes on, so
>>>>> hopefully
>>>>>>>> most
>>>>>>>>>> late
>>>>>>>>>>>>>>> arrivals
>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>> come
>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>> BEFORE
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call.
>>>>>> Those
>>>>>>>> that
>>>>>>>>>>>>> arrive
>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>> properly
>>>>>>>>>>>>>>>>>>>>>>>>>>>> gated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>> describeTransactions
>>>>>>>>>> approach.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we take the approach
>>>> you
>>>>>>>>>> suggested,
>>>>>>>>>>>>> ANY
>>>>>>>>>>>>>>> late
>>>>>>>>>>>>>>>>>>> arrival
>>>>>>>>>>>>>>>>>>>>>>> from a
>>>>>>>>>>>>>>>>>>>>>>>>>>>> previous
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction will be
>>>> added.
>>>>>> And
>>>>>>> we
>>>>>>>>>> don't
>>>>>>>>>>>>> want
>>>>>>>>>>>>>>>>>> that. I
>>>>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>> see
>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> benefit in sending
>>>>>>>> addPartitionsToTxn
>>>>>>>>>>>>> over
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> describeTxns
>>>>>>>>>>>>>>>>>>>>>>>>> call.
>>>>>>>>>>>>>>>>>>>>>>>>>>> They
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> both be one extra RPC to
>>>>> the
>>>>>>> Txn
>>>>>>>>>>>>>> coordinator.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To be clear — newer
>>>> clients
>>>>>>> will
>>>>>>>> use
>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTxns.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that if we
>>>>> have
>>>>>>>> some
>>>>>>>>>> delay
>>>>>>>>>>>>> in
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> client
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> bump
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it could continue to send
>>>>>> epoch
>>>>>>>> 73
>>>>>>>>>> and
>>>>>>>>>>>>> those
>>>>>>>>>>>>>>>>>> records
>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fenced.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Perhaps this is not an
>>>>> issue
>>>>>> if
>>>>>>>> we
>>>>>>>>>> don't
>>>>>>>>>>>>>> allow
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>> produce
>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>> go
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through before the EndTxn
>>>>>>> request
>>>>>>>>>>>>> returns.
>>>>>>>>>>>>>> I'm
>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cases of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure. I will need to
>>>>> think
>>>>>>> on
>>>>>>>>>> this a
>>>>>>>>>>>>> bit.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I wasn't sure if it was
>>>>> that
>>>>>>>>>> confusing.
>>>>>>>>>>>>> But
>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> investigate other ways.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure these are
>>>> the
>>>>>> same
>>>>>>>>>>>>> purgatories
>>>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>>>>> is a
>>>>>>>>>>>>>>>>>>>>>>>>> produce
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory (I was planning
>>>>> on
>>>>>>>> using a
>>>>>>>>>>>>>> callback
>>>>>>>>>>>>>>>>>> rather
>>>>>>>>>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory)
>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the other is simply a
>>>>> request
>>>>>>> to
>>>>>>>>>> append
>>>>>>>>>>>>> to
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> log.
>>>>>>>>>>>>>>>>>>> Not
>>>>>>>>>>>>>>>>>>>>>> sure
>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> structure here for
>>>>> ordering,
>>>>>>> but
>>>>>>>> my
>>>>>>>>>>>>>>>> understanding
>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> broker
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handle the write request
>>>>>> before
>>>>>>>> it
>>>>>>>>>> hears
>>>>>>>>>>>>>> back
>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> Txn
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Coordinator.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if I
>>>>>> misunderstood
>>>>>>>>>> something
>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>>>>>> unclear.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 1, 2022 at
>>>>> 12:15
>>>>>> PM
>>>>>>>>>> Matthias
>>>>>>>>>>>>> J.
>>>>>>>>>>>>>>> Sax
>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the details
>>>>>>> Justine!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side change
>>>>> for
>>>>>> 2
>>>>>>> is
>>>>>>>>>>>>> removing
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> addPartitions
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need to
>>>>> make
>>>>>>>> this
>>>>>>>>>> from
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> txn
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think I did not
>>>> express
>>>>>>> myself
>>>>>>>>>>>>> clearly. I
>>>>>>>>>>>>>>>>>>> understand
>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>> (and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should) change the
>>>>> producer
>>>>>> to
>>>>>>>> not
>>>>>>>>>> send
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
>>>>>>>>>>>>>>>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> longer. But I don't
>>>> thinks
>>>>>>> it's
>>>>>>>>>>>>> requirement
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> change
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> broker?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What I am trying to say
>>>>> is:
>>>>>>> as a
>>>>>>>>>>>>> safe-guard
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>> improvement
>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>> older
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producers, the partition
>>>>>>> leader
>>>>>>>> can
>>>>>>>>>> just
>>>>>>>>>>>>>> send
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request to the
>>>>>> TX-coordinator
>>>>>>>> in any
>>>>>>>>>>>>> case
>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> old
>>>>>>>>>>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> correctly did send the
>>>>>>>>>> `addPartition`
>>>>>>>>>>>>>> request
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> already, the
>>>>> TX-coordinator
>>>>>>> can
>>>>>>>> just
>>>>>>>>>>>>>> "ignore"
>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>> idempotent.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if the old producer has
>>>> a
>>>>>> bug
>>>>>>>> and
>>>>>>>>>> did
>>>>>>>>>>>>>> forget
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> sent
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> `addPartition`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request, we would now
>>>>> ensure
>>>>>>>> that
>>>>>>>>>> the
>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>> indeed
>>>>>>>>>>>>>>>>>>>>>>>> added
>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX and thus fix a
>>>>> potential
>>>>>>>>>> producer bug
>>>>>>>>>>>>>>> (even
>>>>>>>>>>>>>>>>>> if we
>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fencing via the bump
>>>>> epoch).
>>>>>>> --
>>>>>>>> It
>>>>>>>>>>>>> seems to
>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>>>>>>> improvement?
>>>>>>>>>>>>>>>>>>>>>>>>>>> Or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there a reason to not do
>>>>>> this?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is ongoing
>>>> =
>>>>>>>> partition
>>>>>>>>>> was
>>>>>>>>>>>>>> added
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>> via
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn. We
>>>>>> check
>>>>>>>> this
>>>>>>>>>> with
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this wasn't
>>>>>>>> sufficiently
>>>>>>>>>>>>>>> explained
>>>>>>>>>>>>>>>>>> here:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do what I propose
>>>> in
>>>>>>>> (20), we
>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `DescribeTransaction`
>>>>> call,
>>>>>> as
>>>>>>>> the
>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>> leader
>>>>>>>>>>>>>>>>>>>>> adds
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for older clients and we
>>>>> get
>>>>>>>> this
>>>>>>>>>> check
>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>> free.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is that
>>>> if
>>>>>> any
>>>>>>>>>> messages
>>>>>>>>>>>>>>> somehow
>>>>>>>>>>>>>>>>>> come
>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the producer,
>>>>> they
>>>>>>>> will be
>>>>>>>>>>>>>> fenced.
>>>>>>>>>>>>>>>>>>> However,
>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it can be
>>>>>>>> discussed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree that we should
>>>>> have
>>>>>>>> epoch
>>>>>>>>>>>>> fencing.
>>>>>>>>>>>>>> My
>>>>>>>>>>>>>>>>>>>>> question is
>>>>>>>>>>>>>>>>>>>>>>>>>>> different:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Assume we are at epoch
>>>> 73,
>>>>>> and
>>>>>>>> we
>>>>>>>>>> have
>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>> ongoing
>>>>>>>>>>>>>>>>>>>>>>>> transaction,
>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed. It seems
>>>>> natural
>>>>>> to
>>>>>>>>>> write the
>>>>>>>>>>>>>>>> "prepare
>>>>>>>>>>>>>>>>>>>>> commit"
>>>>>>>>>>>>>>>>>>>>>>>>> marker
>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
>>>>> both
>>>>>>> with
>>>>>>>>>> epoch
>>>>>>>>>>>>> 73,
>>>>>>>>>>>>>>> too,
>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>> belongs
>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current transaction. Of
>>>>>>> course,
>>>>>>>> we
>>>>>>>>>> now
>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>> bump
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>> expect
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the next requests to
>>>> have
>>>>>>> epoch
>>>>>>>> 74,
>>>>>>>>>> and
>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>> reject
>>>>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch 73, as the
>>>>>> corresponding
>>>>>>>> TX
>>>>>>>>>> for
>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>> 73
>>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>>>>>>>>>>>>> committed.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems you propose to
>>>>>> write
>>>>>>>> the
>>>>>>>>>>>>> "prepare
>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>>> marker"
>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
>>>>> with
>>>>>>>> epoch 74
>>>>>>>>>>>>>> though,
>>>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>> work,
>>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems confusing. Is
>>>> there
>>>>> a
>>>>>>>> reason
>>>>>>>>>> why
>>>>>>>>>>>>> we
>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> bumped
>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 74
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of the current
>>>>> epoch
>>>>>>> 73?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are checking if
>>>>> the
>>>>>>>>>>>>> transaction is
>>>>>>>>>>>>>>>>>> ongoing,
>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the leader
>>>>>>> partition
>>>>>>>> to
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
>>>>>>>>>>>>>>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for this
>>>>>>>> message to
>>>>>>>>>> come
>>>>>>>>>>>>>>> back,
>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>> theory
>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
>>>> that
>>>>>>> would
>>>>>>>>>> make the
>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why we
>>>> can
>>>>>>> check
>>>>>>>> the
>>>>>>>>>>>>> leader
>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks. Got it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, is this really
>>>> an
>>>>>>>> issue?
>>>>>>>>>> We put
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> produce
>>>>>>>>>>>>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory, so how could
>>>> we
>>>>>>>> process
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
>>>>>>>>>>>>>>>>>>>>>>>>>>> first?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Don't we need to put the
>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>>> purgatory,
>>>>>>>>>>>>>>>>>>>>>>>>>>> too,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for this case, and
>>>> process
>>>>>>> both
>>>>>>>>>> request
>>>>>>>>>>>>>>>> in-order?
>>>>>>>>>>>>>>>>>>>>> (Again,
>>>>>>>>>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>> broker
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> knowledge is limited and
>>>>>> maybe
>>>>>>>> we
>>>>>>>>>> don't
>>>>>>>>>>>>>>>> maintain
>>>>>>>>>>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>>>>>>>>> order
>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case, what seems to be
>>>> an
>>>>>>> issue
>>>>>>>>>> IMHO,
>>>>>>>>>>>>> and I
>>>>>>>>>>>>>>> am
>>>>>>>>>>>>>>>>>>>>> wondering
>>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>> changing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request handling to
>>>>> preserve
>>>>>>>> order
>>>>>>>>>> for
>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>> might be
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> cleaner
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution?)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/30/22 3:28 PM,
>>>> Artem
>>>>>>>> Livshits
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think the interesting
>>>>>> part
>>>>>>> is
>>>>>>>>>> not in
>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>> (because
>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>> tries
>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> figure out when
>>>>>>>>>> UNKNOWN_PRODUCER_ID is
>>>>>>>>>>>>>>>> retriable
>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retryable,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's definitely not
>>>>> fatal),
>>>>>>> but
>>>>>>>>>> what
>>>>>>>>>>>>>> happens
>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'true' and falls
>>>> through.
>>>>>> In
>>>>>>>> the
>>>>>>>>>> old
>>>>>>>>>>>>>>> clients
>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>> fatal,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> keep the behavior in
>>>> the
>>>>>> new
>>>>>>>>>> clients,
>>>>>>>>>>>>> I'd
>>>>>>>>>>>>>>>>>> expect it
>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>> fatal
>>>>>>>>>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> well.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 29, 2022 at
>>>>>> 11:57
>>>>>>>> AM
>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>> Olshan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>> <jolshan@confluent.io.invalid
>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Artem and Jeff,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
>>>> look
>>>>>> and
>>>>>>>>>> sorry for
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> slow
>>>>>>>>>>>>>>>>>>>>>> response.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You both mentioned the
>>>>>>> change
>>>>>>>> to
>>>>>>>>>>>>> handle
>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
>>>>>>>>>>>>>>>>>>>>>>>>>>> errors.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear — this error
>>>> code
>>>>>> will
>>>>>>>> only
>>>>>>>>>> be
>>>>>>>>>>>>> sent
>>>>>>>>>>>>>>>> again
>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> client's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version is high enough
>>>>> to
>>>>>>>> ensure
>>>>>>>>>> we
>>>>>>>>>>>>>> handle
>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>> correctly.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The current (Java)
>>>>> client
>>>>>>>> handles
>>>>>>>>>>>>> this by
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>> (somewhat
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> long)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code snippet:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // An
>>>>> UNKNOWN_PRODUCER_ID
>>>>>>>> means
>>>>>>>>>> that
>>>>>>>>>>>>> we
>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>> lost
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker. Depending on
>>>> the
>>>>>> log
>>>>>>>> start
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // offset, we may want
>>>>> to
>>>>>>>> retry
>>>>>>>>>>>>> these, as
>>>>>>>>>>>>>>>>>>> described
>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below. If
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> none of those apply,
>>>>> then
>>>>>>> for
>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // idempotent
>>>> producer,
>>>>> we
>>>>>>>> will
>>>>>>>>>>>>> locally
>>>>>>>>>>>>>>> bump
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>> reset
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence numbers of
>>>>>>> in-flight
>>>>>>>>>> batches
>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // sequence 0, then
>>>>> retry
>>>>>>> the
>>>>>>>>>> failed
>>>>>>>>>>>>>> batch,
>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>>> now
>>>>>>>>>>>>>>>>>>>>>>>>>>>> succeed.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the transactional
>>>>>> producer,
>>>>>>>> allow
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // batch to fail. When
>>>>>>>> processing
>>>>>>>>>> the
>>>>>>>>>>>>>>> failed
>>>>>>>>>>>>>>>>>>> batch,
>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transition
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an abortable error and
>>>>>> set a
>>>>>>>> flag
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // indicating that we
>>>>> need
>>>>>>> to
>>>>>>>>>> bump the
>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>> (if
>>>>>>>>>>>>>>>>>>>>>>> supported
>>>>>>>>>>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if (error ==
>>>>>>>>>>>>>> Errors.*UNKNOWN_PRODUCER_ID*)
>>>>>>>>>>>>>>> {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>         if
>>>>>>>> (response.logStartOffset
>>>>>>>>>> ==
>>>>>>>>>>>>> -1)
>>>>>>>>>>>>>> {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // We don't
>>>>> know
>>>>>>>> the log
>>>>>>>>>>>>> start
>>>>>>>>>>>>>>>> offset
>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>> response.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the request
>>>>>> until
>>>>>>>> we
>>>>>>>>>> get
>>>>>>>>>>>>> it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // The
>>>>>>>>>> UNKNOWN_PRODUCER_ID
>>>>>>>>>>>>>> error
>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>>> added
>>>>>>>>>>>>>>>>>>>>>>>>> along
>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProduceResponse which
>>>>>>>> includes the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             //
>>>>>> logStartOffset.
>>>>>>>> So
>>>>>>>>>> the
>>>>>>>>>>>>> '-1'
>>>>>>>>>>>>>>>>>> sentinel
>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>> backward
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatibility.
>>>> Instead,
>>>>> it
>>>>>>> is
>>>>>>>>>> possible
>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // a broker
>>>> to
>>>>>> not
>>>>>>>> know
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> logStartOffset at
>>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returning
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response because
>>>> the
>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // may have
>>>>>> moved
>>>>>>>> away
>>>>>>>>>> from
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> broker
>>>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> time
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initially raised to
>>>> the
>>>>>> time
>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // response
>>>>> was
>>>>>>>> being
>>>>>>>>>>>>>>> constructed.
>>>>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>>>>>> cases,
>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retry the request: we
>>>>> are
>>>>>>>>>> guaranteed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // to
>>>>> eventually
>>>>>>>> get a
>>>>>>>>>>>>>>>> logStartOffset
>>>>>>>>>>>>>>>>>>> once
>>>>>>>>>>>>>>>>>>>>>>> things
>>>>>>>>>>>>>>>>>>>>>>>>>>> settle
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> down.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             return true;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>         }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>         if
>>>>>>>>>>>>> (batch.sequenceHasBeenReset()) {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // When the
>>>>>> first
>>>>>>>>>> inflight
>>>>>>>>>>>>>> batch
>>>>>>>>>>>>>>>>>> fails
>>>>>>>>>>>>>>>>>>>>> due to
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> truncation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> then the sequences of
>>>>> all
>>>>>>> the
>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // in flight
>>>>>>> batches
>>>>>>>>>> would
>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>>> restarted
>>>>>>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> beginning.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, when those
>>>>>>> responses
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // come back
>>>>>> from
>>>>>>>> the
>>>>>>>>>>>>> broker,
>>>>>>>>>>>>>>> they
>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>>>>>> come
>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
>>>>> error.
>>>>>>> In
>>>>>>>> this
>>>>>>>>>>>>> case,
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // reset the
>>>>>>>> sequence
>>>>>>>>>>>>> numbers
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> beginning.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             return true;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>         } else if
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>> (lastAckedOffset(batch.topicPartition).orElse(
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
>>>>>>>>>>>>>>>>>>>>>>>> response.logStartOffset) {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // The head
>>>> of
>>>>>> the
>>>>>>>> log
>>>>>>>>>> has
>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>> removed,
>>>>>>>>>>>>>>>>>>>>>>> probably
>>>>>>>>>>>>>>>>>>>>>>>>> due
>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retention time
>>>> elapsing.
>>>>>> In
>>>>>>>> this
>>>>>>>>>> case,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // we expect
>>>>> to
>>>>>>>> lose the
>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>> state.
>>>>>>>>>>>>>>>>>>>>> For
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer, reset the
>>>>>>> sequences
>>>>>>>> of
>>>>>>>>>> all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // inflight
>>>>>>> batches
>>>>>>>> to
>>>>>>>>>> be
>>>>>>>>>>>>> from
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> beginning
>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>> retry
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the transaction
>>>>> does
>>>>>>> not
>>>>>>>>>> need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // be
>>>> aborted.
>>>>>> For
>>>>>>>> the
>>>>>>>>>>>>>> idempotent
>>>>>>>>>>>>>>>>>>>>> producer,
>>>>>>>>>>>>>>>>>>>>>>> bump
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> avoid
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reusing (sequence,
>>>>> epoch)
>>>>>>>> pairs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             if
>>>>>>>> (isTransactional()) {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>> txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>> this.producerIdAndEpoch);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             } else {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>> requestEpochBumpForPartition(batch.topicPartition);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             return true;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>         }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>         if
>>>>>>> (!isTransactional())
>>>>>>>> {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // For the
>>>>>>>> idempotent
>>>>>>>>>>>>> producer,
>>>>>>>>>>>>>>>>>> always
>>>>>>>>>>>>>>>>>>>>> retry
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors. If the batch
>>>> has
>>>>>> the
>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             // producer
>>>> ID
>>>>>> and
>>>>>>>>>> epoch,
>>>>>>>>>>>>>>> request a
>>>>>>>>>>>>>>>>>> bump
>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Otherwise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the
>>>> produce.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>> requestEpochBumpForPartition(batch.topicPartition);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             return true;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>         }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I was considering
>>>>> keeping
>>>>>>> this
>>>>>>>>>>>>> behavior —
>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>> am
>>>>>>>>>>>>>>>>>>>>> open
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> simplifying
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We are leaving changes
>>>>> to
>>>>>>>> older
>>>>>>>>>>>>> clients
>>>>>>>>>>>>>> off
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> table
>>>>>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> caused
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many issues for
>>>> clients
>>>>> in
>>>>>>> the
>>>>>>>>>> past.
>>>>>>>>>>>>>>>> Previously
>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>> fatal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we didn't have the
>>>>>>> mechanisms
>>>>>>>> in
>>>>>>>>>>>>> place to
>>>>>>>>>>>>>>>>>> detect
>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> legitimate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case vs some bug or
>>>> gap
>>>>> in
>>>>>>> the
>>>>>>>>>>>>> protocol.
>>>>>>>>>>>>>>>>>> Ensuring
>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> own epoch should close
>>>>>> this
>>>>>>>> gap.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> And to address Jeff's
>>>>>> second
>>>>>>>>>> point:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *does the typical
>>>>> produce
>>>>>>>> request
>>>>>>>>>> path
>>>>>>>>>>>>>>> append
>>>>>>>>>>>>>>>>>>>>> records
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> local
>>>>>>>>>>>>>>>>>>>>>>>>>>> log
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along*
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *with the
>>>>>>>> currentTxnFirstOffset
>>>>>>>>>>>>>>> information?
>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand*
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *when the field is
>>>>> written
>>>>>>> to
>>>>>>>>>> disk.*
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, the first produce
>>>>>>> request
>>>>>>>>>>>>> populates
>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>> field
>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>> writes
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> offset
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as part of the record
>>>>>> batch
>>>>>>>> and
>>>>>>>>>> also
>>>>>>>>>>>>> to
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>>>>>>> snapshot.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we reload the records
>>>> on
>>>>>>>> restart
>>>>>>>>>>>>> and/or
>>>>>>>>>>>>>>>>>>>>> reassignment,
>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>> repopulate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> field with the
>>>> snapshot
>>>>>> from
>>>>>>>> disk
>>>>>>>>>>>>> along
>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> rest
>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if there
>>>> are
>>>>>>>> further
>>>>>>>>>>>>> comments
>>>>>>>>>>>>>>>>>> and/or
>>>>>>>>>>>>>>>>>>>>>>> questions.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
>>>> at
>>>>>> 9:00
>>>>>>>> PM
>>>>>>>>>> Jeff
>>>>>>>>>>>>> Kim
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>> <jeff.kim@confluent.io.invalid
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP! I
>>>>>> have
>>>>>>>> two
>>>>>>>>>>>>>> questions:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1) For new clients,
>>>> we
>>>>>> can
>>>>>>>> once
>>>>>>>>>> again
>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for sequences
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that are non-zero
>>>> when
>>>>>>> there
>>>>>>>> is
>>>>>>>>>> no
>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>> present
>>>>>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This will indicate we
>>>>>>> missed
>>>>>>>> the
>>>>>>>>>> 0
>>>>>>>>>>>>>>> sequence
>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>> yet
>>>>>>>>>>>>>>>>>>>>>>>>>>> want
>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the log.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would like to
>>>>>> understand
>>>>>>>> the
>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>> behavior
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> handle
>>>>>>>>>>>>>>>>>>>>>>>>>> older
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and if there are any
>>>>>>> changes
>>>>>>>> we
>>>>>>>>>> are
>>>>>>>>>>>>>>> making.
>>>>>>>>>>>>>>>>>> Maybe
>>>>>>>>>>>>>>>>>>>>> I'm
>>>>>>>>>>>>>>>>>>>>>>>>> missing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but we would want to
>>>>>>> identify
>>>>>>>>>>>>> whether we
>>>>>>>>>>>>>>>>>> missed
>>>>>>>>>>>>>>>>>>>>> the 0
>>>>>>>>>>>>>>>>>>>>>>>>> sequence
>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients, no?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2) Upon returning
>>>> from
>>>>>> the
>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>> coordinator, we
>>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>> set
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as ongoing on the
>>>>> leader
>>>>>> by
>>>>>>>>>>>>> populating
>>>>>>>>>>>>>>>>>>>>>>>> currentTxnFirstOffset
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through the typical
>>>>>> produce
>>>>>>>>>> request
>>>>>>>>>>>>>>>> handling.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does the typical
>>>>> produce
>>>>>>>> request
>>>>>>>>>> path
>>>>>>>>>>>>>>> append
>>>>>>>>>>>>>>>>>>>>> records
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> local
>>>>>>>>>>>>>>>>>>>>>>>>>>> log
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>> currentTxnFirstOffset
>>>>>>>>>>>>>>> information?
>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when the field is
>>>>> written
>>>>>>> to
>>>>>>>>>> disk.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeff
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
>>>> at
>>>>>>> 4:44
>>>>>>>> PM
>>>>>>>>>> Artem
>>>>>>>>>>>>>>>> Livshits
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
>>>>> alivshits@confluent.io
>>>>>>>> .invalid>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for the
>>>> KIP.
>>>>>> I
>>>>>>>> have
>>>>>>>>>> one
>>>>>>>>>>>>>>>> question.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 5) For new clients,
>>>> we
>>>>>> can
>>>>>>>> once
>>>>>>>>>>>>> again
>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe we had
>>>>>> problems
>>>>>>>> in the
>>>>>>>>>>>>> past
>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>> returning
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because it was
>>>>>> considered
>>>>>>>> fatal
>>>>>>>>>> and
>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>> client
>>>>>>>>>>>>>>>>>>>>>>>>> restart.
>>>>>>>>>>>>>>>>>>>>>>>>>>> It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good to spell out
>>>> the
>>>>>> new
>>>>>>>> client
>>>>>>>>>>>>>> behavior
>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>> receives
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
>>>>> at
>>>>>>>> 10:00 AM
>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>> Olshan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>> <jo...@confluent.io.invalid>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
>>>>>> look
>>>>>>>>>> Matthias.
>>>>>>>>>>>>>> I've
>>>>>>>>>>>>>>>>>> tried
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> answer
>>>>>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> questions
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 10)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Right — so the
>>>>> hanging
>>>>>>>>>> transaction
>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>> occurs
>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>> late
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> come in and the
>>>>>> partition
>>>>>>>> is
>>>>>>>>>> never
>>>>>>>>>>>>>> added
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>> again.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never add the
>>>>> partition
>>>>>>> to
>>>>>>>> a
>>>>>>>>>>>>>>> transaction,
>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>> never
>>>>>>>>>>>>>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> marker
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never advance the
>>>>> LSO.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do end up
>>>>> adding
>>>>>>> the
>>>>>>>>>>>>> partition
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>> (I
>>>>>>>>>>>>>>>>>>>>>>>>>>>> suppose
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happen before or
>>>>> after
>>>>>>> the
>>>>>>>> late
>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>> comes
>>>>>>>>>>>>>>>>>>> in)
>>>>>>>>>>>>>>>>>>>>>> then
>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> include
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late message in the
>>>>>> next
>>>>>>>>>>>>> (incorrect)
>>>>>>>>>>>>>>>>>>> transaction.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> So perhaps it is
>>>>>> clearer
>>>>>>> to
>>>>>>>>>> make
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> distinction
>>>>>>>>>>>>>>>>>>>>>>> between
>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> eventually get
>>>> added
>>>>> to
>>>>>>> the
>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>> (but
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> wrong
>>>>>>>>>>>>>>>>>>>>>>>>> one)
>>>>>>>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that never get
>>>> added
>>>>>> and
>>>>>>>> become
>>>>>>>>>>>>>> hanging.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side
>>>>> change
>>>>>>> for
>>>>>>>> 2 is
>>>>>>>>>>>>>> removing
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> addPartitions
>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need
>>>>> to
>>>>>>> make
>>>>>>>>>> this
>>>>>>>>>>>>> from
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> txn
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In my opinion, the
>>>>>> issue
>>>>>>>> with
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
>>>>>>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is that we don't
>>>> have
>>>>>> the
>>>>>>>> epoch
>>>>>>>>>>>>> bump,
>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> belongs to the
>>>>> previous
>>>>>>>>>>>>> transaction or
>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>> one.
>>>>>>>>>>>>>>>>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition has been
>>>>>> added
>>>>>>> to
>>>>>>>>>> this
>>>>>>>>>>>>>>>>>> transaction.
>>>>>>>>>>>>>>>>>>> Of
>>>>>>>>>>>>>>>>>>>>>>> course,
>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> means
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't completely
>>>>> cover
>>>>>>> the
>>>>>>>> case
>>>>>>>>>>>>> where
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>>>>>>>> late
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have added the
>>>>>> partition
>>>>>>> to
>>>>>>>>>> the new
>>>>>>>>>>>>>>>>>>> transaction,
>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>> that's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something we will
>>>>> need
>>>>>>> the
>>>>>>>> new
>>>>>>>>>>>>> clients
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> cover.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is
>>>>> ongoing
>>>>>> =
>>>>>>>>>> partition
>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>> added to
>>>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>> via
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn.
>>>>> We
>>>>>>>> check
>>>>>>>>>> this
>>>>>>>>>>>>> with
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this
>>>>> wasn't
>>>>>>>>>> sufficiently
>>>>>>>>>>>>>>>>>> explained
>>>>>>>>>>>>>>>>>>>>> here:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is
>>>> that
>>>>>> if
>>>>>>>> any
>>>>>>>>>>>>> messages
>>>>>>>>>>>>>>>>>> somehow
>>>>>>>>>>>>>>>>>>>>> come
>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the
>>>>> producer,
>>>>>>> they
>>>>>>>>>> will be
>>>>>>>>>>>>>>>> fenced.
>>>>>>>>>>>>>>>>>>>>> However,
>>>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it
>>>> can
>>>>> be
>>>>>>>>>> discussed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 50)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It should be
>>>>>> synchronous
>>>>>>>>>> because
>>>>>>>>>>>>> if we
>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>> event
>>>>>>>>>>>>>>>>>>>>>>>> (ie,
>>>>>>>>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes us to need
>>>> to
>>>>>>> abort
>>>>>>>> the
>>>>>>>>>>>>>>>> transaction,
>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions to send
>>>>>>>> transaction
>>>>>>>>>>>>> markers
>>>>>>>>>>>>>>> to.
>>>>>>>>>>>>>>>>>> We
>>>>>>>>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we added them to
>>>> the
>>>>>>>>>> coordinator
>>>>>>>>>>>>> via
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Previously we have
>>>>> had
>>>>>>>>>> asynchronous
>>>>>>>>>>>>>>> calls
>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> past
>>>>>>>>>>>>>>>>>>>>>>>> (ie,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit markers when
>>>>> the
>>>>>>>>>>>>> transaction is
>>>>>>>>>>>>>>>>>>> completed)
>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>> often
>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes confusion as
>>>>> we
>>>>>>>> need to
>>>>>>>>>> wait
>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>> operations
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complete.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing commit
>>>>> markers
>>>>>>>> case,
>>>>>>>>>>>>> clients
>>>>>>>>>>>>>>> often
>>>>>>>>>>>>>>>>>> see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>> CONCURRENT_TRANSACTIONs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error messages and
>>>>> that
>>>>>>>> can be
>>>>>>>>>>>>>>> confusing.
>>>>>>>>>>>>>>>>>> For
>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>> reason,
>>>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>> may
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simpler to just
>>>> have
>>>>>>>>>> synchronous
>>>>>>>>>>>>>> calls —
>>>>>>>>>>>>>>>>>>>>> especially
>>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> block
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some operation's
>>>>>>> completion
>>>>>>>>>> anyway
>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>> start
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. And
>>>>> yes, I
>>>>>>>> meant
>>>>>>>>>>>>>>>> coordinator. I
>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>> fix
>>>>>>>>>>>>>>>>>>>>>>>>> that.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are
>>>> checking
>>>>> if
>>>>>>> the
>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>> ongoing,
>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the
>>>> leader
>>>>>>>> partition
>>>>>>>>>> to
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for
>>>>> this
>>>>>>>>>> message to
>>>>>>>>>>>>>> come
>>>>>>>>>>>>>>>>>> back,
>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>> theory
>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
>>>>>> that
>>>>>>>> would
>>>>>>>>>> make
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why
>>>> we
>>>>>> can
>>>>>>>> check
>>>>>>>>>> the
>>>>>>>>>>>>>>> leader
>>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm happy to update
>>>>> the
>>>>>>>> KIP if
>>>>>>>>>>>>> some of
>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>>> things
>>>>>>>>>>>>>>>>>>>>>>> were
>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Nov 21,
>>>> 2022
>>>>> at
>>>>>>>> 7:11 PM
>>>>>>>>>>>>>> Matthias
>>>>>>>>>>>>>>>> J.
>>>>>>>>>>>>>>>>>>> Sax <
>>>>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the
>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Couple of
>>>>>> clarification
>>>>>>>>>> questions
>>>>>>>>>>>>> (I
>>>>>>>>>>>>>> am
>>>>>>>>>>>>>>>>>> not a
>>>>>>>>>>>>>>>>>>>>>> broker
>>>>>>>>>>>>>>>>>>>>>>>>> expert
>>>>>>>>>>>>>>>>>>>>>>>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some question are
>>>>>>> obvious
>>>>>>>> for
>>>>>>>>>>>>> others,
>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>> me
>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>> lack
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker knowledge).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (10)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What happens if
>>>> the
>>>>>>>> message
>>>>>>>>>> come
>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request? It seems
>>>>> the
>>>>>>>> broker
>>>>>>>>>>>>> hosting
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> data
>>>>>>>>>>>>>>>>>>>>>>>> partitions
>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anything about it
>>>>> and
>>>>>>>> append
>>>>>>>>>> it to
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> partition,
>>>>>>>>>>>>>>>>>>>>>>> too?
>>>>>>>>>>>>>>>>>>>>>>>>> What
>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> difference between
>>>>>> both
>>>>>>>> cases?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, it seems a
>>>> TX
>>>>>>> would
>>>>>>>> only
>>>>>>>>>>>>> hang,
>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>>>> is no
>>>>>>>>>>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> either committer
>>>> or
>>>>>>>> aborted?
>>>>>>>>>> Thus,
>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>> above,
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> TX
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> might
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually not hang
>>>>> (of
>>>>>>>> course,
>>>>>>>>>> we
>>>>>>>>>>>>>> might
>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>> EOS
>>>>>>>>>>>>>>>>>>>>>>>>> violation
>>>>>>>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX was aborted and
>>>>> the
>>>>>>>> second
>>>>>>>>>>>>>>> committed,
>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (20)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Of course, 1 and
>>>> 2
>>>>>>>> require
>>>>>>>>>>>>>> client-side
>>>>>>>>>>>>>>>>>>>>> changes, so
>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>> older
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> those approaches
>>>>> won’t
>>>>>>>> apply.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For (1) I
>>>> understand
>>>>>>> why a
>>>>>>>>>> client
>>>>>>>>>>>>>>> change
>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>> necessary,
>>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we need a client
>>>>>> change
>>>>>>>> for
>>>>>>>>>> (2).
>>>>>>>>>>>>> Can
>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>> elaborate?
>>>>>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>>>>>>>>> Later
>>>>>>>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> explain
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that we should
>>>> send
>>>>> a
>>>>>>>>>>>>>>>>>>>>> DescribeTransactionRequest,
>>>>>>>>>>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>>>>>>>>>> am
>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Can't we not just
>>>> do
>>>>>> an
>>>>>>>>>> implicit
>>>>>>>>>>>>>>>>>>>>> AddPartiitonToTx,
>>>>>>>>>>>>>>>>>>>>>>> too?
>>>>>>>>>>>>>>>>>>>>>>>>> If
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> old
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer correctly
>>>>>>>> registered
>>>>>>>>>> the
>>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>>>> already,
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can just ignore it
>>>>> as
>>>>>>>> it's an
>>>>>>>>>>>>>>> idempotent
>>>>>>>>>>>>>>>>>>>>> operation?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (30)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To cover older
>>>>>> clients,
>>>>>>>> we
>>>>>>>>>> will
>>>>>>>>>>>>>>> ensure a
>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>> ongoing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we write to a
>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Not sure what you
>>>>> mean
>>>>>>> by
>>>>>>>>>> this?
>>>>>>>>>>>>> Can
>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>> elaborate?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (40)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [the
>>>>> TX-coordinator]
>>>>>>> will
>>>>>>>>>> write
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> prepare
>>>>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch and send
>>>>>>>>>>>>> WriteTxnMarkerRequests
>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> bumped
>>>>>>>>>>>>>>>>>>>>>>>>>> epoch.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why do we use the
>>>>>> bumped
>>>>>>>>>> epoch for
>>>>>>>>>>>>>>> both?
>>>>>>>>>>>>>>>> It
>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>> intuitive
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the current epoch,
>>>>> and
>>>>>>>> only
>>>>>>>>>> return
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> bumped
>>>>>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (50) "Implicit
>>>>>>>>>>>>>>> AddPartitionToTransaction"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why does the
>>>>>> implicitly
>>>>>>>> sent
>>>>>>>>>>>>> request
>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>> synchronous?
>>>>>>>>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also says
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in case we need
>>>> to
>>>>>>> abort
>>>>>>>> and
>>>>>>>>>>>>> need to
>>>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>>>> partitions
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What do you mean
>>>> by
>>>>>>> this?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we don’t want to
>>>>>> write
>>>>>>>> to it
>>>>>>>>>>>>> before
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> manager
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Do you mean
>>>>>>> TX-coordinator
>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>> "manager"?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (60)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For older clients
>>>>> and
>>>>>>>> ensuring
>>>>>>>>>>>>> that
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> TX
>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>> ongoing,
>>>>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> describe a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> race condition. I
>>>> am
>>>>>> not
>>>>>>>> sure
>>>>>>>>>> if I
>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>> follow
>>>>>>>>>>>>>>>>>>>>> here.
>>>>>>>>>>>>>>>>>>>>>>> Can
>>>>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> elaborate?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/18/22 1:21
>>>> PM,
>>>>>>>> Justine
>>>>>>>>>>>>> Olshan
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey all!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to
>>>> start a
>>>>>>>>>> discussion
>>>>>>>>>>>>> on my
>>>>>>>>>>>>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> add
>>>>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checks on
>>>>>> transactions
>>>>>>> to
>>>>>>>>>> avoid
>>>>>>>>>>>>>>> hanging
>>>>>>>>>>>>>>>>>>>>>>> transactions.
>>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an issue for some
>>>>>> time,
>>>>>>>> so I
>>>>>>>>>>>>> really
>>>>>>>>>>>>>>> hope
>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> helpful
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users of EOS.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The KIP includes
>>>>>>> changes
>>>>>>>> that
>>>>>>>>>>>>> will
>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>> compatible
>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>> old
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> changes to
>>>> improve
>>>>>>>>>> performance
>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> correctness
>>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>> clients.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Please take a
>>>> look
>>>>>> and
>>>>>>>> leave
>>>>>>>>>> any
>>>>>>>>>>>>>>>> comments
>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>> may
>>>>>>>>>>>>>>>>>>>>>>>> have!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> JIRA:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-14402
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
> 

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
Hey Matthias,

I have actually never heard of KIP-280 so thanks for bringing it up. That
seems interesting. I wonder how it would work though -- would it build an
offset map with just the latest timestamp for a key? I wonder if ordering
assumptions are baked in there, why not use offset-based compaction.

I was also not aware of this "guarantee" with regards to broker side time.
I think that we can do in order handling for a given producer, but not
across all producers. However, we can't guarantee that anyway.

Let me know if you have any concerns here.

Thanks,
Justine

On Mon, Jan 23, 2023 at 6:33 PM Matthias J. Sax <mj...@apache.org> wrote:

> Just a side note about Guozhang comments about timestamps.
>
> If the producer sets the timestamp, putting the record into purgatory
> seems not to be an issue (as already said: for this case we don't
> guarantee timestamp order between writes of different producers anyway).
> However, if the broker sets the timestamp, the expectation is that there
> is no out-of-order data in the partition ever; if we would introduce
> out-of-order data for this case (for interleaved writes of different
> producers), it seems we would violate the current contract? (To be fair:
> I don't know if that's an official contract, but I assume people rely on
> this behavior -- and it "advertised" in many public talks...)
>
> About compaction: there is actually KIP-280 that adds timestamp based
> compaction what is a very useful feature for Kafka Streams with regard
> to out-of-order data handling. So the impact if we introduce
> out-of-order data could be larger scoped.
>
>
> -Matthias
>
>
> On 1/20/23 4:48 PM, Justine Olshan wrote:
> > Hey Artem,
> >
> > I see there is a check for transactional producers. I'm wondering if we
> > don't handle the epoch overflow case. I'm also not sure it will be a huge
> > issue to extend to transactional producers, but maybe I'm missing
> something.
> >
> > As for the recovery path -- I think Guozhang's point was if we have a bad
> > client that repeatedly tries to produce without adding to the transaction
> > we would do the following:
> > a) if not fatal, we just fail the produce request over and over
> > b) if fatal, we fence the producer
> >
> > Here with B, the issue with the client would be made clear more quickly.
> I
> > suppose there are some intermediate cases where the issue only occurs
> > sometimes, but I wonder if we should consider how to recover with clients
> > who don't behave as expected anyway.
> >
> > I think there is a place for the abortable error that we are adding --
> just
> > abort and try again. But I think there are also some cases where trying
> to
> > recover overcomplicates some logic. Especially if we are considering
> older
> > clients -- there I'm not sure if there's a ton we can do besides fail the
> > batch or fence the producer. With newer clients, we can consider more
> > options for what can just be recovered after aborting. But epochs might
> be
> > a hard one unless we also want to reset producer ID.
> >
> > Thanks,
> > Justine
> >
> >
> >
> > On Fri, Jan 20, 2023 at 3:59 PM Artem Livshits
> > <al...@confluent.io.invalid> wrote:
> >
> >>>   besides the poorly written client case
> >>
> >> A poorly written client could create a lot of grief to people who run
> Kafka
> >> brokers :-), so when deciding to make an error fatal I would see if
> there
> >> is a reasonable recovery path rather than how often it could happen.
> If we
> >> have solid implementation of transactions (which I hope we'll do as a
> >> result of this KIP), it would help to recover from a large class of
> errors
> >> by just aborting a transaction, even if the cause of error is a race
> >> condition or etc.
> >>
> >> -Artem
> >>
> >> On Fri, Jan 20, 2023 at 3:26 PM Justine Olshan
> >> <jo...@confluent.io.invalid>
> >> wrote:
> >>
> >>> Artem --
> >>> I guess the discussion path we were going down is when we expect to see
> >>> this error. I mentioned that it was hard to come up with cases for when
> >> the
> >>> producer would still be around to receive the error besides the poorly
> >>> written client case.
> >>> If we don't expect to have a producer to receive the response, it sort
> of
> >>> makes sense for it to be fatal.
> >>>
> >>> I had some discussion with Jason offline about the epoch being off
> cases
> >>> and I'm not sure we could find a ton (outside of produce requests)
> where
> >> we
> >>> could/should recover. I'd be happy to hear some examples though, maybe
> >> I'm
> >>> missing something.
> >>>
> >>> Thanks,
> >>> Justine
> >>>
> >>> On Fri, Jan 20, 2023 at 3:19 PM Artem Livshits
> >>> <al...@confluent.io.invalid> wrote:
> >>>
> >>>> In general, I'd like to avoid fatal errors as much as possible, in
> some
> >>>> sense fatal errors just push out recovery logic to the application
> >> which
> >>>> either complicates the application or leads to disruption (we've seen
> >>> cases
> >>>> when a transient broker error could lead to work stoppage when
> >>> applications
> >>>> need to be manually restarted).  I think we should strive to define
> >>>> recovery logic for most errors (and/or encapsulate it in the Kafka
> >> client
> >>>> as much as possible).
> >>>>
> >>>> One benefit of transactions is that they simplify recovery from
> errors,
> >>>> pretty much any error (that's not handled transparently by retries in
> >>> Kafka
> >>>> client) can be handled by the application via aborting the transaction
> >>> and
> >>>> repeating the transactional logic again.  One tricky error is an error
> >>>> during commit, because we don't know the outcome.  For commit errors,
> >> the
> >>>> recommendation should be to retry the commit until it returns the
> >>> specific
> >>>> result (committed or aborted).
> >>>>
> >>>> -Artem
> >>>>
> >>>> On Fri, Jan 20, 2023 at 2:52 PM Justine Olshan
> >>>> <jo...@confluent.io.invalid>
> >>>> wrote:
> >>>>
> >>>>> That's a fair point about other clients.
> >>>>>
> >>>>> I think the abortable error case is interesting because I'm curious
> >> how
> >>>>> other clients would handle this. I assume they would need to
> >> implement
> >>>>> handling for the error code unless they did something like "any
> >> unknown
> >>>>> error codes/any codes that aren't x,y,z are retriable." I would hope
> >>> that
> >>>>> unknown error codes were fatal, and if the code was implemented it
> >>> would
> >>>>> abort the transaction. But I will think on this too.
> >>>>>
> >>>>> As for InvalidRecord -- you mentioned it was not fatal, but I'm
> >> taking
> >>> a
> >>>>> look through the code. We would see this on handling the produce
> >>>> response.
> >>>>> If I recall correctly, we check if errors are retriable. I think this
> >>>> error
> >>>>> would not be retriable. But I guess the concern here is that it is
> >> not
> >>>>> enough for just that batch to fail. I guess I hadn't considered fully
> >>>>> fencing the old producer but there are valid arguments here why we
> >>> would
> >>>>> want to.
> >>>>>
> >>>>> Thanks,
> >>>>> Justine
> >>>>>
> >>>>> On Fri, Jan 20, 2023 at 2:35 PM Guozhang Wang <
> >>>> guozhang.wang.us@gmail.com>
> >>>>> wrote:
> >>>>>
> >>>>>> Thanks Justine for the replies! I agree with most of your thoughts.
> >>>>>>
> >>>>>> Just for 3/7), though I agree for our own AK producer, since we do
> >>>>>> "nextRequest(boolean hasIncompleteBatches)", we guarantee the
> >> end-txn
> >>>>>> would not be sent until we've effectively flushed, but I was
> >>> referring
> >>>>>> to any future bugs or other buggy clients that the same client may
> >>> get
> >>>>>> into this situation, in which case we should give the client a
> >> clear
> >>>>>> msg that "you did something wrong, and hence now you should fatally
> >>>>>> close yourself". What I'm concerned about is that, by seeing an
> >>>>>> "abortable error" or in some rare cases an "invalid record", the
> >>>>>> client could not realize "something that's really bad happened". So
> >>>>>> it's not about adding a new error, it's mainly about those real
> >> buggy
> >>>>>> situations causing such "should never happen" cases, the errors
> >>> return
> >>>>>> would not be informative enough.
> >>>>>>
> >>>>>> Thinking in other ways, if we believe that for most cases such
> >> error
> >>>>>> codes would not reach the original clients since they would be
> >>>>>> disconnected or even gone by that time, and only in some rare cases
> >>>>>> they would still be seen by the sending clients, then why not make
> >>>>>> them more fatal and more specific than generic.
> >>>>>>
> >>>>>> Guozhang
> >>>>>>
> >>>>>> On Fri, Jan 20, 2023 at 1:59 PM Justine Olshan
> >>>>>> <jo...@confluent.io.invalid> wrote:
> >>>>>>>
> >>>>>>> Hey Guozhang. Thanks for taking a look and for the detailed
> >>> comments!
> >>>>>> I'll
> >>>>>>> do my best to address below.
> >>>>>>>
> >>>>>>> 1. I see what you are saying here, but I think I need to look
> >>> through
> >>>>> the
> >>>>>>> sequence of events you mention. Typically we've seen this issue
> >> in
> >>> a
> >>>>> few
> >>>>>>> cases.
> >>>>>>>
> >>>>>>>   One is when we have a producer disconnect when trying to
> >> produce.
> >>>>>>> Typically in these cases, we abort the transaction. We've seen
> >> that
> >>>>> after
> >>>>>>> the markers are written, the disconnection can sometimes cause
> >> the
> >>>>>> request
> >>>>>>> to get flushed to the broker. In this case, we don't need client
> >>>>> handling
> >>>>>>> because the producer we are responding to is gone. We just needed
> >>> to
> >>>>> make
> >>>>>>> sure we didn't write to the log on the broker side. I'm trying to
> >>>> think
> >>>>>> of
> >>>>>>> a case where we do have the client to return to. I'd think the
> >> same
> >>>>>> client
> >>>>>>> couldn't progress to committing the transaction unless the
> >> produce
> >>>>>> request
> >>>>>>> returned right? Of course, there is the incorrectly written
> >> clients
> >>>>> case.
> >>>>>>> I'll think on this a bit more and let you know if I come up with
> >>>>> another
> >>>>>>> scenario when we would return to an active client when the
> >>>> transaction
> >>>>> is
> >>>>>>> no longer ongoing.
> >>>>>>>
> >>>>>>> I was not aware that we checked the result of a send after we
> >>> commit
> >>>>>>> though. I'll need to look into that a bit more.
> >>>>>>>
> >>>>>>> 2. There were some questions about this in the discussion. The
> >> plan
> >>>> is
> >>>>> to
> >>>>>>> handle overflow with the mechanism we currently have in the
> >>> producer.
> >>>>> If
> >>>>>> we
> >>>>>>> try to bump and the epoch will overflow, we actually allocate a
> >> new
> >>>>>>> producer ID. I need to confirm the fencing logic on the last
> >> epoch
> >>>> (ie,
> >>>>>> we
> >>>>>>> probably shouldn't allow any records to be produced with the
> >> final
> >>>>> epoch
> >>>>>>> since we can never properly fence that one).
> >>>>>>>
> >>>>>>> 3. I can agree with you that the current error handling is
> >> messy. I
> >>>>>> recall
> >>>>>>> taking a look at your KIP a while back, but I think I mostly saw
> >>> the
> >>>>>>> section about how the errors were wrapped. Maybe I need to take
> >>>> another
> >>>>>>> look. As for abortable error, the idea was that the handling
> >> would
> >>> be
> >>>>>>> simple -- if this error is seen, the transaction should be
> >> aborted
> >>> --
> >>>>> no
> >>>>>>> other logic about previous state or requests necessary. Is your
> >>>> concern
> >>>>>>> simply about adding new errors? We were hoping to have an error
> >>> that
> >>>>>> would
> >>>>>>> have one meaning and many of the current errors have a history of
> >>>>> meaning
> >>>>>>> different things on different client versions. That was the main
> >>>>>> motivation
> >>>>>>> for adding a new error.
> >>>>>>>
> >>>>>>> 4. This is a good point about record timestamp reordering.
> >>> Timestamps
> >>>>>> don't
> >>>>>>> affect compaction, but they do affect retention deletion. For
> >> that,
> >>>>> kafka
> >>>>>>> considers the largest timestamp in the segment, so I think a
> >> small
> >>>>> amount
> >>>>>>> of reordering (hopefully on the order of milliseconds or even
> >>>> seconds)
> >>>>>> will
> >>>>>>> be ok. We take timestamps from clients so there is already a
> >>>>> possibility
> >>>>>>> for some drift and non-monotonically increasing timestamps.
> >>>>>>>
> >>>>>>> 5. Thanks for catching. The error is there, but it's actually
> >> that
> >>>>> those
> >>>>>>> fields should be 4+! Due to how the message generator works, I
> >>>> actually
> >>>>>>> have to redefine those fields inside the
> >>>>> `"AddPartitionsToTxnTransaction`
> >>>>>>> block for it to build correctly. I'll fix it to be correct.
> >>>>>>>
> >>>>>>> 6. Correct -- we will only add the request to purgatory if the
> >>> cache
> >>>>> has
> >>>>>> no
> >>>>>>> ongoing transaction. I can change the wording to make that
> >> clearer
> >>>> that
> >>>>>> we
> >>>>>>> only place the request in purgatory if we need to contact the
> >>>>> transaction
> >>>>>>> coordinator.
> >>>>>>>
> >>>>>>> 7. We did take a look at some of the errors and it was hard to
> >> come
> >>>> up
> >>>>>> with
> >>>>>>> a good one. I agree that InvalidTxnStateException is ideal except
> >>> for
> >>>>> the
> >>>>>>> fact that it hasn't been returned on Produce requests before. The
> >>>> error
> >>>>>>> handling for clients is a bit vague (which is why I opened
> >>>> KAFKA-14439
> >>>>>>> <https://issues.apache.org/jira/browse/KAFKA-14439>), but the
> >>>> decision
> >>>>>> we
> >>>>>>> made here was to only return errors that have been previously
> >>>> returned
> >>>>> to
> >>>>>>> producers. As for not being fatal, I think part of the theory was
> >>>> that
> >>>>> in
> >>>>>>> many cases, the producer would be disconnected. (See point 1) and
> >>>> this
> >>>>>>> would just be an error to return from the server. I did plan to
> >>> think
> >>>>>> about
> >>>>>>> other cases, so let me know if you think of any as well!
> >>>>>>>
> >>>>>>> Lots to say! Let me know if you have further thoughts!
> >>>>>>> Justine
> >>>>>>>
> >>>>>>> On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang <
> >>>>>> guozhang.wang.us@gmail.com>
> >>>>>>> wrote:
> >>>>>>>
> >>>>>>>> Hello Justine,
> >>>>>>>>
> >>>>>>>> Thanks for the great write-up! I made a quick pass through it
> >> and
> >>>>> here
> >>>>>>>> are some thoughts (I have not been able to read through this
> >>> thread
> >>>>> so
> >>>>>>>> pardon me if they have overlapped or subsumed by previous
> >>>> comments):
> >>>>>>>>
> >>>>>>>> First are some meta ones:
> >>>>>>>>
> >>>>>>>> 1. I think we need to also improve the client's experience once
> >>> we
> >>>>>>>> have this defence in place. More concretely, say a user's
> >>> producer
> >>>>>>>> code is like following:
> >>>>>>>>
> >>>>>>>> future = producer.send();
> >>>>>>>> // producer.flush();
> >>>>>>>> producer.commitTransaction();
> >>>>>>>> future.get();
> >>>>>>>>
> >>>>>>>> Which resulted in the order of a) produce-request sent by
> >>> producer,
> >>>>> b)
> >>>>>>>> end-txn-request sent by producer, c) end-txn-response sent
> >> back,
> >>> d)
> >>>>>>>> txn-marker-request sent from coordinator to partition leader,
> >> e)
> >>>>>>>> produce-request finally received by the partition leader,
> >> before
> >>>> this
> >>>>>>>> KIP e) step would be accepted causing a dangling txn; now it
> >>> would
> >>>> be
> >>>>>>>> rejected in step e) which is good. But from the client's point
> >> of
> >>>>> view
> >>>>>>>> now it becomes confusing since the `commitTransaction()`
> >> returns
> >>>>>>>> successfully, but the "future" throws an invalid-epoch error,
> >> and
> >>>>> they
> >>>>>>>> are not sure if the transaction did succeed or not. In fact, it
> >>>>>>>> "partially succeeded" with some msgs being rejected but others
> >>>>>>>> committed successfully.
> >>>>>>>>
> >>>>>>>> Of course the easy way to avoid this is, always call
> >>>>>>>> "producer.flush()" before commitTxn and that's what we do
> >>>> ourselves,
> >>>>>>>> and what we recommend users do. But I suspect not everyone does
> >>> it.
> >>>>> In
> >>>>>>>> fact I just checked the javadoc in KafkaProducer and our code
> >>>> snippet
> >>>>>>>> does not include a `flush()` call. So I'm thinking maybe we can
> >>> in
> >>>>>>>> side the `commitTxn` code to enforce flushing before sending
> >> the
> >>>>>>>> end-txn request.
> >>>>>>>>
> >>>>>>>> 2. I'd like to clarify a bit details on "just add partitions to
> >>> the
> >>>>>>>> transaction on the first produce request during a transaction".
> >>> My
> >>>>>>>> understanding is that the partition leader's cache has the
> >>> producer
> >>>>> id
> >>>>>>>> / sequence / epoch for the latest txn, either on-going or is
> >>>>> completed
> >>>>>>>> (upon receiving the marker request from coordinator). When a
> >>>> produce
> >>>>>>>> request is received, if
> >>>>>>>>
> >>>>>>>> * producer's epoch < cached epoch, or producer's epoch ==
> >> cached
> >>>>> epoch
> >>>>>>>> but the latest txn is completed, leader directly reject with
> >>>>>>>> invalid-epoch.
> >>>>>>>> * producer's epoch > cached epoch, park the the request and
> >> send
> >>>>>>>> add-partitions request to coordinator.
> >>>>>>>>
> >>>>>>>> In order to do it, does the coordinator need to bump the
> >> sequence
> >>>> and
> >>>>>>>> reset epoch to 0 when the next epoch is going to overflow? If
> >> no
> >>>> need
> >>>>>>>> to do so, then how we handle the (admittedly rare, but still
> >> may
> >>>>>>>> happen) epoch overflow situation?
> >>>>>>>>
> >>>>>>>> 3. I'm a bit concerned about adding a generic "ABORTABLE_ERROR"
> >>>> given
> >>>>>>>> we already have a pretty messy error classification and error
> >>>>> handling
> >>>>>>>> on the producer clients side --- I have a summary about the
> >>> issues
> >>>>> and
> >>>>>>>> a proposal to address this in
> >>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
> >>>>>>>> -- I understand we do not want to use "UNKNOWN_PRODUCER_ID"
> >>> anymore
> >>>>>>>> and in fact we intend to deprecate it in KIP-360 and eventually
> >>>>> remove
> >>>>>>>> it; but I'm wondering can we still use specific error codes.
> >> E.g.
> >>>>> what
> >>>>>>>> about "InvalidProducerEpochException" since for new clients,
> >> the
> >>>>>>>> actual reason this would actually be rejected is indeed because
> >>> the
> >>>>>>>> epoch on the coordinator caused the add-partitions-request from
> >>> the
> >>>>>>>> brokers to be rejected anyways?
> >>>>>>>>
> >>>>>>>> 4. It seems we put the producer request into purgatory before
> >> we
> >>>> ever
> >>>>>>>> append the records, while other producer's records may still be
> >>>>>>>> appended during the time; and that potentially may result in
> >> some
> >>>>>>>> re-ordering compared with reception order. I'm not super
> >>> concerned
> >>>>>>>> about it since Kafka does not guarantee reception ordering
> >> across
> >>>>>>>> producers anyways, but it may make the timestamps of records
> >>>> inside a
> >>>>>>>> partition to be more out-of-ordered. Are we aware of any
> >>> scenarios
> >>>>>>>> such as future enhancements on log compactions that may be
> >>> affected
> >>>>> by
> >>>>>>>> this effect?
> >>>>>>>>
> >>>>>>>> Below are just minor comments:
> >>>>>>>>
> >>>>>>>> 5. In "AddPartitionsToTxnTransaction" field of
> >>>>>>>> "AddPartitionsToTxnRequest" RPC, the versions of those inner
> >>> fields
> >>>>>>>> are "0-3" while I thought they should be "0+" still?
> >>>>>>>>
> >>>>>>>> 6. Regarding "we can place the request in a purgatory of sorts
> >>> and
> >>>>>>>> check if there is any state for the transaction on the
> >> broker": i
> >>>>>>>> think at this time when we just do the checks against the
> >> cached
> >>>>>>>> state, we do not need to put the request to purgatory yet?
> >>>>>>>>
> >>>>>>>> 7. This is related to 3) above. I feel using
> >>>> "InvalidRecordException"
> >>>>>>>> for older clients may also be a bit confusing, and also it is
> >> not
> >>>>>>>> fatal -- for old clients, it better to be fatal since this
> >>>> indicates
> >>>>>>>> the clients is doing something wrong and hence it should be
> >>> closed.
> >>>>>>>> And in general I'd prefer to use slightly more specific meaning
> >>>> error
> >>>>>>>> codes for clients. That being said, I also feel
> >>>>>>>> "InvalidProducerEpochException" is not suitable for old
> >> versioned
> >>>>>>>> clients, and we'd have to pick one that old clients recognize.
> >>> I'd
> >>>>>>>> prefer "InvalidTxnStateException" but that one is supposed to
> >> be
> >>>>>>>> returned from txn coordinators only today. I'd suggest we do a
> >>>> quick
> >>>>>>>> check in the current client's code path and see if that one
> >> would
> >>>> be
> >>>>>>>> handled if it's from a produce-response, and if yes, use this
> >>> one;
> >>>>>>>> otherwise, use "ProducerFencedException" which is much less
> >>>>> meaningful
> >>>>>>>> but it's still a fatal error.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> Thanks,
> >>>>>>>> Guozhang
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
> >>>>>>>> <jo...@confluent.io.invalid> wrote:
> >>>>>>>>>
> >>>>>>>>> Yeah -- looks like we already have code to handle bumping the
> >>>> epoch
> >>>>>> and
> >>>>>>>>> when the epoch is Short.MAX_VALUE, we get a new producer ID.
> >>>> Since
> >>>>>> this
> >>>>>>>> is
> >>>>>>>>> already the behavior, do we want to change it further?
> >>>>>>>>>
> >>>>>>>>> Justine
> >>>>>>>>>
> >>>>>>>>> On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <
> >>>>> jolshan@confluent.io
> >>>>>>>
> >>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Hey all, just wanted to quickly update and say I've
> >> modified
> >>>> the
> >>>>>> KIP to
> >>>>>>>>>> explicitly mention that AddOffsetCommitsToTxnRequest will
> >> be
> >>>>>> replaced
> >>>>>>>> by
> >>>>>>>>>> a coordinator-side (inter-broker) AddPartitionsToTxn
> >> implicit
> >>>>>> request.
> >>>>>>>> This
> >>>>>>>>>> mirrors the user partitions and will implicitly add offset
> >>>>>> partitions
> >>>>>>>> to
> >>>>>>>>>> transactions when we commit offsets on them. We will
> >>> deprecate
> >>>>>>>> AddOffsetCommitsToTxnRequest
> >>>>>>>>>> for new clients.
> >>>>>>>>>>
> >>>>>>>>>> Also to address Artem's comments --
> >>>>>>>>>> I'm a bit unsure if the changes here will change the
> >> previous
> >>>>>> behavior
> >>>>>>>> for
> >>>>>>>>>> fencing producers. In the case you mention in the first
> >>>>> paragraph,
> >>>>>> are
> >>>>>>>> you
> >>>>>>>>>> saying we bump the epoch before we try to abort the
> >>>> transaction?
> >>>>> I
> >>>>>>>> think I
> >>>>>>>>>> need to understand the scenarios you mention a bit better.
> >>>>>>>>>>
> >>>>>>>>>> As for the second part -- I think it makes sense to have
> >> some
> >>>>> sort
> >>>>>> of
> >>>>>>>>>> "sentinel" epoch to signal epoch is about to overflow (I
> >>> think
> >>>> we
> >>>>>> sort
> >>>>>>>> of
> >>>>>>>>>> have this value in place in some ways) so we can codify it
> >> in
> >>>> the
> >>>>>> KIP.
> >>>>>>>> I'll
> >>>>>>>>>> look into that and try to update soon.
> >>>>>>>>>>
> >>>>>>>>>> Thanks,
> >>>>>>>>>> Justine.
> >>>>>>>>>>
> >>>>>>>>>> On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
> >>>>>>>>>> <al...@confluent.io.invalid> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> It's good to know that KIP-588 addressed some of the
> >> issues.
> >>>>>> Looking
> >>>>>>>> at
> >>>>>>>>>>> the code, it still looks like there are some cases that
> >>> would
> >>>>>> result
> >>>>>>>> in
> >>>>>>>>>>> fatal error, e.g. PRODUCER_FENCED is issued by the
> >>> transaction
> >>>>>>>> coordinator
> >>>>>>>>>>> if epoch doesn't match, and the client treats it as a
> >> fatal
> >>>>> error
> >>>>>>>> (code in
> >>>>>>>>>>> TransactionManager request handling).  If we consider, for
> >>>>>> example,
> >>>>>>>>>>> committing a transaction that returns a timeout, but
> >>> actually
> >>>>>>>> succeeds,
> >>>>>>>>>>> trying to abort it or re-commit may result in
> >>> PRODUCER_FENCED
> >>>>>> error
> >>>>>>>>>>> (because of epoch bump).
> >>>>>>>>>>>
> >>>>>>>>>>> For failed commits, specifically, we need to know the
> >> actual
> >>>>>> outcome,
> >>>>>>>>>>> because if we return an error the application may think
> >> that
> >>>> the
> >>>>>>>>>>> transaction is aborted and redo the work, leading to
> >>>> duplicates.
> >>>>>>>>>>>
> >>>>>>>>>>> Re: overflowing epoch.  We could either do it on the TC
> >> and
> >>>>> return
> >>>>>>>> both
> >>>>>>>>>>> producer id and epoch (e.g. change the protocol), or
> >> signal
> >>>> the
> >>>>>> client
> >>>>>>>>>>> that
> >>>>>>>>>>> it needs to get a new producer id.  Checking for max epoch
> >>>> could
> >>>>>> be a
> >>>>>>>>>>> reasonable signal, the value to check should probably be
> >>>> present
> >>>>>> in
> >>>>>>>> the
> >>>>>>>>>>> KIP
> >>>>>>>>>>> as this is effectively a part of the contract.  Also, the
> >> TC
> >>>>>> should
> >>>>>>>>>>> probably return an error if the client didn't change
> >>> producer
> >>>> id
> >>>>>> after
> >>>>>>>>>>> hitting max epoch.
> >>>>>>>>>>>
> >>>>>>>>>>> -Artem
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
> >>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> Thanks for the discussion Artem.
> >>>>>>>>>>>>
> >>>>>>>>>>>> With respect to the handling of fenced producers, we
> >> have
> >>>> some
> >>>>>>>> behavior
> >>>>>>>>>>>> already in place. As of KIP-588:
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> >>>>>>>>>>>> ,
> >>>>>>>>>>>> we handle timeouts more gracefully. The producer can
> >>>> recover.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Produce requests can also recover from epoch fencing by
> >>>>>> aborting the
> >>>>>>>>>>>> transaction and starting over.
> >>>>>>>>>>>>
> >>>>>>>>>>>> What other cases were you considering that would cause
> >> us
> >>> to
> >>>>>> have a
> >>>>>>>>>>> fenced
> >>>>>>>>>>>> epoch but we'd want to recover?
> >>>>>>>>>>>>
> >>>>>>>>>>>> The first point about handling epoch overflows is fair.
> >> I
> >>>>> think
> >>>>>>>> there is
> >>>>>>>>>>>> some logic we'd need to consider. (ie, if we are one
> >> away
> >>>> from
> >>>>>> the
> >>>>>>>> max
> >>>>>>>>>>>> epoch, we need to reset the producer ID.) I'm still
> >>>> wondering
> >>>>> if
> >>>>>>>> there
> >>>>>>>>>>> is a
> >>>>>>>>>>>> way to direct this from the response, or if everything
> >>>> should
> >>>>> be
> >>>>>>>> done on
> >>>>>>>>>>>> the client side. Let me know if you have any thoughts
> >>> here.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thanks,
> >>>>>>>>>>>> Justine
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
> >>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> There are some workflows in the client that are
> >> implied
> >>> by
> >>>>>>>> protocol
> >>>>>>>>>>>>> changes, e.g.:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> - for new clients, epoch changes with every
> >> transaction
> >>>> and
> >>>>>> can
> >>>>>>>>>>> overflow,
> >>>>>>>>>>>>> in old clients this condition was handled
> >> transparently,
> >>>>>> because
> >>>>>>>> epoch
> >>>>>>>>>>>> was
> >>>>>>>>>>>>> bumped in InitProducerId and it would return a new
> >>>> producer
> >>>>>> id if
> >>>>>>>>>>> epoch
> >>>>>>>>>>>>> overflows, the new clients would need to implement
> >> some
> >>>>>> workflow
> >>>>>>>> to
> >>>>>>>>>>>> refresh
> >>>>>>>>>>>>> producer id
> >>>>>>>>>>>>> - how to handle fenced producers, for new clients
> >> epoch
> >>>>>> changes
> >>>>>>>> with
> >>>>>>>>>>>> every
> >>>>>>>>>>>>> transaction, so in presence of failures during
> >> commits /
> >>>>>> aborts,
> >>>>>>>> the
> >>>>>>>>>>>>> producer could get easily fenced, old clients would
> >>> pretty
> >>>>>> much
> >>>>>>>> would
> >>>>>>>>>>> get
> >>>>>>>>>>>>> fenced when a new incarnation of the producer was
> >>>>> initialized
> >>>>>> with
> >>>>>>>>>>>>> InitProducerId so it's ok to treat as a fatal error,
> >> the
> >>>> new
> >>>>>>>> clients
> >>>>>>>>>>>> would
> >>>>>>>>>>>>> need to implement some workflow to handle that error,
> >>>>>> otherwise
> >>>>>>>> they
> >>>>>>>>>>>> could
> >>>>>>>>>>>>> get fenced by themselves
> >>>>>>>>>>>>> - in particular (as a subset of the previous issue),
> >>> what
> >>>>>> would
> >>>>>>>> the
> >>>>>>>>>>>> client
> >>>>>>>>>>>>> do if it got a timeout during commit?  commit could've
> >>>>>> succeeded
> >>>>>>>> or
> >>>>>>>>>>>> failed
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Not sure if this has to be defined in the KIP as
> >>>>> implementing
> >>>>>>>> those
> >>>>>>>>>>>>> probably wouldn't require protocol changes, but we
> >> have
> >>>>>> multiple
> >>>>>>>>>>>>> implementations of Kafka clients, so probably would be
> >>>> good
> >>>>> to
> >>>>>>>> have
> >>>>>>>>>>> some
> >>>>>>>>>>>>> client implementation guidance.  Could also be done
> >> as a
> >>>>>> separate
> >>>>>>>> doc.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> -Artem
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
> >>>>>>>>>>>> <jolshan@confluent.io.invalid
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hey all, I've updated the KIP to incorporate Jason's
> >>>>>>>> suggestions.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 1. Use AddPartitionsToTxn + verify flag to check on
> >>> old
> >>>>>> clients
> >>>>>>>>>>>>>> 2. Updated AddPartitionsToTxn API to support
> >>> transaction
> >>>>>>>> batching
> >>>>>>>>>>>>>> 3. Mention IBP bump
> >>>>>>>>>>>>>> 4. Mention auth change on new AddPartitionsToTxn
> >>>> version.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I'm planning on opening a vote soon.
> >>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <
> >>>>>>>> jolshan@confluent.io
> >>>>>>>>>>>>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thanks Jason. Those changes make sense to me. I
> >> will
> >>>>>> update
> >>>>>>>> the
> >>>>>>>>>>> KIP.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
> >>>>>>>>>>>>>> <ja...@confluent.io.invalid>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Hey Justine,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I was wondering about compatibility here. When
> >> we
> >>>>> send
> >>>>>>>> requests
> >>>>>>>>>>>>>>>> between brokers, we want to ensure that the
> >>> receiving
> >>>>>> broker
> >>>>>>>>>>>>> understands
> >>>>>>>>>>>>>>>> the request (specifically the new fields).
> >>> Typically
> >>>>>> this is
> >>>>>>>> done
> >>>>>>>>>>>> via
> >>>>>>>>>>>>>>>> IBP/metadata version.
> >>>>>>>>>>>>>>>> I'm trying to think if there is a way around it
> >> but
> >>>> I'm
> >>>>>> not
> >>>>>>>> sure
> >>>>>>>>>>>> there
> >>>>>>>>>>>>>> is.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Yes. I think we would gate usage of this behind
> >> an
> >>>> IBP
> >>>>>> bump.
> >>>>>>>> Does
> >>>>>>>>>>>> that
> >>>>>>>>>>>>>>>> seem
> >>>>>>>>>>>>>>>> reasonable?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> As for the improvements -- can you clarify how
> >>> the
> >>>>>> multiple
> >>>>>>>>>>>>>>>> transactional
> >>>>>>>>>>>>>>>> IDs would help here? Were you thinking of a case
> >>>> where
> >>>>> we
> >>>>>>>>>>> wait/batch
> >>>>>>>>>>>>>>>> multiple produce requests together? My
> >>> understanding
> >>>>> for
> >>>>>> now
> >>>>>>>> was
> >>>>>>>>>>> 1
> >>>>>>>>>>>>>>>> transactional ID and one validation per 1 produce
> >>>>>> request.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Each call to `AddPartitionsToTxn` is essentially
> >> a
> >>>>> write
> >>>>>> to
> >>>>>>>> the
> >>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>> log and must block on replication. The more we
> >> can
> >>>> fit
> >>>>>> into a
> >>>>>>>>>>> single
> >>>>>>>>>>>>>>>> request, the more writes we can do in parallel.
> >> The
> >>>>>>>> alternative
> >>>>>>>>>>> is
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>> use of more connections, but usually we prefer
> >>>> batching
> >>>>>>>> since the
> >>>>>>>>>>>>>> network
> >>>>>>>>>>>>>>>> stack is not really optimized for high
> >>>>> connection/request
> >>>>>>>> loads.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Finally with respect to the authorizations, I
> >>> think
> >>>>> it
> >>>>>>>> makes
> >>>>>>>>>>> sense
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> skip
> >>>>>>>>>>>>>>>> topic authorizations, but I'm a bit confused by
> >> the
> >>>>>> "leader
> >>>>>>>> ID"
> >>>>>>>>>>>> field.
> >>>>>>>>>>>>>>>> Wouldn't we just want to flag the request as
> >> from a
> >>>>>> broker
> >>>>>>>> (does
> >>>>>>>>>>> it
> >>>>>>>>>>>>>> matter
> >>>>>>>>>>>>>>>> which one?).
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> We could also make it version-based. For the next
> >>>>>> version, we
> >>>>>>>>>>> could
> >>>>>>>>>>>>>>>> require
> >>>>>>>>>>>>>>>> CLUSTER auth. So clients would not be able to use
> >>> the
> >>>>> API
> >>>>>>>>>>> anymore,
> >>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>> probably what we want.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> -Jason
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> >>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> As a follow up, I was just thinking about the
> >>>>> batching
> >>>>>> a
> >>>>>>>> bit
> >>>>>>>>>>> more.
> >>>>>>>>>>>>>>>>> I suppose if we have one request in flight and
> >> we
> >>>>>> queue up
> >>>>>>>> the
> >>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>> produce requests in some sort of purgatory, we
> >>>> could
> >>>>>> send
> >>>>>>>>>>>>> information
> >>>>>>>>>>>>>>>> out
> >>>>>>>>>>>>>>>>> for all of them rather than one by one. So that
> >>>> would
> >>>>>> be a
> >>>>>>>>>>> benefit
> >>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>> batching partitions to add per transaction.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I'll need to think a bit more on the design of
> >>> this
> >>>>>> part
> >>>>>>>> of the
> >>>>>>>>>>>> KIP,
> >>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>> will update the KIP in the next few days.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan
> >> <
> >>>>>>>>>>>>> jolshan@confluent.io>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Hey Jason -- thanks for the input -- I was
> >> just
> >>>>>> digging
> >>>>>>>> a bit
> >>>>>>>>>>>>> deeper
> >>>>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>>> the design + implementation of the validation
> >>>> calls
> >>>>>> here
> >>>>>>>> and
> >>>>>>>>>>>> what
> >>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>> say
> >>>>>>>>>>>>>>>>>> makes sense.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I was wondering about compatibility here.
> >> When
> >>> we
> >>>>>> send
> >>>>>>>>>>> requests
> >>>>>>>>>>>>>>>>>> between brokers, we want to ensure that the
> >>>>> receiving
> >>>>>>>> broker
> >>>>>>>>>>>>>>>> understands
> >>>>>>>>>>>>>>>>>> the request (specifically the new fields).
> >>>>> Typically
> >>>>>>>> this is
> >>>>>>>>>>>> done
> >>>>>>>>>>>>>> via
> >>>>>>>>>>>>>>>>>> IBP/metadata version.
> >>>>>>>>>>>>>>>>>> I'm trying to think if there is a way around
> >> it
> >>>> but
> >>>>>> I'm
> >>>>>>>> not
> >>>>>>>>>>> sure
> >>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>> is.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> As for the improvements -- can you clarify
> >> how
> >>>> the
> >>>>>>>> multiple
> >>>>>>>>>>>>>>>> transactional
> >>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking of a
> >>> case
> >>>>>> where we
> >>>>>>>>>>>>> wait/batch
> >>>>>>>>>>>>>>>>>> multiple produce requests together? My
> >>>>> understanding
> >>>>>> for
> >>>>>>>> now
> >>>>>>>>>>>> was 1
> >>>>>>>>>>>>>>>>>> transactional ID and one validation per 1
> >>> produce
> >>>>>>>> request.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Finally with respect to the authorizations, I
> >>>> think
> >>>>>> it
> >>>>>>>> makes
> >>>>>>>>>>>> sense
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> skip
> >>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit confused
> >> by
> >>>> the
> >>>>>>>> "leader
> >>>>>>>>>>> ID"
> >>>>>>>>>>>>>> field.
> >>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the request as
> >>>> from a
> >>>>>>>> broker
> >>>>>>>>>>> (does
> >>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>> matter
> >>>>>>>>>>>>>>>>>> which one?).
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I think I want to adopt these suggestions,
> >> just
> >>>> had
> >>>>>> a few
> >>>>>>>>>>>>> questions
> >>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> details.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Thu, Jan 5, 2023 at 5:05 PM Jason
> >> Gustafson
> >>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Hi Justine,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thanks for the proposal.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> I was thinking about the implementation a
> >>> little
> >>>>>> bit.
> >>>>>>>> In the
> >>>>>>>>>>>>>> current
> >>>>>>>>>>>>>>>>>>> proposal, the behavior depends on whether we
> >>>> have
> >>>>> an
> >>>>>>>> old or
> >>>>>>>>>>> new
> >>>>>>>>>>>>>>>> client.
> >>>>>>>>>>>>>>>>>>> For
> >>>>>>>>>>>>>>>>>>> old clients, we send `DescribeTransactions`
> >>> and
> >>>>>> verify
> >>>>>>>> the
> >>>>>>>>>>>> result
> >>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>> new clients, we send `AddPartitionsToTxn`.
> >> We
> >>>>> might
> >>>>>> be
> >>>>>>>> able
> >>>>>>>>>>> to
> >>>>>>>>>>>>>>>> simplify
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> implementation if we can use the same
> >> request
> >>>>> type.
> >>>>>> For
> >>>>>>>>>>>> example,
> >>>>>>>>>>>>>>>> what if
> >>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>> bump the protocol version for
> >>>> `AddPartitionsToTxn`
> >>>>>> and
> >>>>>>>> add a
> >>>>>>>>>>>>>>>>>>> `validateOnly`
> >>>>>>>>>>>>>>>>>>> flag? For older versions, we can set
> >>>>>>>> `validateOnly=true` so
> >>>>>>>>>>>> that
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> request only returns successfully if the
> >>>> partition
> >>>>>> had
> >>>>>>>>>>> already
> >>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>> added.
> >>>>>>>>>>>>>>>>>>> For new versions, we can set
> >>>> `validateOnly=false`
> >>>>>> and
> >>>>>>>> the
> >>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>> added to the transaction. The other slightly
> >>>>>> annoying
> >>>>>>>> thing
> >>>>>>>>>>>> that
> >>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>> get around is the need to collect the
> >>>> transaction
> >>>>>> state
> >>>>>>>> for
> >>>>>>>>>>> all
> >>>>>>>>>>>>>>>>> partitions
> >>>>>>>>>>>>>>>>>>> even when we only care about a subset.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Some additional improvements to consider:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> - We can give `AddPartitionsToTxn` better
> >>> batch
> >>>>>> support
> >>>>>>>> for
> >>>>>>>>>>>>>>>> inter-broker
> >>>>>>>>>>>>>>>>>>> usage. Currently we only allow one
> >>>>>> `TransactionalId` to
> >>>>>>>> be
> >>>>>>>>>>>>>> specified,
> >>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>> the broker may get some benefit being able
> >> to
> >>>>> batch
> >>>>>>>> across
> >>>>>>>>>>>>> multiple
> >>>>>>>>>>>>>>>>>>> transactions.
> >>>>>>>>>>>>>>>>>>> - Another small improvement is skipping
> >> topic
> >>>>>>>> authorization
> >>>>>>>>>>>>> checks
> >>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>> `AddPartitionsToTxn` when the request is
> >> from
> >>> a
> >>>>>> broker.
> >>>>>>>>>>> Perhaps
> >>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>> add
> >>>>>>>>>>>>>>>>>>> a field for the `LeaderId` or something like
> >>>> that
> >>>>>> and
> >>>>>>>>>>> require
> >>>>>>>>>>>>>> CLUSTER
> >>>>>>>>>>>>>>>>>>> permission when set.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>> Jason
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
> >>>>>>>>>>>> <jun@confluent.io.invalid
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Hi, Justine,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Thanks for the explanation. It makes sense
> >>> to
> >>>> me
> >>>>>> now.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Jun
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 1:42 PM Justine
> >>> Olshan
> >>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Hi Jun,
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> My understanding of the mechanism is
> >> that
> >>>> when
> >>>>>> we
> >>>>>>>> get to
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>> last
> >>>>>>>>>>>>>>>>>>> epoch,
> >>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>> increment to the fencing/last epoch and
> >> if
> >>>> any
> >>>>>>>> further
> >>>>>>>>>>>>> requests
> >>>>>>>>>>>>>>>> come
> >>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>> this producer ID they are fenced. Then
> >> the
> >>>>>> producer
> >>>>>>>>>>> gets a
> >>>>>>>>>>>>> new
> >>>>>>>>>>>>>> ID
> >>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>> restarts with epoch/sequence 0. The
> >> fenced
> >>>>> epoch
> >>>>>>>> sticks
> >>>>>>>>>>>>> around
> >>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> duration of producer.id.expiration.ms
> >> and
> >>>>>> blocks
> >>>>>>>> any
> >>>>>>>>>>> late
> >>>>>>>>>>>>>>>> messages
> >>>>>>>>>>>>>>>>>>>> there.
> >>>>>>>>>>>>>>>>>>>>> The new ID will get to take advantage of
> >>> the
> >>>>>>>> improved
> >>>>>>>>>>>>> semantics
> >>>>>>>>>>>>>>>>> around
> >>>>>>>>>>>>>>>>>>>>> non-zero start sequences. So I think we
> >>> are
> >>>>>> covered.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> The only potential issue is overloading
> >>> the
> >>>>>> cache,
> >>>>>>>> but
> >>>>>>>>>>>>>> hopefully
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> improvements (lowered
> >>>>> producer.id.expiration.ms
> >>>>>> )
> >>>>>>>> will
> >>>>>>>>>>> help
> >>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>> that.
> >>>>>>>>>>>>>>>>>>>> Let
> >>>>>>>>>>>>>>>>>>>>> me know if you still have concerns.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
> >>>>>>>>>>>>>>>> <ju...@confluent.io.invalid>
> >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 70. The proposed fencing logic doesn't
> >>>> apply
> >>>>>> when
> >>>>>>>> pid
> >>>>>>>>>>>>>> changes,
> >>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>> right? If so, I am not sure how
> >> complete
> >>>> we
> >>>>>> are
> >>>>>>>>>>>> addressing
> >>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>> issue
> >>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>> the pid changes more frequently.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Jun
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 16, 2022 at 9:16 AM
> >> Justine
> >>>>> Olshan
> >>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Thanks for replying!
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 70.We already do the overflow
> >>> mechanism,
> >>>>> so
> >>>>>> my
> >>>>>>>>>>> change
> >>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>> happen more often.
> >>>>>>>>>>>>>>>>>>>>>>> I was also not suggesting a new
> >> field
> >>> in
> >>>>> the
> >>>>>>>> log,
> >>>>>>>>>>> but
> >>>>>>>>>>>> in
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> response,
> >>>>>>>>>>>>>>>>>>>>>>> which would be gated by the client
> >>>>> version.
> >>>>>>>> Sorry if
> >>>>>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>> unclear. I think we are starting to
> >>>>> diverge.
> >>>>>>>>>>>>>>>>>>>>>>> The goal of this KIP is to not
> >> change
> >>> to
> >>>>> the
> >>>>>>>> marker
> >>>>>>>>>>>>> format
> >>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>> all.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 71. Yes, I guess I was going under
> >> the
> >>>>>>>> assumption
> >>>>>>>>>>> that
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> log
> >>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>> look at its last epoch and treat it
> >> as
> >>>> the
> >>>>>>>> current
> >>>>>>>>>>>>> epoch. I
> >>>>>>>>>>>>>>>>>>> suppose
> >>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>> have some special logic that if the
> >>> last
> >>>>>> epoch
> >>>>>>>> was
> >>>>>>>>>>> on a
> >>>>>>>>>>>>>>>> marker
> >>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>>>>> expect the next epoch or something
> >>> like
> >>>>>> that. We
> >>>>>>>>>>> just
> >>>>>>>>>>>>> need
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> distinguish
> >>>>>>>>>>>>>>>>>>>>>>> based on whether we had a
> >> commit/abort
> >>>>>> marker.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 72.
> >>>>>>>>>>>>>>>>>>>>>>>> if the producer epoch hasn't been
> >>>> bumped
> >>>>>> on
> >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
> >>>> message
> >>>>>> will
> >>>>>>>> fail
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>> sequence
> >>>>>>>>>>>>>>>>>>>>>> validation
> >>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the producer
> >>>> epoch
> >>>>>> has
> >>>>>>>> been
> >>>>>>>>>>>>> bumped,
> >>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>> ignore
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck message
> >>>> could
> >>>>>> be
> >>>>>>>>>>> appended
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> log.
> >>>>>>>>>>>>>>>>>>>> So,
> >>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
> >> guard?
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> I'm not sure I follow that "the
> >>> message
> >>>>> will
> >>>>>>>> fail
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>> sequence
> >>>>>>>>>>>>>>>>>>>>>> validation".
> >>>>>>>>>>>>>>>>>>>>>>> In some of these cases, we had an
> >>> abort
> >>>>>> marker
> >>>>>>>> (due
> >>>>>>>>>>> to
> >>>>>>>>>>>> an
> >>>>>>>>>>>>>>>> error)
> >>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>> then
> >>>>>>>>>>>>>>>>>>>>>>> the late message comes in with the
> >>>> correct
> >>>>>>>> sequence
> >>>>>>>>>>>>> number.
> >>>>>>>>>>>>>>>> This
> >>>>>>>>>>>>>>>>>>> is a
> >>>>>>>>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>>>> covered by the KIP.
> >>>>>>>>>>>>>>>>>>>>>>> The latter case is actually not
> >>>> something
> >>>>>> we've
> >>>>>>>>>>>>> considered
> >>>>>>>>>>>>>>>>> here. I
> >>>>>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>> generally when we bump the epoch, we
> >>> are
> >>>>>>>> accepting
> >>>>>>>>>>> that
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> sequence
> >>>>>>>>>>>>>>>>>>>>> does
> >>>>>>>>>>>>>>>>>>>>>>> not need to be checked anymore. My
> >>>>>>>> understanding is
> >>>>>>>>>>>> also
> >>>>>>>>>>>>>>>> that we
> >>>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>> typically bump epoch mid transaction
> >>>>> (based
> >>>>>> on a
> >>>>>>>>>>> quick
> >>>>>>>>>>>>> look
> >>>>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> code)
> >>>>>>>>>>>>>>>>>>>>>>> but let me know if that is the case.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 15, 2022 at 12:23 PM Jun
> >>> Rao
> >>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the reply.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 70. Assigning a new pid on int
> >>>> overflow
> >>>>>> seems
> >>>>>>>> a
> >>>>>>>>>>> bit
> >>>>>>>>>>>>>> hacky.
> >>>>>>>>>>>>>>>> If
> >>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>> need a
> >>>>>>>>>>>>>>>>>>>>>>> txn
> >>>>>>>>>>>>>>>>>>>>>>>> level id, it will be better to
> >> model
> >>>>> this
> >>>>>>>>>>> explicitly.
> >>>>>>>>>>>>>>>> Adding a
> >>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>> field
> >>>>>>>>>>>>>>>>>>>>>>>> would require a bit more work
> >> since
> >>> it
> >>>>>>>> requires a
> >>>>>>>>>>> new
> >>>>>>>>>>>>> txn
> >>>>>>>>>>>>>>>>> marker
> >>>>>>>>>>>>>>>>>>>>> format
> >>>>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>> the log. So, we probably need to
> >>> guard
> >>>>> it
> >>>>>>>> with an
> >>>>>>>>>>> IBP
> >>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>> metadata
> >>>>>>>>>>>>>>>>>>>>>> version
> >>>>>>>>>>>>>>>>>>>>>>>> and document the impact on
> >> downgrade
> >>>>> once
> >>>>>> the
> >>>>>>>> new
> >>>>>>>>>>>>> format
> >>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>> written
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> log.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 71. Hmm, once the marker is
> >> written,
> >>>> the
> >>>>>>>> partition
> >>>>>>>>>>>> will
> >>>>>>>>>>>>>>>> expect
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> next
> >>>>>>>>>>>>>>>>>>>>>>>> append to be on the next epoch.
> >> Does
> >>>>> that
> >>>>>>>> cover
> >>>>>>>>>>> the
> >>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>> mentioned?
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 72. Also, just to be clear on the
> >>>>> stucked
> >>>>>>>> message
> >>>>>>>>>>>> issue
> >>>>>>>>>>>>>>>>>>> described
> >>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> motivation. With EoS, we also
> >>> validate
> >>>>> the
> >>>>>>>>>>> sequence
> >>>>>>>>>>>> id
> >>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>> idempotency.
> >>>>>>>>>>>>>>>>>>>>>>> So,
> >>>>>>>>>>>>>>>>>>>>>>>> with the current logic, if the
> >>>> producer
> >>>>>> epoch
> >>>>>>>>>>> hasn't
> >>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>>> bumped on
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
> >>>>> message
> >>>>>> will
> >>>>>>>>>>> fail
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> sequence
> >>>>>>>>>>>>>>>>>>>>>>> validation
> >>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the
> >> producer
> >>>>>> epoch has
> >>>>>>>>>>> been
> >>>>>>>>>>>>>>>> bumped, we
> >>>>>>>>>>>>>>>>>>>> ignore
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck
> >> message
> >>>>>> could be
> >>>>>>>>>>>> appended
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> log.
> >>>>>>>>>>>>>>>>>>>>> So,
> >>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
> >>> guard?
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Jun
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 10:44 AM
> >>>> Justine
> >>>>>>>> Olshan
> >>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
> >>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Matthias — thanks again for
> >> taking
> >>>>> time
> >>>>>> to
> >>>>>>>> look
> >>>>>>>>>>> a
> >>>>>>>>>>>>> this.
> >>>>>>>>>>>>>>>> You
> >>>>>>>>>>>>>>>>>>> said:
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> My proposal was only focusing
> >> to
> >>>>> avoid
> >>>>>>>>>>> dangling
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
> >> added
> >>>>>> without
> >>>>>>>>>>>> registered
> >>>>>>>>>>>>>>>>>>> partition.
> >>>>>>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>>>>>>>> Maybe
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more details
> >> to
> >>>> the
> >>>>>> KIP
> >>>>>>>> about
> >>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>> scenario
> >>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>> better
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I understand what
> >> you
> >>>>> mean
> >>>>>>>> here.
> >>>>>>>>>>> The
> >>>>>>>>>>>>>>>> motivation
> >>>>>>>>>>>>>>>>>>>>> section
> >>>>>>>>>>>>>>>>>>>>>>>>> describes two scenarios about
> >> how
> >>>> the
> >>>>>> record
> >>>>>>>>>>> can be
> >>>>>>>>>>>>>> added
> >>>>>>>>>>>>>>>>>>>> without a
> >>>>>>>>>>>>>>>>>>>>>>>>> registered partition:
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Another way hanging
> >> transactions
> >>>> can
> >>>>>>>> occur is
> >>>>>>>>>>>> that
> >>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>> client
> >>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>> buggy
> >>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>> may somehow try to write to a
> >>>>> partition
> >>>>>>>> before
> >>>>>>>>>>> it
> >>>>>>>>>>>>> adds
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> transaction.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> For the first example of this
> >>> would
> >>>> it
> >>>>>> be
> >>>>>>>>>>> helpful
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>> say
> >>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>> message
> >>>>>>>>>>>>>>>>>>>>>>>>> comes in after the abort, but
> >>> before
> >>>>> the
> >>>>>>>>>>> partition
> >>>>>>>>>>>> is
> >>>>>>>>>>>>>>>> added
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> next
> >>>>>>>>>>>>>>>>>>>>>>>>> transaction so it becomes
> >>> "hanging."
> >>>>>>>> Perhaps the
> >>>>>>>>>>>> next
> >>>>>>>>>>>>>>>>> sentence
> >>>>>>>>>>>>>>>>>>>>>>> describing
> >>>>>>>>>>>>>>>>>>>>>>>>> the message becoming part of the
> >>>> next
> >>>>>>>>>>> transaction
> >>>>>>>>>>>> (a
> >>>>>>>>>>>>>>>>> different
> >>>>>>>>>>>>>>>>>>>>> case)
> >>>>>>>>>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>>>>>>> not properly differentiated.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Jun — thanks for reading the
> >> KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> 70. The int typing was a
> >> concern.
> >>>>>> Currently
> >>>>>>>> we
> >>>>>>>>>>>> have a
> >>>>>>>>>>>>>>>>>>> mechanism
> >>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>> place
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>> fence the final epoch when the
> >>> epoch
> >>>>> is
> >>>>>>>> about to
> >>>>>>>>>>>>>> overflow
> >>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>> assign
> >>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>>>> producer ID with epoch 0. Of
> >>> course,
> >>>>>> this
> >>>>>>>> is a
> >>>>>>>>>>> bit
> >>>>>>>>>>>>>> tricky
> >>>>>>>>>>>>>>>>>>> when it
> >>>>>>>>>>>>>>>>>>>>>> comes
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>> the response back to the client.
> >>>>>>>>>>>>>>>>>>>>>>>>> Making this a long could be
> >>> another
> >>>>>> option,
> >>>>>>>> but
> >>>>>>>>>>> I
> >>>>>>>>>>>>>> wonder
> >>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>>>>> implications on changing this
> >>> field
> >>>> if
> >>>>>> the
> >>>>>>>>>>> epoch is
> >>>>>>>>>>>>>>>>> persisted
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>> disk?
> >>>>>>>>>>>>>>>>>>>>>>>> I'd
> >>>>>>>>>>>>>>>>>>>>>>>>> need to check the usages.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> 71.This was something Matthias
> >>> asked
> >>>>>> about
> >>>>>>>> as
> >>>>>>>>>>>> well. I
> >>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>>>> considering a
> >>>>>>>>>>>>>>>>>>>>>>>>> possible edge case where a
> >> produce
> >>>>>> request
> >>>>>>>> from
> >>>>>>>>>>> a
> >>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>> somehow
> >>>>>>>>>>>>>>>>>>>>>>>>> gets sent right after the marker
> >>> is
> >>>>>>>> written, but
> >>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>> alerted of the newly bumped
> >> epoch.
> >>>> In
> >>>>>> this
> >>>>>>>>>>> case, we
> >>>>>>>>>>>>> may
> >>>>>>>>>>>>>>>>>>> include
> >>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>> record
> >>>>>>>>>>>>>>>>>>>>>>>>> when we don't want to. I suppose
> >>> we
> >>>>>> could
> >>>>>>>> try
> >>>>>>>>>>> to do
> >>>>>>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>>>> client
> >>>>>>>>>>>>>>>>>>>>>>> side
> >>>>>>>>>>>>>>>>>>>>>>>>> to bump the epoch after sending
> >> an
> >>>>>> endTxn as
> >>>>>>>>>>> well
> >>>>>>>>>>>> in
> >>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>> scenario
> >>>>>>>>>>>>>>>>>>>>> —
> >>>>>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>> wonder how it would work when
> >> the
> >>>>>> server is
> >>>>>>>>>>>> aborting
> >>>>>>>>>>>>>>>> based
> >>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>> server-side
> >>>>>>>>>>>>>>>>>>>>>>>>> error. I could also be missing
> >>>>>> something and
> >>>>>>>>>>> this
> >>>>>>>>>>>>>>>> scenario
> >>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>> possible.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Thanks again to everyone reading
> >>> and
> >>>>>>>> commenting.
> >>>>>>>>>>>> Let
> >>>>>>>>>>>>> me
> >>>>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>>>>> further questions or comments.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 9:41 AM
> >>> Jun
> >>>>> Rao
> >>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP. A couple
> >> of
> >>>>>> comments.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> 70. Currently, the producer
> >>> epoch
> >>>> is
> >>>>>> an
> >>>>>>>> int.
> >>>>>>>>>>> I am
> >>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>> sure
> >>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>> enough
> >>>>>>>>>>>>>>>>>>>>>>>>>> to accommodate all
> >> transactions
> >>> in
> >>>>> the
> >>>>>>>>>>> lifetime
> >>>>>>>>>>>> of
> >>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>> producer.
> >>>>>>>>>>>>>>>>>>>>>> Should
> >>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>> change that to a long or add a
> >>> new
> >>>>>> long
> >>>>>>>> field
> >>>>>>>>>>>> like
> >>>>>>>>>>>>>>>> txnId?
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> 71. "it will write the prepare
> >>>>> commit
> >>>>>>>> message
> >>>>>>>>>>>> with
> >>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>> bumped
> >>>>>>>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>> send
> >>>>>>>>>>>>>>>>>>>>>>>>>> WriteTxnMarkerRequests with
> >> the
> >>>>> bumped
> >>>>>>>> epoch."
> >>>>>>>>>>>> Hmm,
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>> associated
> >>>>>>>>>>>>>>>>>>>>>>>>>> with the current txn right?
> >> So,
> >>> it
> >>>>>> seems
> >>>>>>>>>>> weird to
> >>>>>>>>>>>>>>>> write a
> >>>>>>>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>>>>>>> message
> >>>>>>>>>>>>>>>>>>>>>>>>>> with a bumped epoch. Should we
> >>>> only
> >>>>>> bump
> >>>>>>>> up
> >>>>>>>>>>> the
> >>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>> EndTxnResponse
> >>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>> rename the field to sth like
> >>>>>>>>>>> nextProducerEpoch?
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Jun
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 12, 2022 at 8:54
> >> PM
> >>>>>> Matthias
> >>>>>>>> J.
> >>>>>>>>>>> Sax <
> >>>>>>>>>>>>>>>>>>>>> mjsax@apache.org>
> >>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the background.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30: SGTM. My proposal was
> >>>> only
> >>>>>>>> focusing
> >>>>>>>>>>> to
> >>>>>>>>>>>>> avoid
> >>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
> >>>> added
> >>>>>>>> without
> >>>>>>>>>>>>>> registered
> >>>>>>>>>>>>>>>>>>>> partition.
> >>>>>>>>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>>>>>>>>>> Maybe
> >>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more
> >> details
> >>>> to
> >>>>>> the
> >>>>>>>> KIP
> >>>>>>>>>>> about
> >>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>> scenario
> >>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>> better
> >>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> 40: I think you hit a fair
> >>> point
> >>>>>> about
> >>>>>>>> race
> >>>>>>>>>>>>>>>> conditions
> >>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>> client
> >>>>>>>>>>>>>>>>>>>>>>> bugs
> >>>>>>>>>>>>>>>>>>>>>>>>>>> (incorrectly not bumping the
> >>>>>> epoch). The
> >>>>>>>>>>>>>>>>>>> complexity/confusion
> >>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>> using
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the bumped epoch I see, is
> >>>> mainly
> >>>>>> for
> >>>>>>>>>>> internal
> >>>>>>>>>>>>>>>>> debugging,
> >>>>>>>>>>>>>>>>>>> ie,
> >>>>>>>>>>>>>>>>>>>>>>>>> inspecting
> >>>>>>>>>>>>>>>>>>>>>>>>>>> log segment dumps -- it
> >> seems
> >>>>>> harder to
> >>>>>>>>>>> reason
> >>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> system
> >>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>> us
> >>>>>>>>>>>>>>>>>>>>>>>>>>> humans. But if we get better
> >>>>>>>> guarantees, it
> >>>>>>>>>>>> would
> >>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>> worth to
> >>>>>>>>>>>>>>>>>>>>> use
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> bumped epoch.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> 60: as I mentioned already,
> >> I
> >>>>> don't
> >>>>>>>> know the
> >>>>>>>>>>>>> broker
> >>>>>>>>>>>>>>>>>>> internals
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>> provide
> >>>>>>>>>>>>>>>>>>>>>>>>>>> more input. So if nobody
> >> else
> >>>>> chimes
> >>>>>>>> in, we
> >>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>> move
> >>>>>>>>>>>>>>>>>>>>>>> forward
> >>>>>>>>>>>>>>>>>>>>>>>>>>> with your proposal.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> On 12/6/22 4:22 PM, Justine
> >>>> Olshan
> >>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi all,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> After Artem's questions
> >>> about
> >>>>>> error
> >>>>>>>>>>> behavior,
> >>>>>>>>>>>>>> I've
> >>>>>>>>>>>>>>>>>>>>> re-evaluated
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> unknown producer ID
> >>> exception
> >>>>> and
> >>>>>> had
> >>>>>>>> some
> >>>>>>>>>>>>>>>> discussions
> >>>>>>>>>>>>>>>>>>>>> offline.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> I think generally it makes
> >>>> sense
> >>>>>> to
> >>>>>>>>>>> simplify
> >>>>>>>>>>>>>> error
> >>>>>>>>>>>>>>>>>>> handling
> >>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>> cases
> >>>>>>>>>>>>>>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> this and the
> >>>> UNKNOWN_PRODUCER_ID
> >>>>>> error
> >>>>>>>>>>> has a
> >>>>>>>>>>>>>> pretty
> >>>>>>>>>>>>>>>>> long
> >>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>> complicated
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> history. Because of this,
> >> I
> >>>>>> propose
> >>>>>>>>>>> adding a
> >>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>> error
> >>>>>>>>>>>>>>>>>>> code
> >>>>>>>>>>>>>>>>>>>>>>>>>>> ABORTABLE_ERROR
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> that when encountered by
> >> new
> >>>>>> clients
> >>>>>>>>>>> (gated
> >>>>>>>>>>>> by
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> produce
> >>>>>>>>>>>>>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>>>>>>>>>>>> version)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> will simply abort the
> >>>>> transaction.
> >>>>>>>> This
> >>>>>>>>>>>> allows
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> server
> >>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>>>>>>> say
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> in whether the client
> >> aborts
> >>>> and
> >>>>>> makes
> >>>>>>>>>>>> handling
> >>>>>>>>>>>>>>>> much
> >>>>>>>>>>>>>>>>>>>> simpler.
> >>>>>>>>>>>>>>>>>>>>>> In
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> future, we can also use
> >> this
> >>>>>> error in
> >>>>>>>>>>> other
> >>>>>>>>>>>>>>>> situations
> >>>>>>>>>>>>>>>>>>>> where
> >>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>> want
> >>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> abort the transactions. We
> >>> can
> >>>>>> even
> >>>>>>>> use on
> >>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>> apis.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> I've added this to the
> >> KIP.
> >>>> Let
> >>>>> me
> >>>>>>>> know if
> >>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>>> questions
> >>>>>>>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> issues.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 2, 2022 at
> >> 10:22
> >>>> AM
> >>>>>>>> Justine
> >>>>>>>>>>>> Olshan
> >>>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>>> jolshan@confluent.io
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey Matthias,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30 — Maybe I also
> >> didn't
> >>>>>> express
> >>>>>>>>>>> myself
> >>>>>>>>>>>>>>>> clearly.
> >>>>>>>>>>>>>>>>> For
> >>>>>>>>>>>>>>>>>>>>> older
> >>>>>>>>>>>>>>>>>>>>>>>>> clients
> >>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't have a way to
> >>>> distinguish
> >>>>>>>> between a
> >>>>>>>>>>>>>> previous
> >>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> current
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction since we
> >> don't
> >>>> have
> >>>>>> the
> >>>>>>>> epoch
> >>>>>>>>>>>>> bump.
> >>>>>>>>>>>>>>>> This
> >>>>>>>>>>>>>>>>>>> means
> >>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>> late
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> message from the previous
> >>>>>> transaction
> >>>>>>>>>>> may be
> >>>>>>>>>>>>>>>> added to
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>> one.
> >>>>>>>>>>>>>>>>>>>>>>>>>> With
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> older clients — we can't
> >>>>>> guarantee
> >>>>>>>> this
> >>>>>>>>>>>> won't
> >>>>>>>>>>>>>>>> happen
> >>>>>>>>>>>>>>>>>>> if we
> >>>>>>>>>>>>>>>>>>>>>>> already
> >>>>>>>>>>>>>>>>>>>>>>>>>> sent
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call
> >>> (why
> >>>> we
> >>>>>> make
> >>>>>>>>>>> changes
> >>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> newer
> >>>>>>>>>>>>>>>>>>>>>>> client)
> >>>>>>>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> can at least gate some by
> >>>>>> ensuring
> >>>>>>>> that
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>>> has
> >>>>>>>>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>>>>>>>> added
> >>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. The
> >> rationale
> >>>> here
> >>>>>> is
> >>>>>>>> that
> >>>>>>>>>>>> there
> >>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>> likely
> >>>>>>>>>>>>>>>>>>>>> LESS
> >>>>>>>>>>>>>>>>>>>>>>> late
> >>>>>>>>>>>>>>>>>>>>>>>>>>> arrivals
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> as time goes on, so
> >>> hopefully
> >>>>>> most
> >>>>>>>> late
> >>>>>>>>>>>>> arrivals
> >>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>> come
> >>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>> BEFORE
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call.
> >>>> Those
> >>>>>> that
> >>>>>>>>>>> arrive
> >>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>> properly
> >>>>>>>>>>>>>>>>>>>>>>>>>> gated
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> >>> describeTransactions
> >>>>>>>> approach.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we take the approach
> >> you
> >>>>>>>> suggested,
> >>>>>>>>>>> ANY
> >>>>>>>>>>>>> late
> >>>>>>>>>>>>>>>>> arrival
> >>>>>>>>>>>>>>>>>>>>> from a
> >>>>>>>>>>>>>>>>>>>>>>>>>> previous
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction will be
> >> added.
> >>>> And
> >>>>> we
> >>>>>>>> don't
> >>>>>>>>>>> want
> >>>>>>>>>>>>>>>> that. I
> >>>>>>>>>>>>>>>>>>> also
> >>>>>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>>> see
> >>>>>>>>>>>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> benefit in sending
> >>>>>> addPartitionsToTxn
> >>>>>>>>>>> over
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> describeTxns
> >>>>>>>>>>>>>>>>>>>>>>> call.
> >>>>>>>>>>>>>>>>>>>>>>>>> They
> >>>>>>>>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> both be one extra RPC to
> >>> the
> >>>>> Txn
> >>>>>>>>>>>> coordinator.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> To be clear — newer
> >> clients
> >>>>> will
> >>>>>> use
> >>>>>>>>>>>>>>>>> addPartitionsToTxn
> >>>>>>>>>>>>>>>>>>>>>> instead
> >>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTxns.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that if we
> >>> have
> >>>>>> some
> >>>>>>>> delay
> >>>>>>>>>>> in
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> client
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> bump
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> epoch,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> it could continue to send
> >>>> epoch
> >>>>>> 73
> >>>>>>>> and
> >>>>>>>>>>> those
> >>>>>>>>>>>>>>>> records
> >>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>> fenced.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Perhaps this is not an
> >>> issue
> >>>> if
> >>>>>> we
> >>>>>>>> don't
> >>>>>>>>>>>> allow
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> next
> >>>>>>>>>>>>>>>>>>>>>> produce
> >>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>> go
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> through before the EndTxn
> >>>>> request
> >>>>>>>>>>> returns.
> >>>>>>>>>>>> I'm
> >>>>>>>>>>>>>>>> also
> >>>>>>>>>>>>>>>>>>>> thinking
> >>>>>>>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>>>>>>>>>> cases of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure. I will need to
> >>> think
> >>>>> on
> >>>>>>>> this a
> >>>>>>>>>>> bit.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I wasn't sure if it was
> >>> that
> >>>>>>>> confusing.
> >>>>>>>>>>> But
> >>>>>>>>>>>> if
> >>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>> think it
> >>>>>>>>>>>>>>>>>>>>> is,
> >>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> investigate other ways.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure these are
> >> the
> >>>> same
> >>>>>>>>>>> purgatories
> >>>>>>>>>>>>>> since
> >>>>>>>>>>>>>>>> one
> >>>>>>>>>>>>>>>>>>> is a
> >>>>>>>>>>>>>>>>>>>>>>> produce
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory (I was planning
> >>> on
> >>>>>> using a
> >>>>>>>>>>>> callback
> >>>>>>>>>>>>>>>> rather
> >>>>>>>>>>>>>>>>>>> than
> >>>>>>>>>>>>>>>>>>>>>>>> purgatory)
> >>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the other is simply a
> >>> request
> >>>>> to
> >>>>>>>> append
> >>>>>>>>>>> to
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> log.
> >>>>>>>>>>>>>>>>> Not
> >>>>>>>>>>>>>>>>>>>> sure
> >>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> structure here for
> >>> ordering,
> >>>>> but
> >>>>>> my
> >>>>>>>>>>>>>> understanding
> >>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> broker
> >>>>>>>>>>>>>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> handle the write request
> >>>> before
> >>>>>> it
> >>>>>>>> hears
> >>>>>>>>>>>> back
> >>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> Txn
> >>>>>>>>>>>>>>>>>>>>>>>>>> Coordinator.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if I
> >>>> misunderstood
> >>>>>>>> something
> >>>>>>>>>>> or
> >>>>>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>>>>> unclear.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 1, 2022 at
> >>> 12:15
> >>>> PM
> >>>>>>>> Matthias
> >>>>>>>>>>> J.
> >>>>>>>>>>>>> Sax
> >>>>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the details
> >>>>> Justine!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side change
> >>> for
> >>>> 2
> >>>>> is
> >>>>>>>>>>> removing
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> addPartitions
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need to
> >>> make
> >>>>>> this
> >>>>>>>> from
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> txn
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think I did not
> >> express
> >>>>> myself
> >>>>>>>>>>> clearly. I
> >>>>>>>>>>>>>>>>> understand
> >>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>> (and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should) change the
> >>> producer
> >>>> to
> >>>>>> not
> >>>>>>>> send
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> `addPartitions`
> >>>>>>>>>>>>>>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> longer. But I don't
> >> thinks
> >>>>> it's
> >>>>>>>>>>> requirement
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> change
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> broker?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What I am trying to say
> >>> is:
> >>>>> as a
> >>>>>>>>>>> safe-guard
> >>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>> improvement
> >>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>> older
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producers, the partition
> >>>>> leader
> >>>>>> can
> >>>>>>>> just
> >>>>>>>>>>>> send
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request to the
> >>>> TX-coordinator
> >>>>>> in any
> >>>>>>>>>>> case
> >>>>>>>>>>>> --
> >>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> old
> >>>>>>>>>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> correctly did send the
> >>>>>>>> `addPartition`
> >>>>>>>>>>>> request
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> already, the
> >>> TX-coordinator
> >>>>> can
> >>>>>> just
> >>>>>>>>>>>> "ignore"
> >>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>> idempotent.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> However,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if the old producer has
> >> a
> >>>> bug
> >>>>>> and
> >>>>>>>> did
> >>>>>>>>>>>> forget
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> sent
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> `addPartition`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request, we would now
> >>> ensure
> >>>>>> that
> >>>>>>>> the
> >>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>> indeed
> >>>>>>>>>>>>>>>>>>>>>> added
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX and thus fix a
> >>> potential
> >>>>>>>> producer bug
> >>>>>>>>>>>>> (even
> >>>>>>>>>>>>>>>> if we
> >>>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fencing via the bump
> >>> epoch).
> >>>>> --
> >>>>>> It
> >>>>>>>>>>> seems to
> >>>>>>>>>>>>> be
> >>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>> good
> >>>>>>>>>>>>>>>>>>>>>>>> improvement?
> >>>>>>>>>>>>>>>>>>>>>>>>> Or
> >>>>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there a reason to not do
> >>>> this?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is ongoing
> >> =
> >>>>>> partition
> >>>>>>>> was
> >>>>>>>>>>>> added
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>> via
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn. We
> >>>> check
> >>>>>> this
> >>>>>>>> with
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
> >>>>>>>>>>>>>>>>>>>>>>>>>> call.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this wasn't
> >>>>>> sufficiently
> >>>>>>>>>>>>> explained
> >>>>>>>>>>>>>>>> here:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do what I propose
> >> in
> >>>>>> (20), we
> >>>>>>>>>>> don't
> >>>>>>>>>>>>>> really
> >>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `DescribeTransaction`
> >>> call,
> >>>> as
> >>>>>> the
> >>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>> leader
> >>>>>>>>>>>>>>>>>>> adds
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for older clients and we
> >>> get
> >>>>>> this
> >>>>>>>> check
> >>>>>>>>>>> for
> >>>>>>>>>>>>>> free.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is that
> >> if
> >>>> any
> >>>>>>>> messages
> >>>>>>>>>>>>> somehow
> >>>>>>>>>>>>>>>> come
> >>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the producer,
> >>> they
> >>>>>> will be
> >>>>>>>>>>>> fenced.
> >>>>>>>>>>>>>>>>> However,
> >>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it can be
> >>>>>> discussed
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree that we should
> >>> have
> >>>>>> epoch
> >>>>>>>>>>> fencing.
> >>>>>>>>>>>> My
> >>>>>>>>>>>>>>>>>>> question is
> >>>>>>>>>>>>>>>>>>>>>>>>> different:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Assume we are at epoch
> >> 73,
> >>>> and
> >>>>>> we
> >>>>>>>> have
> >>>>>>>>>>> an
> >>>>>>>>>>>>>> ongoing
> >>>>>>>>>>>>>>>>>>>>>> transaction,
> >>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed. It seems
> >>> natural
> >>>> to
> >>>>>>>> write the
> >>>>>>>>>>>>>> "prepare
> >>>>>>>>>>>>>>>>>>> commit"
> >>>>>>>>>>>>>>>>>>>>>>> marker
> >>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
> >>> both
> >>>>> with
> >>>>>>>> epoch
> >>>>>>>>>>> 73,
> >>>>>>>>>>>>> too,
> >>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>> belongs
> >>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current transaction. Of
> >>>>> course,
> >>>>>> we
> >>>>>>>> now
> >>>>>>>>>>> also
> >>>>>>>>>>>>>> bump
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>> expect
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the next requests to
> >> have
> >>>>> epoch
> >>>>>> 74,
> >>>>>>>> and
> >>>>>>>>>>>> would
> >>>>>>>>>>>>>>>> reject
> >>>>>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch 73, as the
> >>>> corresponding
> >>>>>> TX
> >>>>>>>> for
> >>>>>>>>>>> epoch
> >>>>>>>>>>>>> 73
> >>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>> already
> >>>>>>>>>>>>>>>>>>>>>>>>> committed.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems you propose to
> >>>> write
> >>>>>> the
> >>>>>>>>>>> "prepare
> >>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>> marker"
> >>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
> >>> with
> >>>>>> epoch 74
> >>>>>>>>>>>> though,
> >>>>>>>>>>>>>> what
> >>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>> work,
> >>>>>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems confusing. Is
> >> there
> >>> a
> >>>>>> reason
> >>>>>>>> why
> >>>>>>>>>>> we
> >>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>> use
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> bumped
> >>>>>>>>>>>>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>>>>>>>>>> 74
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of the current
> >>> epoch
> >>>>> 73?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are checking if
> >>> the
> >>>>>>>>>>> transaction is
> >>>>>>>>>>>>>>>> ongoing,
> >>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the leader
> >>>>> partition
> >>>>>> to
> >>>>>>>> the
> >>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>> coordinator.
> >>>>>>>>>>>>>>>>>>>>>>>> In
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for this
> >>>>>> message to
> >>>>>>>> come
> >>>>>>>>>>>>> back,
> >>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>> theory
> >>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
> >> that
> >>>>> would
> >>>>>>>> make the
> >>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>>>>> result
> >>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> check
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why we
> >> can
> >>>>> check
> >>>>>> the
> >>>>>>>>>>> leader
> >>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>> write
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks. Got it.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, is this really
> >> an
> >>>>>> issue?
> >>>>>>>> We put
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> produce
> >>>>>>>>>>>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory, so how could
> >> we
> >>>>>> process
> >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
> >>>>>>>>>>>>>>>>>>>>>>>>> first?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Don't we need to put the
> >>>>>>>>>>>>>> `WriteTxnMarkerRequest`
> >>>>>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>>>>>>>> purgatory,
> >>>>>>>>>>>>>>>>>>>>>>>>> too,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for this case, and
> >> process
> >>>>> both
> >>>>>>>> request
> >>>>>>>>>>>>>> in-order?
> >>>>>>>>>>>>>>>>>>> (Again,
> >>>>>>>>>>>>>>>>>>>>> my
> >>>>>>>>>>>>>>>>>>>>>>>> broker
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> knowledge is limited and
> >>>> maybe
> >>>>>> we
> >>>>>>>> don't
> >>>>>>>>>>>>>> maintain
> >>>>>>>>>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>>>>>>> order
> >>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case, what seems to be
> >> an
> >>>>> issue
> >>>>>>>> IMHO,
> >>>>>>>>>>> and I
> >>>>>>>>>>>>> am
> >>>>>>>>>>>>>>>>>>> wondering
> >>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>> changing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request handling to
> >>> preserve
> >>>>>> order
> >>>>>>>> for
> >>>>>>>>>>> this
> >>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>>> might be
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> cleaner
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution?)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/30/22 3:28 PM,
> >> Artem
> >>>>>> Livshits
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think the interesting
> >>>> part
> >>>>> is
> >>>>>>>> not in
> >>>>>>>>>>>> this
> >>>>>>>>>>>>>>>> logic
> >>>>>>>>>>>>>>>>>>>> (because
> >>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>> tries
> >>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> figure out when
> >>>>>>>> UNKNOWN_PRODUCER_ID is
> >>>>>>>>>>>>>> retriable
> >>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>>>>> retryable,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's definitely not
> >>> fatal),
> >>>>> but
> >>>>>>>> what
> >>>>>>>>>>>> happens
> >>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>> logic
> >>>>>>>>>>>>>>>>>>>>>>>>> doesn't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'true' and falls
> >> through.
> >>>> In
> >>>>>> the
> >>>>>>>> old
> >>>>>>>>>>>>> clients
> >>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>> fatal,
> >>>>>>>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> keep the behavior in
> >> the
> >>>> new
> >>>>>>>> clients,
> >>>>>>>>>>> I'd
> >>>>>>>>>>>>>>>> expect it
> >>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>> fatal
> >>>>>>>>>>>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> well.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 29, 2022 at
> >>>> 11:57
> >>>>>> AM
> >>>>>>>>>>> Justine
> >>>>>>>>>>>>>> Olshan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>> <jolshan@confluent.io.invalid
> >>>>>>
> >>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Artem and Jeff,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
> >> look
> >>>> and
> >>>>>>>> sorry for
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>> slow
> >>>>>>>>>>>>>>>>>>>> response.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You both mentioned the
> >>>>> change
> >>>>>> to
> >>>>>>>>>>> handle
> >>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> >>>>>>>>>>>>>>>>>>>>>>>>> errors.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> To
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear — this error
> >> code
> >>>> will
> >>>>>> only
> >>>>>>>> be
> >>>>>>>>>>> sent
> >>>>>>>>>>>>>> again
> >>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> client's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version is high enough
> >>> to
> >>>>>> ensure
> >>>>>>>> we
> >>>>>>>>>>>> handle
> >>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>> correctly.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The current (Java)
> >>> client
> >>>>>> handles
> >>>>>>>>>>> this by
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> following
> >>>>>>>>>>>>>>>>>>>>>>>> (somewhat
> >>>>>>>>>>>>>>>>>>>>>>>>>>> long)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code snippet:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // An
> >>> UNKNOWN_PRODUCER_ID
> >>>>>> means
> >>>>>>>> that
> >>>>>>>>>>> we
> >>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>> lost
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker. Depending on
> >> the
> >>>> log
> >>>>>> start
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // offset, we may want
> >>> to
> >>>>>> retry
> >>>>>>>>>>> these, as
> >>>>>>>>>>>>>>>>> described
> >>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>> each
> >>>>>>>>>>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below. If
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> none of those apply,
> >>> then
> >>>>> for
> >>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // idempotent
> >> producer,
> >>> we
> >>>>>> will
> >>>>>>>>>>> locally
> >>>>>>>>>>>>> bump
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>> reset
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence numbers of
> >>>>> in-flight
> >>>>>>>> batches
> >>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // sequence 0, then
> >>> retry
> >>>>> the
> >>>>>>>> failed
> >>>>>>>>>>>> batch,
> >>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>>>> now
> >>>>>>>>>>>>>>>>>>>>>>>>>> succeed.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the transactional
> >>>> producer,
> >>>>>> allow
> >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // batch to fail. When
> >>>>>> processing
> >>>>>>>> the
> >>>>>>>>>>>>> failed
> >>>>>>>>>>>>>>>>> batch,
> >>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>>>> transition
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an abortable error and
> >>>> set a
> >>>>>> flag
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // indicating that we
> >>> need
> >>>>> to
> >>>>>>>> bump the
> >>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>> (if
> >>>>>>>>>>>>>>>>>>>>> supported
> >>>>>>>>>>>>>>>>>>>>>> by
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if (error ==
> >>>>>>>>>>>> Errors.*UNKNOWN_PRODUCER_ID*)
> >>>>>>>>>>>>> {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        if
> >>>>>> (response.logStartOffset
> >>>>>>>> ==
> >>>>>>>>>>> -1)
> >>>>>>>>>>>> {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // We don't
> >>> know
> >>>>>> the log
> >>>>>>>>>>> start
> >>>>>>>>>>>>>> offset
> >>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>> response.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> We
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the request
> >>>> until
> >>>>>> we
> >>>>>>>> get
> >>>>>>>>>>> it.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // The
> >>>>>>>> UNKNOWN_PRODUCER_ID
> >>>>>>>>>>>> error
> >>>>>>>>>>>>>> code
> >>>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>> added
> >>>>>>>>>>>>>>>>>>>>>>> along
> >>>>>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProduceResponse which
> >>>>>> includes the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            //
> >>>> logStartOffset.
> >>>>>> So
> >>>>>>>> the
> >>>>>>>>>>> '-1'
> >>>>>>>>>>>>>>>> sentinel
> >>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>> backward
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatibility.
> >> Instead,
> >>> it
> >>>>> is
> >>>>>>>> possible
> >>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // a broker
> >> to
> >>>> not
> >>>>>> know
> >>>>>>>> the
> >>>>>>>>>>>>>>>>>>> logStartOffset at
> >>>>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returning
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response because
> >> the
> >>>>>> partition
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // may have
> >>>> moved
> >>>>>> away
> >>>>>>>> from
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>> broker
> >>>>>>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> time
> >>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error was
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initially raised to
> >> the
> >>>> time
> >>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // response
> >>> was
> >>>>>> being
> >>>>>>>>>>>>> constructed.
> >>>>>>>>>>>>>> In
> >>>>>>>>>>>>>>>>>>> these
> >>>>>>>>>>>>>>>>>>>>>> cases,
> >>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retry the request: we
> >>> are
> >>>>>>>> guaranteed
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // to
> >>> eventually
> >>>>>> get a
> >>>>>>>>>>>>>> logStartOffset
> >>>>>>>>>>>>>>>>> once
> >>>>>>>>>>>>>>>>>>>>> things
> >>>>>>>>>>>>>>>>>>>>>>>>> settle
> >>>>>>>>>>>>>>>>>>>>>>>>>>> down.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            return true;
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        if
> >>>>>>>>>>> (batch.sequenceHasBeenReset()) {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // When the
> >>>> first
> >>>>>>>> inflight
> >>>>>>>>>>>> batch
> >>>>>>>>>>>>>>>> fails
> >>>>>>>>>>>>>>>>>>> due to
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> truncation
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> then the sequences of
> >>> all
> >>>>> the
> >>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // in flight
> >>>>> batches
> >>>>>>>> would
> >>>>>>>>>>> have
> >>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>>>> restarted
> >>>>>>>>>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> beginning.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, when those
> >>>>> responses
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // come back
> >>>> from
> >>>>>> the
> >>>>>>>>>>> broker,
> >>>>>>>>>>>>> they
> >>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>> also
> >>>>>>>>>>>>>>>>>>>>>> come
> >>>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> >>> error.
> >>>>> In
> >>>>>> this
> >>>>>>>>>>> case,
> >>>>>>>>>>>> we
> >>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // reset the
> >>>>>> sequence
> >>>>>>>>>>> numbers
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> beginning.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            return true;
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        } else if
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>> (lastAckedOffset(batch.topicPartition).orElse(
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> >>>>>>>>>>>>>>>>>>>>>> response.logStartOffset) {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // The head
> >> of
> >>>> the
> >>>>>> log
> >>>>>>>> has
> >>>>>>>>>>> been
> >>>>>>>>>>>>>>>> removed,
> >>>>>>>>>>>>>>>>>>>>> probably
> >>>>>>>>>>>>>>>>>>>>>>> due
> >>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retention time
> >> elapsing.
> >>>> In
> >>>>>> this
> >>>>>>>> case,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // we expect
> >>> to
> >>>>>> lose the
> >>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>> state.
> >>>>>>>>>>>>>>>>>>> For
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> transactional
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer, reset the
> >>>>> sequences
> >>>>>> of
> >>>>>>>> all
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // inflight
> >>>>> batches
> >>>>>> to
> >>>>>>>> be
> >>>>>>>>>>> from
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> beginning
> >>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>> retry
> >>>>>>>>>>>>>>>>>>>>>>>>>>> them,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the transaction
> >>> does
> >>>>> not
> >>>>>>>> need to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // be
> >> aborted.
> >>>> For
> >>>>>> the
> >>>>>>>>>>>> idempotent
> >>>>>>>>>>>>>>>>>>> producer,
> >>>>>>>>>>>>>>>>>>>>> bump
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> avoid
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reusing (sequence,
> >>> epoch)
> >>>>>> pairs
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            if
> >>>>>> (isTransactional()) {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>> txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>> this.producerIdAndEpoch);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            } else {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>> requestEpochBumpForPartition(batch.topicPartition);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            return true;
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        if
> >>>>> (!isTransactional())
> >>>>>> {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // For the
> >>>>>> idempotent
> >>>>>>>>>>> producer,
> >>>>>>>>>>>>>>>> always
> >>>>>>>>>>>>>>>>>>> retry
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors. If the batch
> >> has
> >>>> the
> >>>>>>>> current
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // producer
> >> ID
> >>>> and
> >>>>>>>> epoch,
> >>>>>>>>>>>>> request a
> >>>>>>>>>>>>>>>> bump
> >>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> epoch.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Otherwise
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the
> >> produce.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>> requestEpochBumpForPartition(batch.topicPartition);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            return true;
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I was considering
> >>> keeping
> >>>>> this
> >>>>>>>>>>> behavior —
> >>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>> am
> >>>>>>>>>>>>>>>>>>> open
> >>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>> simplifying
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We are leaving changes
> >>> to
> >>>>>> older
> >>>>>>>>>>> clients
> >>>>>>>>>>>> off
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> table
> >>>>>>>>>>>>>>>>>>>>> here
> >>>>>>>>>>>>>>>>>>>>>>>> since
> >>>>>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> caused
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many issues for
> >> clients
> >>> in
> >>>>> the
> >>>>>>>> past.
> >>>>>>>>>>>>>> Previously
> >>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>> fatal
> >>>>>>>>>>>>>>>>>>>>>>>>>>> error
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we didn't have the
> >>>>> mechanisms
> >>>>>> in
> >>>>>>>>>>> place to
> >>>>>>>>>>>>>>>> detect
> >>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> legitimate
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case vs some bug or
> >> gap
> >>> in
> >>>>> the
> >>>>>>>>>>> protocol.
> >>>>>>>>>>>>>>>> Ensuring
> >>>>>>>>>>>>>>>>>>> each
> >>>>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>> has
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> its
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> own epoch should close
> >>>> this
> >>>>>> gap.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> And to address Jeff's
> >>>> second
> >>>>>>>> point:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *does the typical
> >>> produce
> >>>>>> request
> >>>>>>>> path
> >>>>>>>>>>>>> append
> >>>>>>>>>>>>>>>>>>> records
> >>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>> local
> >>>>>>>>>>>>>>>>>>>>>>>>> log
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along*
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *with the
> >>>>>> currentTxnFirstOffset
> >>>>>>>>>>>>> information?
> >>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand*
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *when the field is
> >>> written
> >>>>> to
> >>>>>>>> disk.*
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, the first produce
> >>>>> request
> >>>>>>>>>>> populates
> >>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>> field
> >>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>> writes
> >>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> offset
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as part of the record
> >>>> batch
> >>>>>> and
> >>>>>>>> also
> >>>>>>>>>>> to
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>>>>>>> snapshot.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we reload the records
> >> on
> >>>>>> restart
> >>>>>>>>>>> and/or
> >>>>>>>>>>>>>>>>>>> reassignment,
> >>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>> repopulate
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> field with the
> >> snapshot
> >>>> from
> >>>>>> disk
> >>>>>>>>>>> along
> >>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> rest
> >>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if there
> >> are
> >>>>>> further
> >>>>>>>>>>> comments
> >>>>>>>>>>>>>>>> and/or
> >>>>>>>>>>>>>>>>>>>>> questions.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> >> at
> >>>> 9:00
> >>>>>> PM
> >>>>>>>> Jeff
> >>>>>>>>>>> Kim
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> <jeff.kim@confluent.io.invalid
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP! I
> >>>> have
> >>>>>> two
> >>>>>>>>>>>> questions:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1) For new clients,
> >> we
> >>>> can
> >>>>>> once
> >>>>>>>> again
> >>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>> error
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for sequences
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that are non-zero
> >> when
> >>>>> there
> >>>>>> is
> >>>>>>>> no
> >>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>> present
> >>>>>>>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This will indicate we
> >>>>> missed
> >>>>>> the
> >>>>>>>> 0
> >>>>>>>>>>>>> sequence
> >>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>> yet
> >>>>>>>>>>>>>>>>>>>>>>>>> want
> >>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the log.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would like to
> >>>> understand
> >>>>>> the
> >>>>>>>>>>> current
> >>>>>>>>>>>>>>>> behavior
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> handle
> >>>>>>>>>>>>>>>>>>>>>>>> older
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and if there are any
> >>>>> changes
> >>>>>> we
> >>>>>>>> are
> >>>>>>>>>>>>> making.
> >>>>>>>>>>>>>>>> Maybe
> >>>>>>>>>>>>>>>>>>> I'm
> >>>>>>>>>>>>>>>>>>>>>>> missing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but we would want to
> >>>>> identify
> >>>>>>>>>>> whether we
> >>>>>>>>>>>>>>>> missed
> >>>>>>>>>>>>>>>>>>> the 0
> >>>>>>>>>>>>>>>>>>>>>>> sequence
> >>>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients, no?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2) Upon returning
> >> from
> >>>> the
> >>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>> coordinator, we
> >>>>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>> set
> >>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as ongoing on the
> >>> leader
> >>>> by
> >>>>>>>>>>> populating
> >>>>>>>>>>>>>>>>>>>>>> currentTxnFirstOffset
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through the typical
> >>>> produce
> >>>>>>>> request
> >>>>>>>>>>>>>> handling.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does the typical
> >>> produce
> >>>>>> request
> >>>>>>>> path
> >>>>>>>>>>>>> append
> >>>>>>>>>>>>>>>>>>> records
> >>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>> local
> >>>>>>>>>>>>>>>>>>>>>>>>> log
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> >>>>>> currentTxnFirstOffset
> >>>>>>>>>>>>> information?
> >>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> understand
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when the field is
> >>> written
> >>>>> to
> >>>>>>>> disk.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeff
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> >> at
> >>>>> 4:44
> >>>>>> PM
> >>>>>>>> Artem
> >>>>>>>>>>>>>> Livshits
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
> >>> alivshits@confluent.io
> >>>>>> .invalid>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for the
> >> KIP.
> >>>> I
> >>>>>> have
> >>>>>>>> one
> >>>>>>>>>>>>>> question.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 5) For new clients,
> >> we
> >>>> can
> >>>>>> once
> >>>>>>>>>>> again
> >>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>> error
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe we had
> >>>> problems
> >>>>>> in the
> >>>>>>>>>>> past
> >>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>> returning
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because it was
> >>>> considered
> >>>>>> fatal
> >>>>>>>> and
> >>>>>>>>>>>>>> required
> >>>>>>>>>>>>>>>>>>> client
> >>>>>>>>>>>>>>>>>>>>>>> restart.
> >>>>>>>>>>>>>>>>>>>>>>>>> It
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good to spell out
> >> the
> >>>> new
> >>>>>> client
> >>>>>>>>>>>> behavior
> >>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>> receives
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> error.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
> >>> at
> >>>>>> 10:00 AM
> >>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>> Olshan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>> <jo...@confluent.io.invalid>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
> >>>> look
> >>>>>>>> Matthias.
> >>>>>>>>>>>> I've
> >>>>>>>>>>>>>>>> tried
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> answer
> >>>>>>>>>>>>>>>>>>>>>>> your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> questions
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 10)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Right — so the
> >>> hanging
> >>>>>>>> transaction
> >>>>>>>>>>>> only
> >>>>>>>>>>>>>>>> occurs
> >>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>> late
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> come in and the
> >>>> partition
> >>>>>> is
> >>>>>>>> never
> >>>>>>>>>>>> added
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>> again.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> If
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never add the
> >>> partition
> >>>>> to
> >>>>>> a
> >>>>>>>>>>>>> transaction,
> >>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>> never
> >>>>>>>>>>>>>>>>>>>>>>>> write
> >>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> marker
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never advance the
> >>> LSO.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do end up
> >>> adding
> >>>>> the
> >>>>>>>>>>> partition
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>> (I
> >>>>>>>>>>>>>>>>>>>>>>>>>> suppose
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happen before or
> >>> after
> >>>>> the
> >>>>>> late
> >>>>>>>>>>>> message
> >>>>>>>>>>>>>>>> comes
> >>>>>>>>>>>>>>>>> in)
> >>>>>>>>>>>>>>>>>>>> then
> >>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> include
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late message in the
> >>>> next
> >>>>>>>>>>> (incorrect)
> >>>>>>>>>>>>>>>>> transaction.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> So perhaps it is
> >>>> clearer
> >>>>> to
> >>>>>>>> make
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> distinction
> >>>>>>>>>>>>>>>>>>>>> between
> >>>>>>>>>>>>>>>>>>>>>>>>>> messages
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> eventually get
> >> added
> >>> to
> >>>>> the
> >>>>>>>>>>>> transaction
> >>>>>>>>>>>>>> (but
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> wrong
> >>>>>>>>>>>>>>>>>>>>>>> one)
> >>>>>>>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that never get
> >> added
> >>>> and
> >>>>>> become
> >>>>>>>>>>>> hanging.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side
> >>> change
> >>>>> for
> >>>>>> 2 is
> >>>>>>>>>>>> removing
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> addPartitions
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need
> >>> to
> >>>>> make
> >>>>>>>> this
> >>>>>>>>>>> from
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> txn
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In my opinion, the
> >>>> issue
> >>>>>> with
> >>>>>>>> the
> >>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> >>>>>>>>>>>>>>>>>>>>>> call
> >>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>> older
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is that we don't
> >> have
> >>>> the
> >>>>>> epoch
> >>>>>>>>>>> bump,
> >>>>>>>>>>>> so
> >>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> belongs to the
> >>> previous
> >>>>>>>>>>> transaction or
> >>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>> one.
> >>>>>>>>>>>>>>>>>>> We
> >>>>>>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>> check
> >>>>>>>>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition has been
> >>>> added
> >>>>> to
> >>>>>>>> this
> >>>>>>>>>>>>>>>> transaction.
> >>>>>>>>>>>>>>>>> Of
> >>>>>>>>>>>>>>>>>>>>> course,
> >>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>> means
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't completely
> >>> cover
> >>>>> the
> >>>>>> case
> >>>>>>>>>>> where
> >>>>>>>>>>>> we
> >>>>>>>>>>>>>>>> have a
> >>>>>>>>>>>>>>>>>>>> really
> >>>>>>>>>>>>>>>>>>>>>>> late
> >>>>>>>>>>>>>>>>>>>>>>>>>>> message
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have added the
> >>>> partition
> >>>>> to
> >>>>>>>> the new
> >>>>>>>>>>>>>>>>> transaction,
> >>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>> that's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something we will
> >>> need
> >>>>> the
> >>>>>> new
> >>>>>>>>>>> clients
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> cover.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is
> >>> ongoing
> >>>> =
> >>>>>>>> partition
> >>>>>>>>>>> was
> >>>>>>>>>>>>>>>> added to
> >>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>> via
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn.
> >>> We
> >>>>>> check
> >>>>>>>> this
> >>>>>>>>>>> with
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this
> >>> wasn't
> >>>>>>>> sufficiently
> >>>>>>>>>>>>>>>> explained
> >>>>>>>>>>>>>>>>>>> here:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is
> >> that
> >>>> if
> >>>>>> any
> >>>>>>>>>>> messages
> >>>>>>>>>>>>>>>> somehow
> >>>>>>>>>>>>>>>>>>> come
> >>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the
> >>> producer,
> >>>>> they
> >>>>>>>> will be
> >>>>>>>>>>>>>> fenced.
> >>>>>>>>>>>>>>>>>>> However,
> >>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it
> >> can
> >>> be
> >>>>>>>> discussed
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 50)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It should be
> >>>> synchronous
> >>>>>>>> because
> >>>>>>>>>>> if we
> >>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>> event
> >>>>>>>>>>>>>>>>>>>>>> (ie,
> >>>>>>>>>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>>>>>>>> error)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes us to need
> >> to
> >>>>> abort
> >>>>>> the
> >>>>>>>>>>>>>> transaction,
> >>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions to send
> >>>>>> transaction
> >>>>>>>>>>> markers
> >>>>>>>>>>>>> to.
> >>>>>>>>>>>>>>>> We
> >>>>>>>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> partitions
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we added them to
> >> the
> >>>>>>>> coordinator
> >>>>>>>>>>> via
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> >>>>>>>>>>>>>>>>>>>>>>>>>> call.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Previously we have
> >>> had
> >>>>>>>> asynchronous
> >>>>>>>>>>>>> calls
> >>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> past
> >>>>>>>>>>>>>>>>>>>>>> (ie,
> >>>>>>>>>>>>>>>>>>>>>>>>>> writing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit markers when
> >>> the
> >>>>>>>>>>> transaction is
> >>>>>>>>>>>>>>>>> completed)
> >>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>> often
> >>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes confusion as
> >>> we
> >>>>>> need to
> >>>>>>>> wait
> >>>>>>>>>>>> for
> >>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>> operations
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> complete.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing commit
> >>> markers
> >>>>>> case,
> >>>>>>>>>>> clients
> >>>>>>>>>>>>> often
> >>>>>>>>>>>>>>>> see
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> CONCURRENT_TRANSACTIONs
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error messages and
> >>> that
> >>>>>> can be
> >>>>>>>>>>>>> confusing.
> >>>>>>>>>>>>>>>> For
> >>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>> reason,
> >>>>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>> may
> >>>>>>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simpler to just
> >> have
> >>>>>>>> synchronous
> >>>>>>>>>>>> calls —
> >>>>>>>>>>>>>>>>>>> especially
> >>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> block
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some operation's
> >>>>> completion
> >>>>>>>> anyway
> >>>>>>>>>>>>> before
> >>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>> start
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> next
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. And
> >>> yes, I
> >>>>>> meant
> >>>>>>>>>>>>>> coordinator. I
> >>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>> fix
> >>>>>>>>>>>>>>>>>>>>>>> that.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are
> >> checking
> >>> if
> >>>>> the
> >>>>>>>>>>>> transaction
> >>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>> ongoing,
> >>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the
> >> leader
> >>>>>> partition
> >>>>>>>> to
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>> coordinator.
> >>>>>>>>>>>>>>>>>>>>>>>>>> In
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for
> >>> this
> >>>>>>>> message to
> >>>>>>>>>>>> come
> >>>>>>>>>>>>>>>> back,
> >>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>> theory
> >>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
> >>>> that
> >>>>>> would
> >>>>>>>> make
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>>>>>>> result
> >>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> check
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why
> >> we
> >>>> can
> >>>>>> check
> >>>>>>>> the
> >>>>>>>>>>>>> leader
> >>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>> write
> >>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm happy to update
> >>> the
> >>>>>> KIP if
> >>>>>>>>>>> some of
> >>>>>>>>>>>>>> these
> >>>>>>>>>>>>>>>>>>> things
> >>>>>>>>>>>>>>>>>>>>> were
> >>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>> clear.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Nov 21,
> >> 2022
> >>> at
> >>>>>> 7:11 PM
> >>>>>>>>>>>> Matthias
> >>>>>>>>>>>>>> J.
> >>>>>>>>>>>>>>>>> Sax <
> >>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the
> >> KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Couple of
> >>>> clarification
> >>>>>>>> questions
> >>>>>>>>>>> (I
> >>>>>>>>>>>> am
> >>>>>>>>>>>>>>>> not a
> >>>>>>>>>>>>>>>>>>>> broker
> >>>>>>>>>>>>>>>>>>>>>>> expert
> >>>>>>>>>>>>>>>>>>>>>>>>> do
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some question are
> >>>>> obvious
> >>>>>> for
> >>>>>>>>>>> others,
> >>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>> me
> >>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>> my
> >>>>>>>>>>>>>>>>>>>>>>>>>> lack
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker knowledge).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (10)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What happens if
> >> the
> >>>>>> message
> >>>>>>>> come
> >>>>>>>>>>> in
> >>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> next
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request? It seems
> >>> the
> >>>>>> broker
> >>>>>>>>>>> hosting
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> data
> >>>>>>>>>>>>>>>>>>>>>> partitions
> >>>>>>>>>>>>>>>>>>>>>>>>> won't
> >>>>>>>>>>>>>>>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anything about it
> >>> and
> >>>>>> append
> >>>>>>>> it to
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> partition,
> >>>>>>>>>>>>>>>>>>>>> too?
> >>>>>>>>>>>>>>>>>>>>>>> What
> >>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> difference between
> >>>> both
> >>>>>> cases?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, it seems a
> >> TX
> >>>>> would
> >>>>>> only
> >>>>>>>>>>> hang,
> >>>>>>>>>>>> if
> >>>>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>>>> is no
> >>>>>>>>>>>>>>>>>>>>>>>> following
> >>>>>>>>>>>>>>>>>>>>>>>>>> TX
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> either committer
> >> or
> >>>>>> aborted?
> >>>>>>>> Thus,
> >>>>>>>>>>>> for
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>> above,
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> TX
> >>>>>>>>>>>>>>>>>>>>>>>>>>> might
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually not hang
> >>> (of
> >>>>>> course,
> >>>>>>>> we
> >>>>>>>>>>>> might
> >>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>> EOS
> >>>>>>>>>>>>>>>>>>>>>>> violation
> >>>>>>>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX was aborted and
> >>> the
> >>>>>> second
> >>>>>>>>>>>>> committed,
> >>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>>> way
> >>>>>>>>>>>>>>>>>>>>>>>>>>> around).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (20)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Of course, 1 and
> >> 2
> >>>>>> require
> >>>>>>>>>>>> client-side
> >>>>>>>>>>>>>>>>>>> changes, so
> >>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>> older
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> those approaches
> >>> won’t
> >>>>>> apply.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For (1) I
> >> understand
> >>>>> why a
> >>>>>>>> client
> >>>>>>>>>>>>> change
> >>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>> necessary,
> >>>>>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>> sure
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we need a client
> >>>> change
> >>>>>> for
> >>>>>>>> (2).
> >>>>>>>>>>> Can
> >>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>> elaborate?
> >>>>>>>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>>>>>>>>> Later
> >>>>>>>>>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> explain
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that we should
> >> send
> >>> a
> >>>>>>>>>>>>>>>>>>> DescribeTransactionRequest,
> >>>>>>>>>>>>>>>>>>>>> but I
> >>>>>>>>>>>>>>>>>>>>>>> am
> >>>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>> sure
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Can't we not just
> >> do
> >>>> an
> >>>>>>>> implicit
> >>>>>>>>>>>>>>>>>>> AddPartiitonToTx,
> >>>>>>>>>>>>>>>>>>>>> too?
> >>>>>>>>>>>>>>>>>>>>>>> If
> >>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> old
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer correctly
> >>>>>> registered
> >>>>>>>> the
> >>>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>>>> already,
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can just ignore it
> >>> as
> >>>>>> it's an
> >>>>>>>>>>>>> idempotent
> >>>>>>>>>>>>>>>>>>> operation?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (30)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To cover older
> >>>> clients,
> >>>>>> we
> >>>>>>>> will
> >>>>>>>>>>>>> ensure a
> >>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>> ongoing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we write to a
> >>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Not sure what you
> >>> mean
> >>>>> by
> >>>>>>>> this?
> >>>>>>>>>>> Can
> >>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>> elaborate?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (40)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [the
> >>> TX-coordinator]
> >>>>> will
> >>>>>>>> write
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>> prepare
> >>>>>>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>>>>>> message
> >>>>>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch and send
> >>>>>>>>>>> WriteTxnMarkerRequests
> >>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> bumped
> >>>>>>>>>>>>>>>>>>>>>>>> epoch.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why do we use the
> >>>> bumped
> >>>>>>>> epoch for
> >>>>>>>>>>>>> both?
> >>>>>>>>>>>>>> It
> >>>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>>>>>>>> intuitive
> >>>>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the current epoch,
> >>> and
> >>>>>> only
> >>>>>>>> return
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> bumped
> >>>>>>>>>>>>>>>>>>> epoch
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (50) "Implicit
> >>>>>>>>>>>>> AddPartitionToTransaction"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why does the
> >>>> implicitly
> >>>>>> sent
> >>>>>>>>>>> request
> >>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>> synchronous?
> >>>>>>>>>>>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also says
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in case we need
> >> to
> >>>>> abort
> >>>>>> and
> >>>>>>>>>>> need to
> >>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>>>>>> partitions
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What do you mean
> >> by
> >>>>> this?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we don’t want to
> >>>> write
> >>>>>> to it
> >>>>>>>>>>> before
> >>>>>>>>>>>> we
> >>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> manager
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Do you mean
> >>>>> TX-coordinator
> >>>>>>>>>>> instead of
> >>>>>>>>>>>>>>>>> "manager"?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (60)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For older clients
> >>> and
> >>>>>> ensuring
> >>>>>>>>>>> that
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>> TX
> >>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>> ongoing,
> >>>>>>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> describe a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> race condition. I
> >> am
> >>>> not
> >>>>>> sure
> >>>>>>>> if I
> >>>>>>>>>>>> can
> >>>>>>>>>>>>>>>> follow
> >>>>>>>>>>>>>>>>>>> here.
> >>>>>>>>>>>>>>>>>>>>> Can
> >>>>>>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> elaborate?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/18/22 1:21
> >> PM,
> >>>>>> Justine
> >>>>>>>>>>> Olshan
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey all!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to
> >> start a
> >>>>>>>> discussion
> >>>>>>>>>>> on my
> >>>>>>>>>>>>>>>> proposal
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> add
> >>>>>>>>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checks on
> >>>> transactions
> >>>>> to
> >>>>>>>> avoid
> >>>>>>>>>>>>> hanging
> >>>>>>>>>>>>>>>>>>>>> transactions.
> >>>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an issue for some
> >>>> time,
> >>>>>> so I
> >>>>>>>>>>> really
> >>>>>>>>>>>>> hope
> >>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>> helpful
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users of EOS.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The KIP includes
> >>>>> changes
> >>>>>> that
> >>>>>>>>>>> will
> >>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>> compatible
> >>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>> old
> >>>>>>>>>>>>>>>>>>>>>>>>>>> clients
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> changes to
> >> improve
> >>>>>>>> performance
> >>>>>>>>>>> and
> >>>>>>>>>>>>>>>>> correctness
> >>>>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>>>> clients.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Please take a
> >> look
> >>>> and
> >>>>>> leave
> >>>>>>>> any
> >>>>>>>>>>>>>> comments
> >>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>> may
> >>>>>>>>>>>>>>>>>>>>>> have!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> JIRA:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>> https://issues.apache.org/jira/browse/KAFKA-14402
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by "Matthias J. Sax" <mj...@apache.org>.
Just a side note about Guozhang comments about timestamps.

If the producer sets the timestamp, putting the record into purgatory 
seems not to be an issue (as already said: for this case we don't 
guarantee timestamp order between writes of different producers anyway). 
However, if the broker sets the timestamp, the expectation is that there 
is no out-of-order data in the partition ever; if we would introduce 
out-of-order data for this case (for interleaved writes of different 
producers), it seems we would violate the current contract? (To be fair: 
I don't know if that's an official contract, but I assume people rely on 
this behavior -- and it "advertised" in many public talks...)

About compaction: there is actually KIP-280 that adds timestamp based 
compaction what is a very useful feature for Kafka Streams with regard 
to out-of-order data handling. So the impact if we introduce 
out-of-order data could be larger scoped.


-Matthias


On 1/20/23 4:48 PM, Justine Olshan wrote:
> Hey Artem,
> 
> I see there is a check for transactional producers. I'm wondering if we
> don't handle the epoch overflow case. I'm also not sure it will be a huge
> issue to extend to transactional producers, but maybe I'm missing something.
> 
> As for the recovery path -- I think Guozhang's point was if we have a bad
> client that repeatedly tries to produce without adding to the transaction
> we would do the following:
> a) if not fatal, we just fail the produce request over and over
> b) if fatal, we fence the producer
> 
> Here with B, the issue with the client would be made clear more quickly. I
> suppose there are some intermediate cases where the issue only occurs
> sometimes, but I wonder if we should consider how to recover with clients
> who don't behave as expected anyway.
> 
> I think there is a place for the abortable error that we are adding -- just
> abort and try again. But I think there are also some cases where trying to
> recover overcomplicates some logic. Especially if we are considering older
> clients -- there I'm not sure if there's a ton we can do besides fail the
> batch or fence the producer. With newer clients, we can consider more
> options for what can just be recovered after aborting. But epochs might be
> a hard one unless we also want to reset producer ID.
> 
> Thanks,
> Justine
> 
> 
> 
> On Fri, Jan 20, 2023 at 3:59 PM Artem Livshits
> <al...@confluent.io.invalid> wrote:
> 
>>>   besides the poorly written client case
>>
>> A poorly written client could create a lot of grief to people who run Kafka
>> brokers :-), so when deciding to make an error fatal I would see if there
>> is a reasonable recovery path rather than how often it could happen.  If we
>> have solid implementation of transactions (which I hope we'll do as a
>> result of this KIP), it would help to recover from a large class of errors
>> by just aborting a transaction, even if the cause of error is a race
>> condition or etc.
>>
>> -Artem
>>
>> On Fri, Jan 20, 2023 at 3:26 PM Justine Olshan
>> <jo...@confluent.io.invalid>
>> wrote:
>>
>>> Artem --
>>> I guess the discussion path we were going down is when we expect to see
>>> this error. I mentioned that it was hard to come up with cases for when
>> the
>>> producer would still be around to receive the error besides the poorly
>>> written client case.
>>> If we don't expect to have a producer to receive the response, it sort of
>>> makes sense for it to be fatal.
>>>
>>> I had some discussion with Jason offline about the epoch being off cases
>>> and I'm not sure we could find a ton (outside of produce requests) where
>> we
>>> could/should recover. I'd be happy to hear some examples though, maybe
>> I'm
>>> missing something.
>>>
>>> Thanks,
>>> Justine
>>>
>>> On Fri, Jan 20, 2023 at 3:19 PM Artem Livshits
>>> <al...@confluent.io.invalid> wrote:
>>>
>>>> In general, I'd like to avoid fatal errors as much as possible, in some
>>>> sense fatal errors just push out recovery logic to the application
>> which
>>>> either complicates the application or leads to disruption (we've seen
>>> cases
>>>> when a transient broker error could lead to work stoppage when
>>> applications
>>>> need to be manually restarted).  I think we should strive to define
>>>> recovery logic for most errors (and/or encapsulate it in the Kafka
>> client
>>>> as much as possible).
>>>>
>>>> One benefit of transactions is that they simplify recovery from errors,
>>>> pretty much any error (that's not handled transparently by retries in
>>> Kafka
>>>> client) can be handled by the application via aborting the transaction
>>> and
>>>> repeating the transactional logic again.  One tricky error is an error
>>>> during commit, because we don't know the outcome.  For commit errors,
>> the
>>>> recommendation should be to retry the commit until it returns the
>>> specific
>>>> result (committed or aborted).
>>>>
>>>> -Artem
>>>>
>>>> On Fri, Jan 20, 2023 at 2:52 PM Justine Olshan
>>>> <jo...@confluent.io.invalid>
>>>> wrote:
>>>>
>>>>> That's a fair point about other clients.
>>>>>
>>>>> I think the abortable error case is interesting because I'm curious
>> how
>>>>> other clients would handle this. I assume they would need to
>> implement
>>>>> handling for the error code unless they did something like "any
>> unknown
>>>>> error codes/any codes that aren't x,y,z are retriable." I would hope
>>> that
>>>>> unknown error codes were fatal, and if the code was implemented it
>>> would
>>>>> abort the transaction. But I will think on this too.
>>>>>
>>>>> As for InvalidRecord -- you mentioned it was not fatal, but I'm
>> taking
>>> a
>>>>> look through the code. We would see this on handling the produce
>>>> response.
>>>>> If I recall correctly, we check if errors are retriable. I think this
>>>> error
>>>>> would not be retriable. But I guess the concern here is that it is
>> not
>>>>> enough for just that batch to fail. I guess I hadn't considered fully
>>>>> fencing the old producer but there are valid arguments here why we
>>> would
>>>>> want to.
>>>>>
>>>>> Thanks,
>>>>> Justine
>>>>>
>>>>> On Fri, Jan 20, 2023 at 2:35 PM Guozhang Wang <
>>>> guozhang.wang.us@gmail.com>
>>>>> wrote:
>>>>>
>>>>>> Thanks Justine for the replies! I agree with most of your thoughts.
>>>>>>
>>>>>> Just for 3/7), though I agree for our own AK producer, since we do
>>>>>> "nextRequest(boolean hasIncompleteBatches)", we guarantee the
>> end-txn
>>>>>> would not be sent until we've effectively flushed, but I was
>>> referring
>>>>>> to any future bugs or other buggy clients that the same client may
>>> get
>>>>>> into this situation, in which case we should give the client a
>> clear
>>>>>> msg that "you did something wrong, and hence now you should fatally
>>>>>> close yourself". What I'm concerned about is that, by seeing an
>>>>>> "abortable error" or in some rare cases an "invalid record", the
>>>>>> client could not realize "something that's really bad happened". So
>>>>>> it's not about adding a new error, it's mainly about those real
>> buggy
>>>>>> situations causing such "should never happen" cases, the errors
>>> return
>>>>>> would not be informative enough.
>>>>>>
>>>>>> Thinking in other ways, if we believe that for most cases such
>> error
>>>>>> codes would not reach the original clients since they would be
>>>>>> disconnected or even gone by that time, and only in some rare cases
>>>>>> they would still be seen by the sending clients, then why not make
>>>>>> them more fatal and more specific than generic.
>>>>>>
>>>>>> Guozhang
>>>>>>
>>>>>> On Fri, Jan 20, 2023 at 1:59 PM Justine Olshan
>>>>>> <jo...@confluent.io.invalid> wrote:
>>>>>>>
>>>>>>> Hey Guozhang. Thanks for taking a look and for the detailed
>>> comments!
>>>>>> I'll
>>>>>>> do my best to address below.
>>>>>>>
>>>>>>> 1. I see what you are saying here, but I think I need to look
>>> through
>>>>> the
>>>>>>> sequence of events you mention. Typically we've seen this issue
>> in
>>> a
>>>>> few
>>>>>>> cases.
>>>>>>>
>>>>>>>   One is when we have a producer disconnect when trying to
>> produce.
>>>>>>> Typically in these cases, we abort the transaction. We've seen
>> that
>>>>> after
>>>>>>> the markers are written, the disconnection can sometimes cause
>> the
>>>>>> request
>>>>>>> to get flushed to the broker. In this case, we don't need client
>>>>> handling
>>>>>>> because the producer we are responding to is gone. We just needed
>>> to
>>>>> make
>>>>>>> sure we didn't write to the log on the broker side. I'm trying to
>>>> think
>>>>>> of
>>>>>>> a case where we do have the client to return to. I'd think the
>> same
>>>>>> client
>>>>>>> couldn't progress to committing the transaction unless the
>> produce
>>>>>> request
>>>>>>> returned right? Of course, there is the incorrectly written
>> clients
>>>>> case.
>>>>>>> I'll think on this a bit more and let you know if I come up with
>>>>> another
>>>>>>> scenario when we would return to an active client when the
>>>> transaction
>>>>> is
>>>>>>> no longer ongoing.
>>>>>>>
>>>>>>> I was not aware that we checked the result of a send after we
>>> commit
>>>>>>> though. I'll need to look into that a bit more.
>>>>>>>
>>>>>>> 2. There were some questions about this in the discussion. The
>> plan
>>>> is
>>>>> to
>>>>>>> handle overflow with the mechanism we currently have in the
>>> producer.
>>>>> If
>>>>>> we
>>>>>>> try to bump and the epoch will overflow, we actually allocate a
>> new
>>>>>>> producer ID. I need to confirm the fencing logic on the last
>> epoch
>>>> (ie,
>>>>>> we
>>>>>>> probably shouldn't allow any records to be produced with the
>> final
>>>>> epoch
>>>>>>> since we can never properly fence that one).
>>>>>>>
>>>>>>> 3. I can agree with you that the current error handling is
>> messy. I
>>>>>> recall
>>>>>>> taking a look at your KIP a while back, but I think I mostly saw
>>> the
>>>>>>> section about how the errors were wrapped. Maybe I need to take
>>>> another
>>>>>>> look. As for abortable error, the idea was that the handling
>> would
>>> be
>>>>>>> simple -- if this error is seen, the transaction should be
>> aborted
>>> --
>>>>> no
>>>>>>> other logic about previous state or requests necessary. Is your
>>>> concern
>>>>>>> simply about adding new errors? We were hoping to have an error
>>> that
>>>>>> would
>>>>>>> have one meaning and many of the current errors have a history of
>>>>> meaning
>>>>>>> different things on different client versions. That was the main
>>>>>> motivation
>>>>>>> for adding a new error.
>>>>>>>
>>>>>>> 4. This is a good point about record timestamp reordering.
>>> Timestamps
>>>>>> don't
>>>>>>> affect compaction, but they do affect retention deletion. For
>> that,
>>>>> kafka
>>>>>>> considers the largest timestamp in the segment, so I think a
>> small
>>>>> amount
>>>>>>> of reordering (hopefully on the order of milliseconds or even
>>>> seconds)
>>>>>> will
>>>>>>> be ok. We take timestamps from clients so there is already a
>>>>> possibility
>>>>>>> for some drift and non-monotonically increasing timestamps.
>>>>>>>
>>>>>>> 5. Thanks for catching. The error is there, but it's actually
>> that
>>>>> those
>>>>>>> fields should be 4+! Due to how the message generator works, I
>>>> actually
>>>>>>> have to redefine those fields inside the
>>>>> `"AddPartitionsToTxnTransaction`
>>>>>>> block for it to build correctly. I'll fix it to be correct.
>>>>>>>
>>>>>>> 6. Correct -- we will only add the request to purgatory if the
>>> cache
>>>>> has
>>>>>> no
>>>>>>> ongoing transaction. I can change the wording to make that
>> clearer
>>>> that
>>>>>> we
>>>>>>> only place the request in purgatory if we need to contact the
>>>>> transaction
>>>>>>> coordinator.
>>>>>>>
>>>>>>> 7. We did take a look at some of the errors and it was hard to
>> come
>>>> up
>>>>>> with
>>>>>>> a good one. I agree that InvalidTxnStateException is ideal except
>>> for
>>>>> the
>>>>>>> fact that it hasn't been returned on Produce requests before. The
>>>> error
>>>>>>> handling for clients is a bit vague (which is why I opened
>>>> KAFKA-14439
>>>>>>> <https://issues.apache.org/jira/browse/KAFKA-14439>), but the
>>>> decision
>>>>>> we
>>>>>>> made here was to only return errors that have been previously
>>>> returned
>>>>> to
>>>>>>> producers. As for not being fatal, I think part of the theory was
>>>> that
>>>>> in
>>>>>>> many cases, the producer would be disconnected. (See point 1) and
>>>> this
>>>>>>> would just be an error to return from the server. I did plan to
>>> think
>>>>>> about
>>>>>>> other cases, so let me know if you think of any as well!
>>>>>>>
>>>>>>> Lots to say! Let me know if you have further thoughts!
>>>>>>> Justine
>>>>>>>
>>>>>>> On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang <
>>>>>> guozhang.wang.us@gmail.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Hello Justine,
>>>>>>>>
>>>>>>>> Thanks for the great write-up! I made a quick pass through it
>> and
>>>>> here
>>>>>>>> are some thoughts (I have not been able to read through this
>>> thread
>>>>> so
>>>>>>>> pardon me if they have overlapped or subsumed by previous
>>>> comments):
>>>>>>>>
>>>>>>>> First are some meta ones:
>>>>>>>>
>>>>>>>> 1. I think we need to also improve the client's experience once
>>> we
>>>>>>>> have this defence in place. More concretely, say a user's
>>> producer
>>>>>>>> code is like following:
>>>>>>>>
>>>>>>>> future = producer.send();
>>>>>>>> // producer.flush();
>>>>>>>> producer.commitTransaction();
>>>>>>>> future.get();
>>>>>>>>
>>>>>>>> Which resulted in the order of a) produce-request sent by
>>> producer,
>>>>> b)
>>>>>>>> end-txn-request sent by producer, c) end-txn-response sent
>> back,
>>> d)
>>>>>>>> txn-marker-request sent from coordinator to partition leader,
>> e)
>>>>>>>> produce-request finally received by the partition leader,
>> before
>>>> this
>>>>>>>> KIP e) step would be accepted causing a dangling txn; now it
>>> would
>>>> be
>>>>>>>> rejected in step e) which is good. But from the client's point
>> of
>>>>> view
>>>>>>>> now it becomes confusing since the `commitTransaction()`
>> returns
>>>>>>>> successfully, but the "future" throws an invalid-epoch error,
>> and
>>>>> they
>>>>>>>> are not sure if the transaction did succeed or not. In fact, it
>>>>>>>> "partially succeeded" with some msgs being rejected but others
>>>>>>>> committed successfully.
>>>>>>>>
>>>>>>>> Of course the easy way to avoid this is, always call
>>>>>>>> "producer.flush()" before commitTxn and that's what we do
>>>> ourselves,
>>>>>>>> and what we recommend users do. But I suspect not everyone does
>>> it.
>>>>> In
>>>>>>>> fact I just checked the javadoc in KafkaProducer and our code
>>>> snippet
>>>>>>>> does not include a `flush()` call. So I'm thinking maybe we can
>>> in
>>>>>>>> side the `commitTxn` code to enforce flushing before sending
>> the
>>>>>>>> end-txn request.
>>>>>>>>
>>>>>>>> 2. I'd like to clarify a bit details on "just add partitions to
>>> the
>>>>>>>> transaction on the first produce request during a transaction".
>>> My
>>>>>>>> understanding is that the partition leader's cache has the
>>> producer
>>>>> id
>>>>>>>> / sequence / epoch for the latest txn, either on-going or is
>>>>> completed
>>>>>>>> (upon receiving the marker request from coordinator). When a
>>>> produce
>>>>>>>> request is received, if
>>>>>>>>
>>>>>>>> * producer's epoch < cached epoch, or producer's epoch ==
>> cached
>>>>> epoch
>>>>>>>> but the latest txn is completed, leader directly reject with
>>>>>>>> invalid-epoch.
>>>>>>>> * producer's epoch > cached epoch, park the the request and
>> send
>>>>>>>> add-partitions request to coordinator.
>>>>>>>>
>>>>>>>> In order to do it, does the coordinator need to bump the
>> sequence
>>>> and
>>>>>>>> reset epoch to 0 when the next epoch is going to overflow? If
>> no
>>>> need
>>>>>>>> to do so, then how we handle the (admittedly rare, but still
>> may
>>>>>>>> happen) epoch overflow situation?
>>>>>>>>
>>>>>>>> 3. I'm a bit concerned about adding a generic "ABORTABLE_ERROR"
>>>> given
>>>>>>>> we already have a pretty messy error classification and error
>>>>> handling
>>>>>>>> on the producer clients side --- I have a summary about the
>>> issues
>>>>> and
>>>>>>>> a proposal to address this in
>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
>>>>>>>> -- I understand we do not want to use "UNKNOWN_PRODUCER_ID"
>>> anymore
>>>>>>>> and in fact we intend to deprecate it in KIP-360 and eventually
>>>>> remove
>>>>>>>> it; but I'm wondering can we still use specific error codes.
>> E.g.
>>>>> what
>>>>>>>> about "InvalidProducerEpochException" since for new clients,
>> the
>>>>>>>> actual reason this would actually be rejected is indeed because
>>> the
>>>>>>>> epoch on the coordinator caused the add-partitions-request from
>>> the
>>>>>>>> brokers to be rejected anyways?
>>>>>>>>
>>>>>>>> 4. It seems we put the producer request into purgatory before
>> we
>>>> ever
>>>>>>>> append the records, while other producer's records may still be
>>>>>>>> appended during the time; and that potentially may result in
>> some
>>>>>>>> re-ordering compared with reception order. I'm not super
>>> concerned
>>>>>>>> about it since Kafka does not guarantee reception ordering
>> across
>>>>>>>> producers anyways, but it may make the timestamps of records
>>>> inside a
>>>>>>>> partition to be more out-of-ordered. Are we aware of any
>>> scenarios
>>>>>>>> such as future enhancements on log compactions that may be
>>> affected
>>>>> by
>>>>>>>> this effect?
>>>>>>>>
>>>>>>>> Below are just minor comments:
>>>>>>>>
>>>>>>>> 5. In "AddPartitionsToTxnTransaction" field of
>>>>>>>> "AddPartitionsToTxnRequest" RPC, the versions of those inner
>>> fields
>>>>>>>> are "0-3" while I thought they should be "0+" still?
>>>>>>>>
>>>>>>>> 6. Regarding "we can place the request in a purgatory of sorts
>>> and
>>>>>>>> check if there is any state for the transaction on the
>> broker": i
>>>>>>>> think at this time when we just do the checks against the
>> cached
>>>>>>>> state, we do not need to put the request to purgatory yet?
>>>>>>>>
>>>>>>>> 7. This is related to 3) above. I feel using
>>>> "InvalidRecordException"
>>>>>>>> for older clients may also be a bit confusing, and also it is
>> not
>>>>>>>> fatal -- for old clients, it better to be fatal since this
>>>> indicates
>>>>>>>> the clients is doing something wrong and hence it should be
>>> closed.
>>>>>>>> And in general I'd prefer to use slightly more specific meaning
>>>> error
>>>>>>>> codes for clients. That being said, I also feel
>>>>>>>> "InvalidProducerEpochException" is not suitable for old
>> versioned
>>>>>>>> clients, and we'd have to pick one that old clients recognize.
>>> I'd
>>>>>>>> prefer "InvalidTxnStateException" but that one is supposed to
>> be
>>>>>>>> returned from txn coordinators only today. I'd suggest we do a
>>>> quick
>>>>>>>> check in the current client's code path and see if that one
>> would
>>>> be
>>>>>>>> handled if it's from a produce-response, and if yes, use this
>>> one;
>>>>>>>> otherwise, use "ProducerFencedException" which is much less
>>>>> meaningful
>>>>>>>> but it's still a fatal error.
>>>>>>>>
>>>>>>>>
>>>>>>>> Thanks,
>>>>>>>> Guozhang
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
>>>>>>>> <jo...@confluent.io.invalid> wrote:
>>>>>>>>>
>>>>>>>>> Yeah -- looks like we already have code to handle bumping the
>>>> epoch
>>>>>> and
>>>>>>>>> when the epoch is Short.MAX_VALUE, we get a new producer ID.
>>>> Since
>>>>>> this
>>>>>>>> is
>>>>>>>>> already the behavior, do we want to change it further?
>>>>>>>>>
>>>>>>>>> Justine
>>>>>>>>>
>>>>>>>>> On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <
>>>>> jolshan@confluent.io
>>>>>>>
>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Hey all, just wanted to quickly update and say I've
>> modified
>>>> the
>>>>>> KIP to
>>>>>>>>>> explicitly mention that AddOffsetCommitsToTxnRequest will
>> be
>>>>>> replaced
>>>>>>>> by
>>>>>>>>>> a coordinator-side (inter-broker) AddPartitionsToTxn
>> implicit
>>>>>> request.
>>>>>>>> This
>>>>>>>>>> mirrors the user partitions and will implicitly add offset
>>>>>> partitions
>>>>>>>> to
>>>>>>>>>> transactions when we commit offsets on them. We will
>>> deprecate
>>>>>>>> AddOffsetCommitsToTxnRequest
>>>>>>>>>> for new clients.
>>>>>>>>>>
>>>>>>>>>> Also to address Artem's comments --
>>>>>>>>>> I'm a bit unsure if the changes here will change the
>> previous
>>>>>> behavior
>>>>>>>> for
>>>>>>>>>> fencing producers. In the case you mention in the first
>>>>> paragraph,
>>>>>> are
>>>>>>>> you
>>>>>>>>>> saying we bump the epoch before we try to abort the
>>>> transaction?
>>>>> I
>>>>>>>> think I
>>>>>>>>>> need to understand the scenarios you mention a bit better.
>>>>>>>>>>
>>>>>>>>>> As for the second part -- I think it makes sense to have
>> some
>>>>> sort
>>>>>> of
>>>>>>>>>> "sentinel" epoch to signal epoch is about to overflow (I
>>> think
>>>> we
>>>>>> sort
>>>>>>>> of
>>>>>>>>>> have this value in place in some ways) so we can codify it
>> in
>>>> the
>>>>>> KIP.
>>>>>>>> I'll
>>>>>>>>>> look into that and try to update soon.
>>>>>>>>>>
>>>>>>>>>> Thanks,
>>>>>>>>>> Justine.
>>>>>>>>>>
>>>>>>>>>> On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
>>>>>>>>>> <al...@confluent.io.invalid> wrote:
>>>>>>>>>>
>>>>>>>>>>> It's good to know that KIP-588 addressed some of the
>> issues.
>>>>>> Looking
>>>>>>>> at
>>>>>>>>>>> the code, it still looks like there are some cases that
>>> would
>>>>>> result
>>>>>>>> in
>>>>>>>>>>> fatal error, e.g. PRODUCER_FENCED is issued by the
>>> transaction
>>>>>>>> coordinator
>>>>>>>>>>> if epoch doesn't match, and the client treats it as a
>> fatal
>>>>> error
>>>>>>>> (code in
>>>>>>>>>>> TransactionManager request handling).  If we consider, for
>>>>>> example,
>>>>>>>>>>> committing a transaction that returns a timeout, but
>>> actually
>>>>>>>> succeeds,
>>>>>>>>>>> trying to abort it or re-commit may result in
>>> PRODUCER_FENCED
>>>>>> error
>>>>>>>>>>> (because of epoch bump).
>>>>>>>>>>>
>>>>>>>>>>> For failed commits, specifically, we need to know the
>> actual
>>>>>> outcome,
>>>>>>>>>>> because if we return an error the application may think
>> that
>>>> the
>>>>>>>>>>> transaction is aborted and redo the work, leading to
>>>> duplicates.
>>>>>>>>>>>
>>>>>>>>>>> Re: overflowing epoch.  We could either do it on the TC
>> and
>>>>> return
>>>>>>>> both
>>>>>>>>>>> producer id and epoch (e.g. change the protocol), or
>> signal
>>>> the
>>>>>> client
>>>>>>>>>>> that
>>>>>>>>>>> it needs to get a new producer id.  Checking for max epoch
>>>> could
>>>>>> be a
>>>>>>>>>>> reasonable signal, the value to check should probably be
>>>> present
>>>>>> in
>>>>>>>> the
>>>>>>>>>>> KIP
>>>>>>>>>>> as this is effectively a part of the contract.  Also, the
>> TC
>>>>>> should
>>>>>>>>>>> probably return an error if the client didn't change
>>> producer
>>>> id
>>>>>> after
>>>>>>>>>>> hitting max epoch.
>>>>>>>>>>>
>>>>>>>>>>> -Artem
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
>>>>>>>>>>> <jo...@confluent.io.invalid> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Thanks for the discussion Artem.
>>>>>>>>>>>>
>>>>>>>>>>>> With respect to the handling of fenced producers, we
>> have
>>>> some
>>>>>>>> behavior
>>>>>>>>>>>> already in place. As of KIP-588:
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
>>>>>>>>>>>> ,
>>>>>>>>>>>> we handle timeouts more gracefully. The producer can
>>>> recover.
>>>>>>>>>>>>
>>>>>>>>>>>> Produce requests can also recover from epoch fencing by
>>>>>> aborting the
>>>>>>>>>>>> transaction and starting over.
>>>>>>>>>>>>
>>>>>>>>>>>> What other cases were you considering that would cause
>> us
>>> to
>>>>>> have a
>>>>>>>>>>> fenced
>>>>>>>>>>>> epoch but we'd want to recover?
>>>>>>>>>>>>
>>>>>>>>>>>> The first point about handling epoch overflows is fair.
>> I
>>>>> think
>>>>>>>> there is
>>>>>>>>>>>> some logic we'd need to consider. (ie, if we are one
>> away
>>>> from
>>>>>> the
>>>>>>>> max
>>>>>>>>>>>> epoch, we need to reset the producer ID.) I'm still
>>>> wondering
>>>>> if
>>>>>>>> there
>>>>>>>>>>> is a
>>>>>>>>>>>> way to direct this from the response, or if everything
>>>> should
>>>>> be
>>>>>>>> done on
>>>>>>>>>>>> the client side. Let me know if you have any thoughts
>>> here.
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks,
>>>>>>>>>>>> Justine
>>>>>>>>>>>>
>>>>>>>>>>>> On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
>>>>>>>>>>>> <al...@confluent.io.invalid> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> There are some workflows in the client that are
>> implied
>>> by
>>>>>>>> protocol
>>>>>>>>>>>>> changes, e.g.:
>>>>>>>>>>>>>
>>>>>>>>>>>>> - for new clients, epoch changes with every
>> transaction
>>>> and
>>>>>> can
>>>>>>>>>>> overflow,
>>>>>>>>>>>>> in old clients this condition was handled
>> transparently,
>>>>>> because
>>>>>>>> epoch
>>>>>>>>>>>> was
>>>>>>>>>>>>> bumped in InitProducerId and it would return a new
>>>> producer
>>>>>> id if
>>>>>>>>>>> epoch
>>>>>>>>>>>>> overflows, the new clients would need to implement
>> some
>>>>>> workflow
>>>>>>>> to
>>>>>>>>>>>> refresh
>>>>>>>>>>>>> producer id
>>>>>>>>>>>>> - how to handle fenced producers, for new clients
>> epoch
>>>>>> changes
>>>>>>>> with
>>>>>>>>>>>> every
>>>>>>>>>>>>> transaction, so in presence of failures during
>> commits /
>>>>>> aborts,
>>>>>>>> the
>>>>>>>>>>>>> producer could get easily fenced, old clients would
>>> pretty
>>>>>> much
>>>>>>>> would
>>>>>>>>>>> get
>>>>>>>>>>>>> fenced when a new incarnation of the producer was
>>>>> initialized
>>>>>> with
>>>>>>>>>>>>> InitProducerId so it's ok to treat as a fatal error,
>> the
>>>> new
>>>>>>>> clients
>>>>>>>>>>>> would
>>>>>>>>>>>>> need to implement some workflow to handle that error,
>>>>>> otherwise
>>>>>>>> they
>>>>>>>>>>>> could
>>>>>>>>>>>>> get fenced by themselves
>>>>>>>>>>>>> - in particular (as a subset of the previous issue),
>>> what
>>>>>> would
>>>>>>>> the
>>>>>>>>>>>> client
>>>>>>>>>>>>> do if it got a timeout during commit?  commit could've
>>>>>> succeeded
>>>>>>>> or
>>>>>>>>>>>> failed
>>>>>>>>>>>>>
>>>>>>>>>>>>> Not sure if this has to be defined in the KIP as
>>>>> implementing
>>>>>>>> those
>>>>>>>>>>>>> probably wouldn't require protocol changes, but we
>> have
>>>>>> multiple
>>>>>>>>>>>>> implementations of Kafka clients, so probably would be
>>>> good
>>>>> to
>>>>>>>> have
>>>>>>>>>>> some
>>>>>>>>>>>>> client implementation guidance.  Could also be done
>> as a
>>>>>> separate
>>>>>>>> doc.
>>>>>>>>>>>>>
>>>>>>>>>>>>> -Artem
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
>>>>>>>>>>>> <jolshan@confluent.io.invalid
>>>>>>>>>>>>>>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hey all, I've updated the KIP to incorporate Jason's
>>>>>>>> suggestions.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 1. Use AddPartitionsToTxn + verify flag to check on
>>> old
>>>>>> clients
>>>>>>>>>>>>>> 2. Updated AddPartitionsToTxn API to support
>>> transaction
>>>>>>>> batching
>>>>>>>>>>>>>> 3. Mention IBP bump
>>>>>>>>>>>>>> 4. Mention auth change on new AddPartitionsToTxn
>>>> version.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I'm planning on opening a vote soon.
>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <
>>>>>>>> jolshan@confluent.io
>>>>>>>>>>>>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks Jason. Those changes make sense to me. I
>> will
>>>>>> update
>>>>>>>> the
>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hey Justine,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I was wondering about compatibility here. When
>> we
>>>>> send
>>>>>>>> requests
>>>>>>>>>>>>>>>> between brokers, we want to ensure that the
>>> receiving
>>>>>> broker
>>>>>>>>>>>>> understands
>>>>>>>>>>>>>>>> the request (specifically the new fields).
>>> Typically
>>>>>> this is
>>>>>>>> done
>>>>>>>>>>>> via
>>>>>>>>>>>>>>>> IBP/metadata version.
>>>>>>>>>>>>>>>> I'm trying to think if there is a way around it
>> but
>>>> I'm
>>>>>> not
>>>>>>>> sure
>>>>>>>>>>>> there
>>>>>>>>>>>>>> is.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Yes. I think we would gate usage of this behind
>> an
>>>> IBP
>>>>>> bump.
>>>>>>>> Does
>>>>>>>>>>>> that
>>>>>>>>>>>>>>>> seem
>>>>>>>>>>>>>>>> reasonable?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> As for the improvements -- can you clarify how
>>> the
>>>>>> multiple
>>>>>>>>>>>>>>>> transactional
>>>>>>>>>>>>>>>> IDs would help here? Were you thinking of a case
>>>> where
>>>>> we
>>>>>>>>>>> wait/batch
>>>>>>>>>>>>>>>> multiple produce requests together? My
>>> understanding
>>>>> for
>>>>>> now
>>>>>>>> was
>>>>>>>>>>> 1
>>>>>>>>>>>>>>>> transactional ID and one validation per 1 produce
>>>>>> request.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Each call to `AddPartitionsToTxn` is essentially
>> a
>>>>> write
>>>>>> to
>>>>>>>> the
>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>> log and must block on replication. The more we
>> can
>>>> fit
>>>>>> into a
>>>>>>>>>>> single
>>>>>>>>>>>>>>>> request, the more writes we can do in parallel.
>> The
>>>>>>>> alternative
>>>>>>>>>>> is
>>>>>>>>>>>> to
>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>> use of more connections, but usually we prefer
>>>> batching
>>>>>>>> since the
>>>>>>>>>>>>>> network
>>>>>>>>>>>>>>>> stack is not really optimized for high
>>>>> connection/request
>>>>>>>> loads.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Finally with respect to the authorizations, I
>>> think
>>>>> it
>>>>>>>> makes
>>>>>>>>>>> sense
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> skip
>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit confused by
>> the
>>>>>> "leader
>>>>>>>> ID"
>>>>>>>>>>>> field.
>>>>>>>>>>>>>>>> Wouldn't we just want to flag the request as
>> from a
>>>>>> broker
>>>>>>>> (does
>>>>>>>>>>> it
>>>>>>>>>>>>>> matter
>>>>>>>>>>>>>>>> which one?).
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> We could also make it version-based. For the next
>>>>>> version, we
>>>>>>>>>>> could
>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>> CLUSTER auth. So clients would not be able to use
>>> the
>>>>> API
>>>>>>>>>>> anymore,
>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>> probably what we want.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> -Jason
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> As a follow up, I was just thinking about the
>>>>> batching
>>>>>> a
>>>>>>>> bit
>>>>>>>>>>> more.
>>>>>>>>>>>>>>>>> I suppose if we have one request in flight and
>> we
>>>>>> queue up
>>>>>>>> the
>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>> produce requests in some sort of purgatory, we
>>>> could
>>>>>> send
>>>>>>>>>>>>> information
>>>>>>>>>>>>>>>> out
>>>>>>>>>>>>>>>>> for all of them rather than one by one. So that
>>>> would
>>>>>> be a
>>>>>>>>>>> benefit
>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>> batching partitions to add per transaction.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I'll need to think a bit more on the design of
>>> this
>>>>>> part
>>>>>>>> of the
>>>>>>>>>>>> KIP,
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> will update the KIP in the next few days.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan
>> <
>>>>>>>>>>>>> jolshan@confluent.io>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hey Jason -- thanks for the input -- I was
>> just
>>>>>> digging
>>>>>>>> a bit
>>>>>>>>>>>>> deeper
>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>> the design + implementation of the validation
>>>> calls
>>>>>> here
>>>>>>>> and
>>>>>>>>>>>> what
>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>> say
>>>>>>>>>>>>>>>>>> makes sense.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I was wondering about compatibility here.
>> When
>>> we
>>>>>> send
>>>>>>>>>>> requests
>>>>>>>>>>>>>>>>>> between brokers, we want to ensure that the
>>>>> receiving
>>>>>>>> broker
>>>>>>>>>>>>>>>> understands
>>>>>>>>>>>>>>>>>> the request (specifically the new fields).
>>>>> Typically
>>>>>>>> this is
>>>>>>>>>>>> done
>>>>>>>>>>>>>> via
>>>>>>>>>>>>>>>>>> IBP/metadata version.
>>>>>>>>>>>>>>>>>> I'm trying to think if there is a way around
>> it
>>>> but
>>>>>> I'm
>>>>>>>> not
>>>>>>>>>>> sure
>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>> is.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> As for the improvements -- can you clarify
>> how
>>>> the
>>>>>>>> multiple
>>>>>>>>>>>>>>>> transactional
>>>>>>>>>>>>>>>>>> IDs would help here? Were you thinking of a
>>> case
>>>>>> where we
>>>>>>>>>>>>> wait/batch
>>>>>>>>>>>>>>>>>> multiple produce requests together? My
>>>>> understanding
>>>>>> for
>>>>>>>> now
>>>>>>>>>>>> was 1
>>>>>>>>>>>>>>>>>> transactional ID and one validation per 1
>>> produce
>>>>>>>> request.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Finally with respect to the authorizations, I
>>>> think
>>>>>> it
>>>>>>>> makes
>>>>>>>>>>>> sense
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> skip
>>>>>>>>>>>>>>>>>> topic authorizations, but I'm a bit confused
>> by
>>>> the
>>>>>>>> "leader
>>>>>>>>>>> ID"
>>>>>>>>>>>>>> field.
>>>>>>>>>>>>>>>>>> Wouldn't we just want to flag the request as
>>>> from a
>>>>>>>> broker
>>>>>>>>>>> (does
>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>> matter
>>>>>>>>>>>>>>>>>> which one?).
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I think I want to adopt these suggestions,
>> just
>>>> had
>>>>>> a few
>>>>>>>>>>>>> questions
>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> details.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Thu, Jan 5, 2023 at 5:05 PM Jason
>> Gustafson
>>>>>>>>>>>>>>>>> <ja...@confluent.io.invalid>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Hi Justine,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks for the proposal.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I was thinking about the implementation a
>>> little
>>>>>> bit.
>>>>>>>> In the
>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>> proposal, the behavior depends on whether we
>>>> have
>>>>> an
>>>>>>>> old or
>>>>>>>>>>> new
>>>>>>>>>>>>>>>> client.
>>>>>>>>>>>>>>>>>>> For
>>>>>>>>>>>>>>>>>>> old clients, we send `DescribeTransactions`
>>> and
>>>>>> verify
>>>>>>>> the
>>>>>>>>>>>> result
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>> new clients, we send `AddPartitionsToTxn`.
>> We
>>>>> might
>>>>>> be
>>>>>>>> able
>>>>>>>>>>> to
>>>>>>>>>>>>>>>> simplify
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> implementation if we can use the same
>> request
>>>>> type.
>>>>>> For
>>>>>>>>>>>> example,
>>>>>>>>>>>>>>>> what if
>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>> bump the protocol version for
>>>> `AddPartitionsToTxn`
>>>>>> and
>>>>>>>> add a
>>>>>>>>>>>>>>>>>>> `validateOnly`
>>>>>>>>>>>>>>>>>>> flag? For older versions, we can set
>>>>>>>> `validateOnly=true` so
>>>>>>>>>>>> that
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> request only returns successfully if the
>>>> partition
>>>>>> had
>>>>>>>>>>> already
>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>> added.
>>>>>>>>>>>>>>>>>>> For new versions, we can set
>>>> `validateOnly=false`
>>>>>> and
>>>>>>>> the
>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>> added to the transaction. The other slightly
>>>>>> annoying
>>>>>>>> thing
>>>>>>>>>>>> that
>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>> get around is the need to collect the
>>>> transaction
>>>>>> state
>>>>>>>> for
>>>>>>>>>>> all
>>>>>>>>>>>>>>>>> partitions
>>>>>>>>>>>>>>>>>>> even when we only care about a subset.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Some additional improvements to consider:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> - We can give `AddPartitionsToTxn` better
>>> batch
>>>>>> support
>>>>>>>> for
>>>>>>>>>>>>>>>> inter-broker
>>>>>>>>>>>>>>>>>>> usage. Currently we only allow one
>>>>>> `TransactionalId` to
>>>>>>>> be
>>>>>>>>>>>>>> specified,
>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>> the broker may get some benefit being able
>> to
>>>>> batch
>>>>>>>> across
>>>>>>>>>>>>> multiple
>>>>>>>>>>>>>>>>>>> transactions.
>>>>>>>>>>>>>>>>>>> - Another small improvement is skipping
>> topic
>>>>>>>> authorization
>>>>>>>>>>>>> checks
>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>> `AddPartitionsToTxn` when the request is
>> from
>>> a
>>>>>> broker.
>>>>>>>>>>> Perhaps
>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>> add
>>>>>>>>>>>>>>>>>>> a field for the `LeaderId` or something like
>>>> that
>>>>>> and
>>>>>>>>>>> require
>>>>>>>>>>>>>> CLUSTER
>>>>>>>>>>>>>>>>>>> permission when set.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>> Jason
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
>>>>>>>>>>>> <jun@confluent.io.invalid
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Hi, Justine,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks for the explanation. It makes sense
>>> to
>>>> me
>>>>>> now.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Jun
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 1:42 PM Justine
>>> Olshan
>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Hi Jun,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> My understanding of the mechanism is
>> that
>>>> when
>>>>>> we
>>>>>>>> get to
>>>>>>>>>>>> the
>>>>>>>>>>>>>> last
>>>>>>>>>>>>>>>>>>> epoch,
>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>> increment to the fencing/last epoch and
>> if
>>>> any
>>>>>>>> further
>>>>>>>>>>>>> requests
>>>>>>>>>>>>>>>> come
>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>> this producer ID they are fenced. Then
>> the
>>>>>> producer
>>>>>>>>>>> gets a
>>>>>>>>>>>>> new
>>>>>>>>>>>>>> ID
>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> restarts with epoch/sequence 0. The
>> fenced
>>>>> epoch
>>>>>>>> sticks
>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> duration of producer.id.expiration.ms
>> and
>>>>>> blocks
>>>>>>>> any
>>>>>>>>>>> late
>>>>>>>>>>>>>>>> messages
>>>>>>>>>>>>>>>>>>>> there.
>>>>>>>>>>>>>>>>>>>>> The new ID will get to take advantage of
>>> the
>>>>>>>> improved
>>>>>>>>>>>>> semantics
>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>> non-zero start sequences. So I think we
>>> are
>>>>>> covered.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> The only potential issue is overloading
>>> the
>>>>>> cache,
>>>>>>>> but
>>>>>>>>>>>>>> hopefully
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> improvements (lowered
>>>>> producer.id.expiration.ms
>>>>>> )
>>>>>>>> will
>>>>>>>>>>> help
>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>> that.
>>>>>>>>>>>>>>>>>>>> Let
>>>>>>>>>>>>>>>>>>>>> me know if you still have concerns.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
>>>>>>>>>>>>>>>> <ju...@confluent.io.invalid>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks for the explanation.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 70. The proposed fencing logic doesn't
>>>> apply
>>>>>> when
>>>>>>>> pid
>>>>>>>>>>>>>> changes,
>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> right? If so, I am not sure how
>> complete
>>>> we
>>>>>> are
>>>>>>>>>>>> addressing
>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>> issue
>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>> the pid changes more frequently.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Jun
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 16, 2022 at 9:16 AM
>> Justine
>>>>> Olshan
>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Thanks for replying!
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 70.We already do the overflow
>>> mechanism,
>>>>> so
>>>>>> my
>>>>>>>>>>> change
>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>> happen more often.
>>>>>>>>>>>>>>>>>>>>>>> I was also not suggesting a new
>> field
>>> in
>>>>> the
>>>>>>>> log,
>>>>>>>>>>> but
>>>>>>>>>>>> in
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> response,
>>>>>>>>>>>>>>>>>>>>>>> which would be gated by the client
>>>>> version.
>>>>>>>> Sorry if
>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>> unclear. I think we are starting to
>>>>> diverge.
>>>>>>>>>>>>>>>>>>>>>>> The goal of this KIP is to not
>> change
>>> to
>>>>> the
>>>>>>>> marker
>>>>>>>>>>>>> format
>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 71. Yes, I guess I was going under
>> the
>>>>>>>> assumption
>>>>>>>>>>> that
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> log
>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>> look at its last epoch and treat it
>> as
>>>> the
>>>>>>>> current
>>>>>>>>>>>>> epoch. I
>>>>>>>>>>>>>>>>>>> suppose
>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>> have some special logic that if the
>>> last
>>>>>> epoch
>>>>>>>> was
>>>>>>>>>>> on a
>>>>>>>>>>>>>>>> marker
>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>> expect the next epoch or something
>>> like
>>>>>> that. We
>>>>>>>>>>> just
>>>>>>>>>>>>> need
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> distinguish
>>>>>>>>>>>>>>>>>>>>>>> based on whether we had a
>> commit/abort
>>>>>> marker.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 72.
>>>>>>>>>>>>>>>>>>>>>>>> if the producer epoch hasn't been
>>>> bumped
>>>>>> on
>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
>>>> message
>>>>>> will
>>>>>>>> fail
>>>>>>>>>>> the
>>>>>>>>>>>>>>>> sequence
>>>>>>>>>>>>>>>>>>>>>> validation
>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the producer
>>>> epoch
>>>>>> has
>>>>>>>> been
>>>>>>>>>>>>> bumped,
>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>> ignore
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck message
>>>> could
>>>>>> be
>>>>>>>>>>> appended
>>>>>>>>>>>> to
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> log.
>>>>>>>>>>>>>>>>>>>> So,
>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
>> guard?
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I follow that "the
>>> message
>>>>> will
>>>>>>>> fail
>>>>>>>>>>> the
>>>>>>>>>>>>>>>> sequence
>>>>>>>>>>>>>>>>>>>>>> validation".
>>>>>>>>>>>>>>>>>>>>>>> In some of these cases, we had an
>>> abort
>>>>>> marker
>>>>>>>> (due
>>>>>>>>>>> to
>>>>>>>>>>>> an
>>>>>>>>>>>>>>>> error)
>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> then
>>>>>>>>>>>>>>>>>>>>>>> the late message comes in with the
>>>> correct
>>>>>>>> sequence
>>>>>>>>>>>>> number.
>>>>>>>>>>>>>>>> This
>>>>>>>>>>>>>>>>>>> is a
>>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>> covered by the KIP.
>>>>>>>>>>>>>>>>>>>>>>> The latter case is actually not
>>>> something
>>>>>> we've
>>>>>>>>>>>>> considered
>>>>>>>>>>>>>>>>> here. I
>>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>> generally when we bump the epoch, we
>>> are
>>>>>>>> accepting
>>>>>>>>>>> that
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> sequence
>>>>>>>>>>>>>>>>>>>>> does
>>>>>>>>>>>>>>>>>>>>>>> not need to be checked anymore. My
>>>>>>>> understanding is
>>>>>>>>>>>> also
>>>>>>>>>>>>>>>> that we
>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>> typically bump epoch mid transaction
>>>>> (based
>>>>>> on a
>>>>>>>>>>> quick
>>>>>>>>>>>>> look
>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> code)
>>>>>>>>>>>>>>>>>>>>>>> but let me know if that is the case.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 15, 2022 at 12:23 PM Jun
>>> Rao
>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the reply.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 70. Assigning a new pid on int
>>>> overflow
>>>>>> seems
>>>>>>>> a
>>>>>>>>>>> bit
>>>>>>>>>>>>>> hacky.
>>>>>>>>>>>>>>>> If
>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>> need a
>>>>>>>>>>>>>>>>>>>>>>> txn
>>>>>>>>>>>>>>>>>>>>>>>> level id, it will be better to
>> model
>>>>> this
>>>>>>>>>>> explicitly.
>>>>>>>>>>>>>>>> Adding a
>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>> field
>>>>>>>>>>>>>>>>>>>>>>>> would require a bit more work
>> since
>>> it
>>>>>>>> requires a
>>>>>>>>>>> new
>>>>>>>>>>>>> txn
>>>>>>>>>>>>>>>>> marker
>>>>>>>>>>>>>>>>>>>>> format
>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>> the log. So, we probably need to
>>> guard
>>>>> it
>>>>>>>> with an
>>>>>>>>>>> IBP
>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>> metadata
>>>>>>>>>>>>>>>>>>>>>> version
>>>>>>>>>>>>>>>>>>>>>>>> and document the impact on
>> downgrade
>>>>> once
>>>>>> the
>>>>>>>> new
>>>>>>>>>>>>> format
>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>> written
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> log.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 71. Hmm, once the marker is
>> written,
>>>> the
>>>>>>>> partition
>>>>>>>>>>>> will
>>>>>>>>>>>>>>>> expect
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>> append to be on the next epoch.
>> Does
>>>>> that
>>>>>>>> cover
>>>>>>>>>>> the
>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>> mentioned?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 72. Also, just to be clear on the
>>>>> stucked
>>>>>>>> message
>>>>>>>>>>>> issue
>>>>>>>>>>>>>>>>>>> described
>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> motivation. With EoS, we also
>>> validate
>>>>> the
>>>>>>>>>>> sequence
>>>>>>>>>>>> id
>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>> idempotency.
>>>>>>>>>>>>>>>>>>>>>>> So,
>>>>>>>>>>>>>>>>>>>>>>>> with the current logic, if the
>>>> producer
>>>>>> epoch
>>>>>>>>>>> hasn't
>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>> bumped on
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> broker, it seems that the stucked
>>>>> message
>>>>>> will
>>>>>>>>>>> fail
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> sequence
>>>>>>>>>>>>>>>>>>>>>>> validation
>>>>>>>>>>>>>>>>>>>>>>>> and will be ignored. If the
>> producer
>>>>>> epoch has
>>>>>>>>>>> been
>>>>>>>>>>>>>>>> bumped, we
>>>>>>>>>>>>>>>>>>>> ignore
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> sequence check and the stuck
>> message
>>>>>> could be
>>>>>>>>>>>> appended
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> log.
>>>>>>>>>>>>>>>>>>>>> So,
>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>> the latter case that we want to
>>> guard?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Jun
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 10:44 AM
>>>> Justine
>>>>>>>> Olshan
>>>>>>>>>>>>>>>>>>>>>>>> <jo...@confluent.io.invalid>
>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Matthias — thanks again for
>> taking
>>>>> time
>>>>>> to
>>>>>>>> look
>>>>>>>>>>> a
>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>> You
>>>>>>>>>>>>>>>>>>> said:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> My proposal was only focusing
>> to
>>>>> avoid
>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
>> added
>>>>>> without
>>>>>>>>>>>> registered
>>>>>>>>>>>>>>>>>>> partition.
>>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>>>>>> Maybe
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more details
>> to
>>>> the
>>>>>> KIP
>>>>>>>> about
>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>> scenario
>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure I understand what
>> you
>>>>> mean
>>>>>>>> here.
>>>>>>>>>>> The
>>>>>>>>>>>>>>>> motivation
>>>>>>>>>>>>>>>>>>>>> section
>>>>>>>>>>>>>>>>>>>>>>>>> describes two scenarios about
>> how
>>>> the
>>>>>> record
>>>>>>>>>>> can be
>>>>>>>>>>>>>> added
>>>>>>>>>>>>>>>>>>>> without a
>>>>>>>>>>>>>>>>>>>>>>>>> registered partition:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Another way hanging
>> transactions
>>>> can
>>>>>>>> occur is
>>>>>>>>>>>> that
>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>> client
>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>> buggy
>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>> may somehow try to write to a
>>>>> partition
>>>>>>>> before
>>>>>>>>>>> it
>>>>>>>>>>>>> adds
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> transaction.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> For the first example of this
>>> would
>>>> it
>>>>>> be
>>>>>>>>>>> helpful
>>>>>>>>>>>> to
>>>>>>>>>>>>>> say
>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>> comes in after the abort, but
>>> before
>>>>> the
>>>>>>>>>>> partition
>>>>>>>>>>>> is
>>>>>>>>>>>>>>>> added
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>> transaction so it becomes
>>> "hanging."
>>>>>>>> Perhaps the
>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>> sentence
>>>>>>>>>>>>>>>>>>>>>>> describing
>>>>>>>>>>>>>>>>>>>>>>>>> the message becoming part of the
>>>> next
>>>>>>>>>>> transaction
>>>>>>>>>>>> (a
>>>>>>>>>>>>>>>>> different
>>>>>>>>>>>>>>>>>>>>> case)
>>>>>>>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>>>>>> not properly differentiated.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Jun — thanks for reading the
>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> 70. The int typing was a
>> concern.
>>>>>> Currently
>>>>>>>> we
>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>> mechanism
>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>> place
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> fence the final epoch when the
>>> epoch
>>>>> is
>>>>>>>> about to
>>>>>>>>>>>>>> overflow
>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> assign
>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>> producer ID with epoch 0. Of
>>> course,
>>>>>> this
>>>>>>>> is a
>>>>>>>>>>> bit
>>>>>>>>>>>>>> tricky
>>>>>>>>>>>>>>>>>>> when it
>>>>>>>>>>>>>>>>>>>>>> comes
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> the response back to the client.
>>>>>>>>>>>>>>>>>>>>>>>>> Making this a long could be
>>> another
>>>>>> option,
>>>>>>>> but
>>>>>>>>>>> I
>>>>>>>>>>>>>> wonder
>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>>> implications on changing this
>>> field
>>>> if
>>>>>> the
>>>>>>>>>>> epoch is
>>>>>>>>>>>>>>>>> persisted
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> disk?
>>>>>>>>>>>>>>>>>>>>>>>> I'd
>>>>>>>>>>>>>>>>>>>>>>>>> need to check the usages.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> 71.This was something Matthias
>>> asked
>>>>>> about
>>>>>>>> as
>>>>>>>>>>>> well. I
>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>>> considering a
>>>>>>>>>>>>>>>>>>>>>>>>> possible edge case where a
>> produce
>>>>>> request
>>>>>>>> from
>>>>>>>>>>> a
>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>> somehow
>>>>>>>>>>>>>>>>>>>>>>>>> gets sent right after the marker
>>> is
>>>>>>>> written, but
>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>> alerted of the newly bumped
>> epoch.
>>>> In
>>>>>> this
>>>>>>>>>>> case, we
>>>>>>>>>>>>> may
>>>>>>>>>>>>>>>>>>> include
>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>>>>>>>>> when we don't want to. I suppose
>>> we
>>>>>> could
>>>>>>>> try
>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>> client
>>>>>>>>>>>>>>>>>>>>>>> side
>>>>>>>>>>>>>>>>>>>>>>>>> to bump the epoch after sending
>> an
>>>>>> endTxn as
>>>>>>>>>>> well
>>>>>>>>>>>> in
>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>> scenario
>>>>>>>>>>>>>>>>>>>>> —
>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>> wonder how it would work when
>> the
>>>>>> server is
>>>>>>>>>>>> aborting
>>>>>>>>>>>>>>>> based
>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>> server-side
>>>>>>>>>>>>>>>>>>>>>>>>> error. I could also be missing
>>>>>> something and
>>>>>>>>>>> this
>>>>>>>>>>>>>>>> scenario
>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>> possible.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again to everyone reading
>>> and
>>>>>>>> commenting.
>>>>>>>>>>>> Let
>>>>>>>>>>>>> me
>>>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>>> further questions or comments.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Dec 14, 2022 at 9:41 AM
>>> Jun
>>>>> Rao
>>>>>>>>>>>>>>>>>>> <jun@confluent.io.invalid
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Hi, Justine,
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP. A couple
>> of
>>>>>> comments.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 70. Currently, the producer
>>> epoch
>>>> is
>>>>>> an
>>>>>>>> int.
>>>>>>>>>>> I am
>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>> sure
>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>> enough
>>>>>>>>>>>>>>>>>>>>>>>>>> to accommodate all
>> transactions
>>> in
>>>>> the
>>>>>>>>>>> lifetime
>>>>>>>>>>>> of
>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>> producer.
>>>>>>>>>>>>>>>>>>>>>> Should
>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>> change that to a long or add a
>>> new
>>>>>> long
>>>>>>>> field
>>>>>>>>>>>> like
>>>>>>>>>>>>>>>> txnId?
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 71. "it will write the prepare
>>>>> commit
>>>>>>>> message
>>>>>>>>>>>> with
>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>> bumped
>>>>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>> send
>>>>>>>>>>>>>>>>>>>>>>>>>> WriteTxnMarkerRequests with
>> the
>>>>> bumped
>>>>>>>> epoch."
>>>>>>>>>>>> Hmm,
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>> associated
>>>>>>>>>>>>>>>>>>>>>>>>>> with the current txn right?
>> So,
>>> it
>>>>>> seems
>>>>>>>>>>> weird to
>>>>>>>>>>>>>>>> write a
>>>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>> with a bumped epoch. Should we
>>>> only
>>>>>> bump
>>>>>>>> up
>>>>>>>>>>> the
>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>> EndTxnResponse
>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>> rename the field to sth like
>>>>>>>>>>> nextProducerEpoch?
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Jun
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Dec 12, 2022 at 8:54
>> PM
>>>>>> Matthias
>>>>>>>> J.
>>>>>>>>>>> Sax <
>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org>
>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the background.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30: SGTM. My proposal was
>>>> only
>>>>>>>> focusing
>>>>>>>>>>> to
>>>>>>>>>>>>> avoid
>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions if records are
>>>> added
>>>>>>>> without
>>>>>>>>>>>>>> registered
>>>>>>>>>>>>>>>>>>>> partition.
>>>>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
>>>>>>>>>>>>>>>>>>>>>>>>>>> you can add a few more
>> details
>>>> to
>>>>>> the
>>>>>>>> KIP
>>>>>>>>>>> about
>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>> scenario
>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>>>>>> documentation purpose?
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> 40: I think you hit a fair
>>> point
>>>>>> about
>>>>>>>> race
>>>>>>>>>>>>>>>> conditions
>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>> client
>>>>>>>>>>>>>>>>>>>>>>> bugs
>>>>>>>>>>>>>>>>>>>>>>>>>>> (incorrectly not bumping the
>>>>>> epoch). The
>>>>>>>>>>>>>>>>>>> complexity/confusion
>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>> the bumped epoch I see, is
>>>> mainly
>>>>>> for
>>>>>>>>>>> internal
>>>>>>>>>>>>>>>>> debugging,
>>>>>>>>>>>>>>>>>>> ie,
>>>>>>>>>>>>>>>>>>>>>>>>> inspecting
>>>>>>>>>>>>>>>>>>>>>>>>>>> log segment dumps -- it
>> seems
>>>>>> harder to
>>>>>>>>>>> reason
>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> system
>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>> us
>>>>>>>>>>>>>>>>>>>>>>>>>>> humans. But if we get better
>>>>>>>> guarantees, it
>>>>>>>>>>>> would
>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>> worth to
>>>>>>>>>>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped epoch.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> 60: as I mentioned already,
>> I
>>>>> don't
>>>>>>>> know the
>>>>>>>>>>>>> broker
>>>>>>>>>>>>>>>>>>> internals
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> provide
>>>>>>>>>>>>>>>>>>>>>>>>>>> more input. So if nobody
>> else
>>>>> chimes
>>>>>>>> in, we
>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>> move
>>>>>>>>>>>>>>>>>>>>>>> forward
>>>>>>>>>>>>>>>>>>>>>>>>>>> with your proposal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> On 12/6/22 4:22 PM, Justine
>>>> Olshan
>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi all,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> After Artem's questions
>>> about
>>>>>> error
>>>>>>>>>>> behavior,
>>>>>>>>>>>>>> I've
>>>>>>>>>>>>>>>>>>>>> re-evaluated
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> unknown producer ID
>>> exception
>>>>> and
>>>>>> had
>>>>>>>> some
>>>>>>>>>>>>>>>> discussions
>>>>>>>>>>>>>>>>>>>>> offline.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think generally it makes
>>>> sense
>>>>>> to
>>>>>>>>>>> simplify
>>>>>>>>>>>>>> error
>>>>>>>>>>>>>>>>>>> handling
>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>> cases
>>>>>>>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>> this and the
>>>> UNKNOWN_PRODUCER_ID
>>>>>> error
>>>>>>>>>>> has a
>>>>>>>>>>>>>> pretty
>>>>>>>>>>>>>>>>> long
>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>> complicated
>>>>>>>>>>>>>>>>>>>>>>>>>>>> history. Because of this,
>> I
>>>>>> propose
>>>>>>>>>>> adding a
>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>> error
>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>>>>> ABORTABLE_ERROR
>>>>>>>>>>>>>>>>>>>>>>>>>>>> that when encountered by
>> new
>>>>>> clients
>>>>>>>>>>> (gated
>>>>>>>>>>>> by
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> produce
>>>>>>>>>>>>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>>>>>>>>>>>> version)
>>>>>>>>>>>>>>>>>>>>>>>>>>>> will simply abort the
>>>>> transaction.
>>>>>>>> This
>>>>>>>>>>>> allows
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> server
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>> say
>>>>>>>>>>>>>>>>>>>>>>>>>>>> in whether the client
>> aborts
>>>> and
>>>>>> makes
>>>>>>>>>>>> handling
>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>> simpler.
>>>>>>>>>>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> future, we can also use
>> this
>>>>>> error in
>>>>>>>>>>> other
>>>>>>>>>>>>>>>> situations
>>>>>>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>> want
>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> abort the transactions. We
>>> can
>>>>>> even
>>>>>>>> use on
>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>> apis.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've added this to the
>> KIP.
>>>> Let
>>>>> me
>>>>>>>> know if
>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>> questions
>>>>>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 2, 2022 at
>> 10:22
>>>> AM
>>>>>>>> Justine
>>>>>>>>>>>> Olshan
>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>> jolshan@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey Matthias,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20/30 — Maybe I also
>> didn't
>>>>>> express
>>>>>>>>>>> myself
>>>>>>>>>>>>>>>> clearly.
>>>>>>>>>>>>>>>>> For
>>>>>>>>>>>>>>>>>>>>> older
>>>>>>>>>>>>>>>>>>>>>>>>> clients
>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't have a way to
>>>> distinguish
>>>>>>>> between a
>>>>>>>>>>>>>> previous
>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction since we
>> don't
>>>> have
>>>>>> the
>>>>>>>> epoch
>>>>>>>>>>>>> bump.
>>>>>>>>>>>>>>>> This
>>>>>>>>>>>>>>>>>>> means
>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>> late
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message from the previous
>>>>>> transaction
>>>>>>>>>>> may be
>>>>>>>>>>>>>>>> added to
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>> one.
>>>>>>>>>>>>>>>>>>>>>>>>>> With
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older clients — we can't
>>>>>> guarantee
>>>>>>>> this
>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>> happen
>>>>>>>>>>>>>>>>>>> if we
>>>>>>>>>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>>>>>>>>>>>> sent
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call
>>> (why
>>>> we
>>>>>> make
>>>>>>>>>>> changes
>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> newer
>>>>>>>>>>>>>>>>>>>>>>> client)
>>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can at least gate some by
>>>>>> ensuring
>>>>>>>> that
>>>>>>>>>>> the
>>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>> has
>>>>>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>>>>> added
>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. The
>> rationale
>>>> here
>>>>>> is
>>>>>>>> that
>>>>>>>>>>>> there
>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>> likely
>>>>>>>>>>>>>>>>>>>>> LESS
>>>>>>>>>>>>>>>>>>>>>>> late
>>>>>>>>>>>>>>>>>>>>>>>>>>> arrivals
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as time goes on, so
>>> hopefully
>>>>>> most
>>>>>>>> late
>>>>>>>>>>>>> arrivals
>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>> come
>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>> BEFORE
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn call.
>>>> Those
>>>>>> that
>>>>>>>>>>> arrive
>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>> properly
>>>>>>>>>>>>>>>>>>>>>>>>>> gated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>> describeTransactions
>>>>>>>> approach.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we take the approach
>> you
>>>>>>>> suggested,
>>>>>>>>>>> ANY
>>>>>>>>>>>>> late
>>>>>>>>>>>>>>>>> arrival
>>>>>>>>>>>>>>>>>>>>> from a
>>>>>>>>>>>>>>>>>>>>>>>>>> previous
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction will be
>> added.
>>>> And
>>>>> we
>>>>>>>> don't
>>>>>>>>>>> want
>>>>>>>>>>>>>>>> that. I
>>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>> see
>>>>>>>>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> benefit in sending
>>>>>> addPartitionsToTxn
>>>>>>>>>>> over
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> describeTxns
>>>>>>>>>>>>>>>>>>>>>>> call.
>>>>>>>>>>>>>>>>>>>>>>>>> They
>>>>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> both be one extra RPC to
>>> the
>>>>> Txn
>>>>>>>>>>>> coordinator.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To be clear — newer
>> clients
>>>>> will
>>>>>> use
>>>>>>>>>>>>>>>>> addPartitionsToTxn
>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTxns.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that if we
>>> have
>>>>>> some
>>>>>>>> delay
>>>>>>>>>>> in
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> client
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> bump
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it could continue to send
>>>> epoch
>>>>>> 73
>>>>>>>> and
>>>>>>>>>>> those
>>>>>>>>>>>>>>>> records
>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>> fenced.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Perhaps this is not an
>>> issue
>>>> if
>>>>>> we
>>>>>>>> don't
>>>>>>>>>>>> allow
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>> produce
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> go
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through before the EndTxn
>>>>> request
>>>>>>>>>>> returns.
>>>>>>>>>>>> I'm
>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>> cases of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure. I will need to
>>> think
>>>>> on
>>>>>>>> this a
>>>>>>>>>>> bit.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I wasn't sure if it was
>>> that
>>>>>>>> confusing.
>>>>>>>>>>> But
>>>>>>>>>>>> if
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> investigate other ways.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure these are
>> the
>>>> same
>>>>>>>>>>> purgatories
>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>>> is a
>>>>>>>>>>>>>>>>>>>>>>> produce
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory (I was planning
>>> on
>>>>>> using a
>>>>>>>>>>>> callback
>>>>>>>>>>>>>>>> rather
>>>>>>>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>> purgatory)
>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the other is simply a
>>> request
>>>>> to
>>>>>>>> append
>>>>>>>>>>> to
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> log.
>>>>>>>>>>>>>>>>> Not
>>>>>>>>>>>>>>>>>>>> sure
>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> structure here for
>>> ordering,
>>>>> but
>>>>>> my
>>>>>>>>>>>>>> understanding
>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> broker
>>>>>>>>>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handle the write request
>>>> before
>>>>>> it
>>>>>>>> hears
>>>>>>>>>>>> back
>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> Txn
>>>>>>>>>>>>>>>>>>>>>>>>>> Coordinator.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if I
>>>> misunderstood
>>>>>>>> something
>>>>>>>>>>> or
>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>>>> unclear.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 1, 2022 at
>>> 12:15
>>>> PM
>>>>>>>> Matthias
>>>>>>>>>>> J.
>>>>>>>>>>>>> Sax
>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the details
>>>>> Justine!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side change
>>> for
>>>> 2
>>>>> is
>>>>>>>>>>> removing
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> addPartitions
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need to
>>> make
>>>>>> this
>>>>>>>> from
>>>>>>>>>>> the
>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> txn
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think I did not
>> express
>>>>> myself
>>>>>>>>>>> clearly. I
>>>>>>>>>>>>>>>>> understand
>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>> (and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should) change the
>>> producer
>>>> to
>>>>>> not
>>>>>>>> send
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> `addPartitions`
>>>>>>>>>>>>>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> longer. But I don't
>> thinks
>>>>> it's
>>>>>>>>>>> requirement
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> change
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> broker?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What I am trying to say
>>> is:
>>>>> as a
>>>>>>>>>>> safe-guard
>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>> improvement
>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>> older
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producers, the partition
>>>>> leader
>>>>>> can
>>>>>>>> just
>>>>>>>>>>>> send
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> `addPartitions`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request to the
>>>> TX-coordinator
>>>>>> in any
>>>>>>>>>>> case
>>>>>>>>>>>> --
>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> old
>>>>>>>>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> correctly did send the
>>>>>>>> `addPartition`
>>>>>>>>>>>> request
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> already, the
>>> TX-coordinator
>>>>> can
>>>>>> just
>>>>>>>>>>>> "ignore"
>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>> idempotent.
>>>>>>>>>>>>>>>>>>>>>>>>>>> However,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if the old producer has
>> a
>>>> bug
>>>>>> and
>>>>>>>> did
>>>>>>>>>>>> forget
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> sent
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> `addPartition`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request, we would now
>>> ensure
>>>>>> that
>>>>>>>> the
>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>> indeed
>>>>>>>>>>>>>>>>>>>>>> added
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX and thus fix a
>>> potential
>>>>>>>> producer bug
>>>>>>>>>>>>> (even
>>>>>>>>>>>>>>>> if we
>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fencing via the bump
>>> epoch).
>>>>> --
>>>>>> It
>>>>>>>>>>> seems to
>>>>>>>>>>>>> be
>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>>>>> improvement?
>>>>>>>>>>>>>>>>>>>>>>>>> Or
>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there a reason to not do
>>>> this?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is ongoing
>> =
>>>>>> partition
>>>>>>>> was
>>>>>>>>>>>> added
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>> via
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn. We
>>>> check
>>>>>> this
>>>>>>>> with
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
>>>>>>>>>>>>>>>>>>>>>>>>>> call.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this wasn't
>>>>>> sufficiently
>>>>>>>>>>>>> explained
>>>>>>>>>>>>>>>> here:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do what I propose
>> in
>>>>>> (20), we
>>>>>>>>>>> don't
>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `DescribeTransaction`
>>> call,
>>>> as
>>>>>> the
>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>> leader
>>>>>>>>>>>>>>>>>>> adds
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for older clients and we
>>> get
>>>>>> this
>>>>>>>> check
>>>>>>>>>>> for
>>>>>>>>>>>>>> free.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is that
>> if
>>>> any
>>>>>>>> messages
>>>>>>>>>>>>> somehow
>>>>>>>>>>>>>>>> come
>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the producer,
>>> they
>>>>>> will be
>>>>>>>>>>>> fenced.
>>>>>>>>>>>>>>>>> However,
>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it can be
>>>>>> discussed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree that we should
>>> have
>>>>>> epoch
>>>>>>>>>>> fencing.
>>>>>>>>>>>> My
>>>>>>>>>>>>>>>>>>> question is
>>>>>>>>>>>>>>>>>>>>>>>>> different:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Assume we are at epoch
>> 73,
>>>> and
>>>>>> we
>>>>>>>> have
>>>>>>>>>>> an
>>>>>>>>>>>>>> ongoing
>>>>>>>>>>>>>>>>>>>>>> transaction,
>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> committed. It seems
>>> natural
>>>> to
>>>>>>>> write the
>>>>>>>>>>>>>> "prepare
>>>>>>>>>>>>>>>>>>> commit"
>>>>>>>>>>>>>>>>>>>>>>> marker
>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
>>> both
>>>>> with
>>>>>>>> epoch
>>>>>>>>>>> 73,
>>>>>>>>>>>>> too,
>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>> belongs
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current transaction. Of
>>>>> course,
>>>>>> we
>>>>>>>> now
>>>>>>>>>>> also
>>>>>>>>>>>>>> bump
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>> expect
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the next requests to
>> have
>>>>> epoch
>>>>>> 74,
>>>>>>>> and
>>>>>>>>>>>> would
>>>>>>>>>>>>>>>> reject
>>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch 73, as the
>>>> corresponding
>>>>>> TX
>>>>>>>> for
>>>>>>>>>>> epoch
>>>>>>>>>>>>> 73
>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>>>>>>>>>>> committed.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems you propose to
>>>> write
>>>>>> the
>>>>>>>>>>> "prepare
>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>> marker"
>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `WriteTxMarkerRequest`
>>> with
>>>>>> epoch 74
>>>>>>>>>>>> though,
>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>> work,
>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems confusing. Is
>> there
>>> a
>>>>>> reason
>>>>>>>> why
>>>>>>>>>>> we
>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> bumped
>>>>>>>>>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>>>>>>>>> 74
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of the current
>>> epoch
>>>>> 73?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are checking if
>>> the
>>>>>>>>>>> transaction is
>>>>>>>>>>>>>>>> ongoing,
>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the leader
>>>>> partition
>>>>>> to
>>>>>>>> the
>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>> coordinator.
>>>>>>>>>>>>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for this
>>>>>> message to
>>>>>>>> come
>>>>>>>>>>>>> back,
>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>> theory
>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
>> that
>>>>> would
>>>>>>>> make the
>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why we
>> can
>>>>> check
>>>>>> the
>>>>>>>>>>> leader
>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks. Got it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, is this really
>> an
>>>>>> issue?
>>>>>>>> We put
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> produce
>>>>>>>>>>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purgatory, so how could
>> we
>>>>>> process
>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
>>>>>>>>>>>>>>>>>>>>>>>>> first?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Don't we need to put the
>>>>>>>>>>>>>> `WriteTxnMarkerRequest`
>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>> purgatory,
>>>>>>>>>>>>>>>>>>>>>>>>> too,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for this case, and
>> process
>>>>> both
>>>>>>>> request
>>>>>>>>>>>>>> in-order?
>>>>>>>>>>>>>>>>>>> (Again,
>>>>>>>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>> broker
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> knowledge is limited and
>>>> maybe
>>>>>> we
>>>>>>>> don't
>>>>>>>>>>>>>> maintain
>>>>>>>>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>>>>>>> order
>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case, what seems to be
>> an
>>>>> issue
>>>>>>>> IMHO,
>>>>>>>>>>> and I
>>>>>>>>>>>>> am
>>>>>>>>>>>>>>>>>>> wondering
>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>> changing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request handling to
>>> preserve
>>>>>> order
>>>>>>>> for
>>>>>>>>>>> this
>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>> might be
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> cleaner
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution?)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/30/22 3:28 PM,
>> Artem
>>>>>> Livshits
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think the interesting
>>>> part
>>>>> is
>>>>>>>> not in
>>>>>>>>>>>> this
>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>> (because
>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>> tries
>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> figure out when
>>>>>>>> UNKNOWN_PRODUCER_ID is
>>>>>>>>>>>>>> retriable
>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>> retryable,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's definitely not
>>> fatal),
>>>>> but
>>>>>>>> what
>>>>>>>>>>>> happens
>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'true' and falls
>> through.
>>>> In
>>>>>> the
>>>>>>>> old
>>>>>>>>>>>>> clients
>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>> fatal,
>>>>>>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> keep the behavior in
>> the
>>>> new
>>>>>>>> clients,
>>>>>>>>>>> I'd
>>>>>>>>>>>>>>>> expect it
>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>> fatal
>>>>>>>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> well.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 29, 2022 at
>>>> 11:57
>>>>>> AM
>>>>>>>>>>> Justine
>>>>>>>>>>>>>> Olshan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>> <jolshan@confluent.io.invalid
>>>>>>
>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Artem and Jeff,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
>> look
>>>> and
>>>>>>>> sorry for
>>>>>>>>>>>> the
>>>>>>>>>>>>>> slow
>>>>>>>>>>>>>>>>>>>> response.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You both mentioned the
>>>>> change
>>>>>> to
>>>>>>>>>>> handle
>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
>>>>>>>>>>>>>>>>>>>>>>>>> errors.
>>>>>>>>>>>>>>>>>>>>>>>>>>> To
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear — this error
>> code
>>>> will
>>>>>> only
>>>>>>>> be
>>>>>>>>>>> sent
>>>>>>>>>>>>>> again
>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> client's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> version is high enough
>>> to
>>>>>> ensure
>>>>>>>> we
>>>>>>>>>>>> handle
>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>> correctly.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The current (Java)
>>> client
>>>>>> handles
>>>>>>>>>>> this by
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>> (somewhat
>>>>>>>>>>>>>>>>>>>>>>>>>>> long)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code snippet:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // An
>>> UNKNOWN_PRODUCER_ID
>>>>>> means
>>>>>>>> that
>>>>>>>>>>> we
>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>> lost
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker. Depending on
>> the
>>>> log
>>>>>> start
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // offset, we may want
>>> to
>>>>>> retry
>>>>>>>>>>> these, as
>>>>>>>>>>>>>>>>> described
>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below. If
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> none of those apply,
>>> then
>>>>> for
>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // idempotent
>> producer,
>>> we
>>>>>> will
>>>>>>>>>>> locally
>>>>>>>>>>>>> bump
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>> reset
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sequence numbers of
>>>>> in-flight
>>>>>>>> batches
>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // sequence 0, then
>>> retry
>>>>> the
>>>>>>>> failed
>>>>>>>>>>>> batch,
>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>> now
>>>>>>>>>>>>>>>>>>>>>>>>>> succeed.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the transactional
>>>> producer,
>>>>>> allow
>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // batch to fail. When
>>>>>> processing
>>>>>>>> the
>>>>>>>>>>>>> failed
>>>>>>>>>>>>>>>>> batch,
>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>> transition
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an abortable error and
>>>> set a
>>>>>> flag
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // indicating that we
>>> need
>>>>> to
>>>>>>>> bump the
>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>> (if
>>>>>>>>>>>>>>>>>>>>> supported
>>>>>>>>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if (error ==
>>>>>>>>>>>> Errors.*UNKNOWN_PRODUCER_ID*)
>>>>>>>>>>>>> {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        if
>>>>>> (response.logStartOffset
>>>>>>>> ==
>>>>>>>>>>> -1)
>>>>>>>>>>>> {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // We don't
>>> know
>>>>>> the log
>>>>>>>>>>> start
>>>>>>>>>>>>>> offset
>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>> response.
>>>>>>>>>>>>>>>>>>>>>>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the request
>>>> until
>>>>>> we
>>>>>>>> get
>>>>>>>>>>> it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // The
>>>>>>>> UNKNOWN_PRODUCER_ID
>>>>>>>>>>>> error
>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>> added
>>>>>>>>>>>>>>>>>>>>>>> along
>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProduceResponse which
>>>>>> includes the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            //
>>>> logStartOffset.
>>>>>> So
>>>>>>>> the
>>>>>>>>>>> '-1'
>>>>>>>>>>>>>>>> sentinel
>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>> backward
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatibility.
>> Instead,
>>> it
>>>>> is
>>>>>>>> possible
>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // a broker
>> to
>>>> not
>>>>>> know
>>>>>>>> the
>>>>>>>>>>>>>>>>>>> logStartOffset at
>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returning
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the response because
>> the
>>>>>> partition
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // may have
>>>> moved
>>>>>> away
>>>>>>>> from
>>>>>>>>>>> the
>>>>>>>>>>>>>>>> broker
>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> time
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initially raised to
>> the
>>>> time
>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // response
>>> was
>>>>>> being
>>>>>>>>>>>>> constructed.
>>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>>>> cases,
>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retry the request: we
>>> are
>>>>>>>> guaranteed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // to
>>> eventually
>>>>>> get a
>>>>>>>>>>>>>> logStartOffset
>>>>>>>>>>>>>>>>> once
>>>>>>>>>>>>>>>>>>>>> things
>>>>>>>>>>>>>>>>>>>>>>>>> settle
>>>>>>>>>>>>>>>>>>>>>>>>>>> down.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            return true;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        if
>>>>>>>>>>> (batch.sequenceHasBeenReset()) {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // When the
>>>> first
>>>>>>>> inflight
>>>>>>>>>>>> batch
>>>>>>>>>>>>>>>> fails
>>>>>>>>>>>>>>>>>>> due to
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> truncation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> then the sequences of
>>> all
>>>>> the
>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // in flight
>>>>> batches
>>>>>>>> would
>>>>>>>>>>> have
>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>> restarted
>>>>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> beginning.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, when those
>>>>> responses
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // come back
>>>> from
>>>>>> the
>>>>>>>>>>> broker,
>>>>>>>>>>>>> they
>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>>>> come
>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
>>> error.
>>>>> In
>>>>>> this
>>>>>>>>>>> case,
>>>>>>>>>>>> we
>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // reset the
>>>>>> sequence
>>>>>>>>>>> numbers
>>>>>>>>>>>> to
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> beginning.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            return true;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        } else if
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>> (lastAckedOffset(batch.topicPartition).orElse(
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
>>>>>>>>>>>>>>>>>>>>>> response.logStartOffset) {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // The head
>> of
>>>> the
>>>>>> log
>>>>>>>> has
>>>>>>>>>>> been
>>>>>>>>>>>>>>>> removed,
>>>>>>>>>>>>>>>>>>>>> probably
>>>>>>>>>>>>>>>>>>>>>>> due
>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> retention time
>> elapsing.
>>>> In
>>>>>> this
>>>>>>>> case,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // we expect
>>> to
>>>>>> lose the
>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>> state.
>>>>>>>>>>>>>>>>>>> For
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer, reset the
>>>>> sequences
>>>>>> of
>>>>>>>> all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // inflight
>>>>> batches
>>>>>> to
>>>>>>>> be
>>>>>>>>>>> from
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> beginning
>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>> retry
>>>>>>>>>>>>>>>>>>>>>>>>>>> them,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the transaction
>>> does
>>>>> not
>>>>>>>> need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // be
>> aborted.
>>>> For
>>>>>> the
>>>>>>>>>>>> idempotent
>>>>>>>>>>>>>>>>>>> producer,
>>>>>>>>>>>>>>>>>>>>> bump
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> avoid
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reusing (sequence,
>>> epoch)
>>>>>> pairs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            if
>>>>>> (isTransactional()) {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>> txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>> this.producerIdAndEpoch);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            } else {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>> requestEpochBumpForPartition(batch.topicPartition);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            return true;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        if
>>>>> (!isTransactional())
>>>>>> {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // For the
>>>>>> idempotent
>>>>>>>>>>> producer,
>>>>>>>>>>>>>>>> always
>>>>>>>>>>>>>>>>>>> retry
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> errors. If the batch
>> has
>>>> the
>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            // producer
>> ID
>>>> and
>>>>>>>> epoch,
>>>>>>>>>>>>> request a
>>>>>>>>>>>>>>>> bump
>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> epoch.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Otherwise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just retry the
>> produce.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>> requestEpochBumpForPartition(batch.topicPartition);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            return true;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I was considering
>>> keeping
>>>>> this
>>>>>>>>>>> behavior —
>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>> am
>>>>>>>>>>>>>>>>>>> open
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>> simplifying
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We are leaving changes
>>> to
>>>>>> older
>>>>>>>>>>> clients
>>>>>>>>>>>> off
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> table
>>>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> caused
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many issues for
>> clients
>>> in
>>>>> the
>>>>>>>> past.
>>>>>>>>>>>>>> Previously
>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>> fatal
>>>>>>>>>>>>>>>>>>>>>>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we didn't have the
>>>>> mechanisms
>>>>>> in
>>>>>>>>>>> place to
>>>>>>>>>>>>>>>> detect
>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> legitimate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case vs some bug or
>> gap
>>> in
>>>>> the
>>>>>>>>>>> protocol.
>>>>>>>>>>>>>>>> Ensuring
>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>> has
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> own epoch should close
>>>> this
>>>>>> gap.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> And to address Jeff's
>>>> second
>>>>>>>> point:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *does the typical
>>> produce
>>>>>> request
>>>>>>>> path
>>>>>>>>>>>>> append
>>>>>>>>>>>>>>>>>>> records
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> local
>>>>>>>>>>>>>>>>>>>>>>>>> log
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along*
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *with the
>>>>>> currentTxnFirstOffset
>>>>>>>>>>>>> information?
>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand*
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *when the field is
>>> written
>>>>> to
>>>>>>>> disk.*
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, the first produce
>>>>> request
>>>>>>>>>>> populates
>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>> field
>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>> writes
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> offset
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as part of the record
>>>> batch
>>>>>> and
>>>>>>>> also
>>>>>>>>>>> to
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>>>>> snapshot.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we reload the records
>> on
>>>>>> restart
>>>>>>>>>>> and/or
>>>>>>>>>>>>>>>>>>> reassignment,
>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>> repopulate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> field with the
>> snapshot
>>>> from
>>>>>> disk
>>>>>>>>>>> along
>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> rest
>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me know if there
>> are
>>>>>> further
>>>>>>>>>>> comments
>>>>>>>>>>>>>>>> and/or
>>>>>>>>>>>>>>>>>>>>> questions.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
>> at
>>>> 9:00
>>>>>> PM
>>>>>>>> Jeff
>>>>>>>>>>> Kim
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> <jeff.kim@confluent.io.invalid
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP! I
>>>> have
>>>>>> two
>>>>>>>>>>>> questions:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1) For new clients,
>> we
>>>> can
>>>>>> once
>>>>>>>> again
>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for sequences
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that are non-zero
>> when
>>>>> there
>>>>>> is
>>>>>>>> no
>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>> present
>>>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This will indicate we
>>>>> missed
>>>>>> the
>>>>>>>> 0
>>>>>>>>>>>>> sequence
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>> yet
>>>>>>>>>>>>>>>>>>>>>>>>> want
>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the log.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would like to
>>>> understand
>>>>>> the
>>>>>>>>>>> current
>>>>>>>>>>>>>>>> behavior
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> handle
>>>>>>>>>>>>>>>>>>>>>>>> older
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and if there are any
>>>>> changes
>>>>>> we
>>>>>>>> are
>>>>>>>>>>>>> making.
>>>>>>>>>>>>>>>> Maybe
>>>>>>>>>>>>>>>>>>> I'm
>>>>>>>>>>>>>>>>>>>>>>> missing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but we would want to
>>>>> identify
>>>>>>>>>>> whether we
>>>>>>>>>>>>>>>> missed
>>>>>>>>>>>>>>>>>>> the 0
>>>>>>>>>>>>>>>>>>>>>>> sequence
>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> older
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients, no?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2) Upon returning
>> from
>>>> the
>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>> coordinator, we
>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>> set
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as ongoing on the
>>> leader
>>>> by
>>>>>>>>>>> populating
>>>>>>>>>>>>>>>>>>>>>> currentTxnFirstOffset
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> through the typical
>>>> produce
>>>>>>>> request
>>>>>>>>>>>>>> handling.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does the typical
>>> produce
>>>>>> request
>>>>>>>> path
>>>>>>>>>>>>> append
>>>>>>>>>>>>>>>>>>> records
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> local
>>>>>>>>>>>>>>>>>>>>>>>>> log
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> along
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>> currentTxnFirstOffset
>>>>>>>>>>>>> information?
>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>> understand
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when the field is
>>> written
>>>>> to
>>>>>>>> disk.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeff
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
>> at
>>>>> 4:44
>>>>>> PM
>>>>>>>> Artem
>>>>>>>>>>>>>> Livshits
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
>>> alivshits@confluent.io
>>>>>> .invalid>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Justine,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for the
>> KIP.
>>>> I
>>>>>> have
>>>>>>>> one
>>>>>>>>>>>>>> question.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 5) For new clients,
>> we
>>>> can
>>>>>> once
>>>>>>>>>>> again
>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe we had
>>>> problems
>>>>>> in the
>>>>>>>>>>> past
>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>> returning
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> UNKNOWN_PRODUCER_ID
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because it was
>>>> considered
>>>>>> fatal
>>>>>>>> and
>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>> client
>>>>>>>>>>>>>>>>>>>>>>> restart.
>>>>>>>>>>>>>>>>>>>>>>>>> It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good to spell out
>> the
>>>> new
>>>>>> client
>>>>>>>>>>>> behavior
>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>> receives
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> error.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Artem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022
>>> at
>>>>>> 10:00 AM
>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>> Olshan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>> <jo...@confluent.io.invalid>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking a
>>>> look
>>>>>>>> Matthias.
>>>>>>>>>>>> I've
>>>>>>>>>>>>>>>> tried
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> answer
>>>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> questions
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> below:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 10)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Right — so the
>>> hanging
>>>>>>>> transaction
>>>>>>>>>>>> only
>>>>>>>>>>>>>>>> occurs
>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>> late
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> come in and the
>>>> partition
>>>>>> is
>>>>>>>> never
>>>>>>>>>>>> added
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>> again.
>>>>>>>>>>>>>>>>>>>>>>>>>>> If
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never add the
>>> partition
>>>>> to
>>>>>> a
>>>>>>>>>>>>> transaction,
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>> never
>>>>>>>>>>>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> marker
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> never advance the
>>> LSO.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we do end up
>>> adding
>>>>> the
>>>>>>>>>>> partition
>>>>>>>>>>>> to
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>> (I
>>>>>>>>>>>>>>>>>>>>>>>>>> suppose
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happen before or
>>> after
>>>>> the
>>>>>> late
>>>>>>>>>>>> message
>>>>>>>>>>>>>>>> comes
>>>>>>>>>>>>>>>>> in)
>>>>>>>>>>>>>>>>>>>> then
>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> include
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> late message in the
>>>> next
>>>>>>>>>>> (incorrect)
>>>>>>>>>>>>>>>>> transaction.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> So perhaps it is
>>>> clearer
>>>>> to
>>>>>>>> make
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> distinction
>>>>>>>>>>>>>>>>>>>>> between
>>>>>>>>>>>>>>>>>>>>>>>>>> messages
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> eventually get
>> added
>>> to
>>>>> the
>>>>>>>>>>>> transaction
>>>>>>>>>>>>>> (but
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> wrong
>>>>>>>>>>>>>>>>>>>>>>> one)
>>>>>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messages
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that never get
>> added
>>>> and
>>>>>> become
>>>>>>>>>>>> hanging.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 20)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The client side
>>> change
>>>>> for
>>>>>> 2 is
>>>>>>>>>>>> removing
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> addPartitions
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call. We don't need
>>> to
>>>>> make
>>>>>>>> this
>>>>>>>>>>> from
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> txn
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coordinator,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only server side.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In my opinion, the
>>>> issue
>>>>>> with
>>>>>>>> the
>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
>>>>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>> older
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is that we don't
>> have
>>>> the
>>>>>> epoch
>>>>>>>>>>> bump,
>>>>>>>>>>>> so
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> belongs to the
>>> previous
>>>>>>>>>>> transaction or
>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>> one.
>>>>>>>>>>>>>>>>>>> We
>>>>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partition has been
>>>> added
>>>>> to
>>>>>>>> this
>>>>>>>>>>>>>>>> transaction.
>>>>>>>>>>>>>>>>> Of
>>>>>>>>>>>>>>>>>>>>> course,
>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>> means
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't completely
>>> cover
>>>>> the
>>>>>> case
>>>>>>>>>>> where
>>>>>>>>>>>> we
>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>>>>>> late
>>>>>>>>>>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have added the
>>>> partition
>>>>> to
>>>>>>>> the new
>>>>>>>>>>>>>>>>> transaction,
>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>> that's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something we will
>>> need
>>>>> the
>>>>>> new
>>>>>>>>>>> clients
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> cover.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Transaction is
>>> ongoing
>>>> =
>>>>>>>> partition
>>>>>>>>>>> was
>>>>>>>>>>>>>>>> added to
>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>> via
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn.
>>> We
>>>>>> check
>>>>>>>> this
>>>>>>>>>>> with
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> DescribeTransactions
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me know if this
>>> wasn't
>>>>>>>> sufficiently
>>>>>>>>>>>>>>>> explained
>>>>>>>>>>>>>>>>>>> here:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 40)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The idea here is
>> that
>>>> if
>>>>>> any
>>>>>>>>>>> messages
>>>>>>>>>>>>>>>> somehow
>>>>>>>>>>>>>>>>>>> come
>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch to the
>>> producer,
>>>>> they
>>>>>>>> will be
>>>>>>>>>>>>>> fenced.
>>>>>>>>>>>>>>>>>>> However,
>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is necessary, it
>> can
>>> be
>>>>>>>> discussed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 50)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It should be
>>>> synchronous
>>>>>>>> because
>>>>>>>>>>> if we
>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>> event
>>>>>>>>>>>>>>>>>>>>>> (ie,
>>>>>>>>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>> error)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes us to need
>> to
>>>>> abort
>>>>>> the
>>>>>>>>>>>>>> transaction,
>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> partitions to send
>>>>>> transaction
>>>>>>>>>>> markers
>>>>>>>>>>>>> to.
>>>>>>>>>>>>>>>> We
>>>>>>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> partitions
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we added them to
>> the
>>>>>>>> coordinator
>>>>>>>>>>> via
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
>>>>>>>>>>>>>>>>>>>>>>>>>> call.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Previously we have
>>> had
>>>>>>>> asynchronous
>>>>>>>>>>>>> calls
>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> past
>>>>>>>>>>>>>>>>>>>>>> (ie,
>>>>>>>>>>>>>>>>>>>>>>>>>> writing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit markers when
>>> the
>>>>>>>>>>> transaction is
>>>>>>>>>>>>>>>>> completed)
>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>> often
>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> causes confusion as
>>> we
>>>>>> need to
>>>>>>>> wait
>>>>>>>>>>>> for
>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>> operations
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>> complete.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing commit
>>> markers
>>>>>> case,
>>>>>>>>>>> clients
>>>>>>>>>>>>> often
>>>>>>>>>>>>>>>> see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> CONCURRENT_TRANSACTIONs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error messages and
>>> that
>>>>>> can be
>>>>>>>>>>>>> confusing.
>>>>>>>>>>>>>>>> For
>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> reason,
>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>> may
>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simpler to just
>> have
>>>>>>>> synchronous
>>>>>>>>>>>> calls —
>>>>>>>>>>>>>>>>>>> especially
>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> block
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some operation's
>>>>> completion
>>>>>>>> anyway
>>>>>>>>>>>>> before
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>> start
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction. And
>>> yes, I
>>>>>> meant
>>>>>>>>>>>>>> coordinator. I
>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>> fix
>>>>>>>>>>>>>>>>>>>>>>> that.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 60)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> When we are
>> checking
>>> if
>>>>> the
>>>>>>>>>>>> transaction
>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>> ongoing,
>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> round
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trip from the
>> leader
>>>>>> partition
>>>>>>>> to
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>> coordinator.
>>>>>>>>>>>>>>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> time
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we are waiting for
>>> this
>>>>>>>> message to
>>>>>>>>>>>> come
>>>>>>>>>>>>>>>> back,
>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>> theory
>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a commit/abort call
>>>> that
>>>>>> would
>>>>>>>> make
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> date. That is why
>> we
>>>> can
>>>>>> check
>>>>>>>> the
>>>>>>>>>>>>> leader
>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm happy to update
>>> the
>>>>>> KIP if
>>>>>>>>>>> some of
>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>> things
>>>>>>>>>>>>>>>>>>>>> were
>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>> clear.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Nov 21,
>> 2022
>>> at
>>>>>> 7:11 PM
>>>>>>>>>>>> Matthias
>>>>>>>>>>>>>> J.
>>>>>>>>>>>>>>>>> Sax <
>>>>>>>>>>>>>>>>>>>>>>>>>> mjsax@apache.org
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the
>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Couple of
>>>> clarification
>>>>>>>> questions
>>>>>>>>>>> (I
>>>>>>>>>>>> am
>>>>>>>>>>>>>>>> not a
>>>>>>>>>>>>>>>>>>>> broker
>>>>>>>>>>>>>>>>>>>>>>> expert
>>>>>>>>>>>>>>>>>>>>>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some question are
>>>>> obvious
>>>>>> for
>>>>>>>>>>> others,
>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>> me
>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>> lack
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> broker knowledge).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (10)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What happens if
>> the
>>>>>> message
>>>>>>>> come
>>>>>>>>>>> in
>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addPartitionsToTxn
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request? It seems
>>> the
>>>>>> broker
>>>>>>>>>>> hosting
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> data
>>>>>>>>>>>>>>>>>>>>>> partitions
>>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anything about it
>>> and
>>>>>> append
>>>>>>>> it to
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> partition,
>>>>>>>>>>>>>>>>>>>>> too?
>>>>>>>>>>>>>>>>>>>>>>> What
>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> difference between
>>>> both
>>>>>> cases?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, it seems a
>> TX
>>>>> would
>>>>>> only
>>>>>>>>>>> hang,
>>>>>>>>>>>> if
>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>> is no
>>>>>>>>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>> TX
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> either committer
>> or
>>>>>> aborted?
>>>>>>>> Thus,
>>>>>>>>>>>> for
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>> above,
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> TX
>>>>>>>>>>>>>>>>>>>>>>>>>>> might
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually not hang
>>> (of
>>>>>> course,
>>>>>>>> we
>>>>>>>>>>>> might
>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>> EOS
>>>>>>>>>>>>>>>>>>>>>>> violation
>>>>>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX was aborted and
>>> the
>>>>>> second
>>>>>>>>>>>>> committed,
>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>>>>>>>>> around).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (20)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Of course, 1 and
>> 2
>>>>>> require
>>>>>>>>>>>> client-side
>>>>>>>>>>>>>>>>>>> changes, so
>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>> older
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clients,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> those approaches
>>> won’t
>>>>>> apply.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For (1) I
>> understand
>>>>> why a
>>>>>>>> client
>>>>>>>>>>>>> change
>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>> necessary,
>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we need a client
>>>> change
>>>>>> for
>>>>>>>> (2).
>>>>>>>>>>> Can
>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>> elaborate?
>>>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>>>>>>> Later
>>>>>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> explain
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that we should
>> send
>>> a
>>>>>>>>>>>>>>>>>>> DescribeTransactionRequest,
>>>>>>>>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>>>>>>>> am
>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>> sure
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Can't we not just
>> do
>>>> an
>>>>>>>> implicit
>>>>>>>>>>>>>>>>>>> AddPartiitonToTx,
>>>>>>>>>>>>>>>>>>>>> too?
>>>>>>>>>>>>>>>>>>>>>>> If
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> old
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer correctly
>>>>>> registered
>>>>>>>> the
>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>> already,
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> TX-coordinator
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can just ignore it
>>> as
>>>>>> it's an
>>>>>>>>>>>>> idempotent
>>>>>>>>>>>>>>>>>>> operation?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (30)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To cover older
>>>> clients,
>>>>>> we
>>>>>>>> will
>>>>>>>>>>>>> ensure a
>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>> ongoing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we write to a
>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Not sure what you
>>> mean
>>>>> by
>>>>>>>> this?
>>>>>>>>>>> Can
>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>> elaborate?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (40)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [the
>>> TX-coordinator]
>>>>> will
>>>>>>>> write
>>>>>>>>>>> the
>>>>>>>>>>>>>>>> prepare
>>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bumped
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> epoch and send
>>>>>>>>>>> WriteTxnMarkerRequests
>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> bumped
>>>>>>>>>>>>>>>>>>>>>>>> epoch.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why do we use the
>>>> bumped
>>>>>>>> epoch for
>>>>>>>>>>>>> both?
>>>>>>>>>>>>>> It
>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>> intuitive
>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the current epoch,
>>> and
>>>>>> only
>>>>>>>> return
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> bumped
>>>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> producer?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (50) "Implicit
>>>>>>>>>>>>> AddPartitionToTransaction"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Why does the
>>>> implicitly
>>>>>> sent
>>>>>>>>>>> request
>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>> synchronous?
>>>>>>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also says
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in case we need
>> to
>>>>> abort
>>>>>> and
>>>>>>>>>>> need to
>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>> partitions
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What do you mean
>> by
>>>>> this?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we don’t want to
>>>> write
>>>>>> to it
>>>>>>>>>>> before
>>>>>>>>>>>> we
>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> manager
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Do you mean
>>>>> TX-coordinator
>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>> "manager"?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (60)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For older clients
>>> and
>>>>>> ensuring
>>>>>>>>>>> that
>>>>>>>>>>>> the
>>>>>>>>>>>>>> TX
>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>> ongoing,
>>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> describe a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> race condition. I
>> am
>>>> not
>>>>>> sure
>>>>>>>> if I
>>>>>>>>>>>> can
>>>>>>>>>>>>>>>> follow
>>>>>>>>>>>>>>>>>>> here.
>>>>>>>>>>>>>>>>>>>>> Can
>>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> elaborate?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 11/18/22 1:21
>> PM,
>>>>>> Justine
>>>>>>>>>>> Olshan
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey all!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to
>> start a
>>>>>>>> discussion
>>>>>>>>>>> on my
>>>>>>>>>>>>>>>> proposal
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> add
>>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> server-side
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checks on
>>>> transactions
>>>>> to
>>>>>>>> avoid
>>>>>>>>>>>>> hanging
>>>>>>>>>>>>>>>>>>>>> transactions.
>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an issue for some
>>>> time,
>>>>>> so I
>>>>>>>>>>> really
>>>>>>>>>>>>> hope
>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>> helpful
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> many
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users of EOS.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The KIP includes
>>>>> changes
>>>>>> that
>>>>>>>>>>> will
>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>> compatible
>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>> old
>>>>>>>>>>>>>>>>>>>>>>>>>>> clients
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> changes to
>> improve
>>>>>>>> performance
>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> correctness
>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>> clients.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Please take a
>> look
>>>> and
>>>>>> leave
>>>>>>>> any
>>>>>>>>>>>>>> comments
>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>> may
>>>>>>>>>>>>>>>>>>>>>> have!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> JIRA:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>> https://issues.apache.org/jira/browse/KAFKA-14402
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Justine
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
> 

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
Hey Artem,

I see there is a check for transactional producers. I'm wondering if we
don't handle the epoch overflow case. I'm also not sure it will be a huge
issue to extend to transactional producers, but maybe I'm missing something.

As for the recovery path -- I think Guozhang's point was if we have a bad
client that repeatedly tries to produce without adding to the transaction
we would do the following:
a) if not fatal, we just fail the produce request over and over
b) if fatal, we fence the producer

Here with B, the issue with the client would be made clear more quickly. I
suppose there are some intermediate cases where the issue only occurs
sometimes, but I wonder if we should consider how to recover with clients
who don't behave as expected anyway.

I think there is a place for the abortable error that we are adding -- just
abort and try again. But I think there are also some cases where trying to
recover overcomplicates some logic. Especially if we are considering older
clients -- there I'm not sure if there's a ton we can do besides fail the
batch or fence the producer. With newer clients, we can consider more
options for what can just be recovered after aborting. But epochs might be
a hard one unless we also want to reset producer ID.

Thanks,
Justine



On Fri, Jan 20, 2023 at 3:59 PM Artem Livshits
<al...@confluent.io.invalid> wrote:

> >  besides the poorly written client case
>
> A poorly written client could create a lot of grief to people who run Kafka
> brokers :-), so when deciding to make an error fatal I would see if there
> is a reasonable recovery path rather than how often it could happen.  If we
> have solid implementation of transactions (which I hope we'll do as a
> result of this KIP), it would help to recover from a large class of errors
> by just aborting a transaction, even if the cause of error is a race
> condition or etc.
>
> -Artem
>
> On Fri, Jan 20, 2023 at 3:26 PM Justine Olshan
> <jo...@confluent.io.invalid>
> wrote:
>
> > Artem --
> > I guess the discussion path we were going down is when we expect to see
> > this error. I mentioned that it was hard to come up with cases for when
> the
> > producer would still be around to receive the error besides the poorly
> > written client case.
> > If we don't expect to have a producer to receive the response, it sort of
> > makes sense for it to be fatal.
> >
> > I had some discussion with Jason offline about the epoch being off cases
> > and I'm not sure we could find a ton (outside of produce requests) where
> we
> > could/should recover. I'd be happy to hear some examples though, maybe
> I'm
> > missing something.
> >
> > Thanks,
> > Justine
> >
> > On Fri, Jan 20, 2023 at 3:19 PM Artem Livshits
> > <al...@confluent.io.invalid> wrote:
> >
> > > In general, I'd like to avoid fatal errors as much as possible, in some
> > > sense fatal errors just push out recovery logic to the application
> which
> > > either complicates the application or leads to disruption (we've seen
> > cases
> > > when a transient broker error could lead to work stoppage when
> > applications
> > > need to be manually restarted).  I think we should strive to define
> > > recovery logic for most errors (and/or encapsulate it in the Kafka
> client
> > > as much as possible).
> > >
> > > One benefit of transactions is that they simplify recovery from errors,
> > > pretty much any error (that's not handled transparently by retries in
> > Kafka
> > > client) can be handled by the application via aborting the transaction
> > and
> > > repeating the transactional logic again.  One tricky error is an error
> > > during commit, because we don't know the outcome.  For commit errors,
> the
> > > recommendation should be to retry the commit until it returns the
> > specific
> > > result (committed or aborted).
> > >
> > > -Artem
> > >
> > > On Fri, Jan 20, 2023 at 2:52 PM Justine Olshan
> > > <jo...@confluent.io.invalid>
> > > wrote:
> > >
> > > > That's a fair point about other clients.
> > > >
> > > > I think the abortable error case is interesting because I'm curious
> how
> > > > other clients would handle this. I assume they would need to
> implement
> > > > handling for the error code unless they did something like "any
> unknown
> > > > error codes/any codes that aren't x,y,z are retriable." I would hope
> > that
> > > > unknown error codes were fatal, and if the code was implemented it
> > would
> > > > abort the transaction. But I will think on this too.
> > > >
> > > > As for InvalidRecord -- you mentioned it was not fatal, but I'm
> taking
> > a
> > > > look through the code. We would see this on handling the produce
> > > response.
> > > > If I recall correctly, we check if errors are retriable. I think this
> > > error
> > > > would not be retriable. But I guess the concern here is that it is
> not
> > > > enough for just that batch to fail. I guess I hadn't considered fully
> > > > fencing the old producer but there are valid arguments here why we
> > would
> > > > want to.
> > > >
> > > > Thanks,
> > > > Justine
> > > >
> > > > On Fri, Jan 20, 2023 at 2:35 PM Guozhang Wang <
> > > guozhang.wang.us@gmail.com>
> > > > wrote:
> > > >
> > > > > Thanks Justine for the replies! I agree with most of your thoughts.
> > > > >
> > > > > Just for 3/7), though I agree for our own AK producer, since we do
> > > > > "nextRequest(boolean hasIncompleteBatches)", we guarantee the
> end-txn
> > > > > would not be sent until we've effectively flushed, but I was
> > referring
> > > > > to any future bugs or other buggy clients that the same client may
> > get
> > > > > into this situation, in which case we should give the client a
> clear
> > > > > msg that "you did something wrong, and hence now you should fatally
> > > > > close yourself". What I'm concerned about is that, by seeing an
> > > > > "abortable error" or in some rare cases an "invalid record", the
> > > > > client could not realize "something that's really bad happened". So
> > > > > it's not about adding a new error, it's mainly about those real
> buggy
> > > > > situations causing such "should never happen" cases, the errors
> > return
> > > > > would not be informative enough.
> > > > >
> > > > > Thinking in other ways, if we believe that for most cases such
> error
> > > > > codes would not reach the original clients since they would be
> > > > > disconnected or even gone by that time, and only in some rare cases
> > > > > they would still be seen by the sending clients, then why not make
> > > > > them more fatal and more specific than generic.
> > > > >
> > > > > Guozhang
> > > > >
> > > > > On Fri, Jan 20, 2023 at 1:59 PM Justine Olshan
> > > > > <jo...@confluent.io.invalid> wrote:
> > > > > >
> > > > > > Hey Guozhang. Thanks for taking a look and for the detailed
> > comments!
> > > > > I'll
> > > > > > do my best to address below.
> > > > > >
> > > > > > 1. I see what you are saying here, but I think I need to look
> > through
> > > > the
> > > > > > sequence of events you mention. Typically we've seen this issue
> in
> > a
> > > > few
> > > > > > cases.
> > > > > >
> > > > > >  One is when we have a producer disconnect when trying to
> produce.
> > > > > > Typically in these cases, we abort the transaction. We've seen
> that
> > > > after
> > > > > > the markers are written, the disconnection can sometimes cause
> the
> > > > > request
> > > > > > to get flushed to the broker. In this case, we don't need client
> > > > handling
> > > > > > because the producer we are responding to is gone. We just needed
> > to
> > > > make
> > > > > > sure we didn't write to the log on the broker side. I'm trying to
> > > think
> > > > > of
> > > > > > a case where we do have the client to return to. I'd think the
> same
> > > > > client
> > > > > > couldn't progress to committing the transaction unless the
> produce
> > > > > request
> > > > > > returned right? Of course, there is the incorrectly written
> clients
> > > > case.
> > > > > > I'll think on this a bit more and let you know if I come up with
> > > > another
> > > > > > scenario when we would return to an active client when the
> > > transaction
> > > > is
> > > > > > no longer ongoing.
> > > > > >
> > > > > > I was not aware that we checked the result of a send after we
> > commit
> > > > > > though. I'll need to look into that a bit more.
> > > > > >
> > > > > > 2. There were some questions about this in the discussion. The
> plan
> > > is
> > > > to
> > > > > > handle overflow with the mechanism we currently have in the
> > producer.
> > > > If
> > > > > we
> > > > > > try to bump and the epoch will overflow, we actually allocate a
> new
> > > > > > producer ID. I need to confirm the fencing logic on the last
> epoch
> > > (ie,
> > > > > we
> > > > > > probably shouldn't allow any records to be produced with the
> final
> > > > epoch
> > > > > > since we can never properly fence that one).
> > > > > >
> > > > > > 3. I can agree with you that the current error handling is
> messy. I
> > > > > recall
> > > > > > taking a look at your KIP a while back, but I think I mostly saw
> > the
> > > > > > section about how the errors were wrapped. Maybe I need to take
> > > another
> > > > > > look. As for abortable error, the idea was that the handling
> would
> > be
> > > > > > simple -- if this error is seen, the transaction should be
> aborted
> > --
> > > > no
> > > > > > other logic about previous state or requests necessary. Is your
> > > concern
> > > > > > simply about adding new errors? We were hoping to have an error
> > that
> > > > > would
> > > > > > have one meaning and many of the current errors have a history of
> > > > meaning
> > > > > > different things on different client versions. That was the main
> > > > > motivation
> > > > > > for adding a new error.
> > > > > >
> > > > > > 4. This is a good point about record timestamp reordering.
> > Timestamps
> > > > > don't
> > > > > > affect compaction, but they do affect retention deletion. For
> that,
> > > > kafka
> > > > > > considers the largest timestamp in the segment, so I think a
> small
> > > > amount
> > > > > > of reordering (hopefully on the order of milliseconds or even
> > > seconds)
> > > > > will
> > > > > > be ok. We take timestamps from clients so there is already a
> > > > possibility
> > > > > > for some drift and non-monotonically increasing timestamps.
> > > > > >
> > > > > > 5. Thanks for catching. The error is there, but it's actually
> that
> > > > those
> > > > > > fields should be 4+! Due to how the message generator works, I
> > > actually
> > > > > > have to redefine those fields inside the
> > > > `"AddPartitionsToTxnTransaction`
> > > > > > block for it to build correctly. I'll fix it to be correct.
> > > > > >
> > > > > > 6. Correct -- we will only add the request to purgatory if the
> > cache
> > > > has
> > > > > no
> > > > > > ongoing transaction. I can change the wording to make that
> clearer
> > > that
> > > > > we
> > > > > > only place the request in purgatory if we need to contact the
> > > > transaction
> > > > > > coordinator.
> > > > > >
> > > > > > 7. We did take a look at some of the errors and it was hard to
> come
> > > up
> > > > > with
> > > > > > a good one. I agree that InvalidTxnStateException is ideal except
> > for
> > > > the
> > > > > > fact that it hasn't been returned on Produce requests before. The
> > > error
> > > > > > handling for clients is a bit vague (which is why I opened
> > > KAFKA-14439
> > > > > > <https://issues.apache.org/jira/browse/KAFKA-14439>), but the
> > > decision
> > > > > we
> > > > > > made here was to only return errors that have been previously
> > > returned
> > > > to
> > > > > > producers. As for not being fatal, I think part of the theory was
> > > that
> > > > in
> > > > > > many cases, the producer would be disconnected. (See point 1) and
> > > this
> > > > > > would just be an error to return from the server. I did plan to
> > think
> > > > > about
> > > > > > other cases, so let me know if you think of any as well!
> > > > > >
> > > > > > Lots to say! Let me know if you have further thoughts!
> > > > > > Justine
> > > > > >
> > > > > > On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang <
> > > > > guozhang.wang.us@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hello Justine,
> > > > > > >
> > > > > > > Thanks for the great write-up! I made a quick pass through it
> and
> > > > here
> > > > > > > are some thoughts (I have not been able to read through this
> > thread
> > > > so
> > > > > > > pardon me if they have overlapped or subsumed by previous
> > > comments):
> > > > > > >
> > > > > > > First are some meta ones:
> > > > > > >
> > > > > > > 1. I think we need to also improve the client's experience once
> > we
> > > > > > > have this defence in place. More concretely, say a user's
> > producer
> > > > > > > code is like following:
> > > > > > >
> > > > > > > future = producer.send();
> > > > > > > // producer.flush();
> > > > > > > producer.commitTransaction();
> > > > > > > future.get();
> > > > > > >
> > > > > > > Which resulted in the order of a) produce-request sent by
> > producer,
> > > > b)
> > > > > > > end-txn-request sent by producer, c) end-txn-response sent
> back,
> > d)
> > > > > > > txn-marker-request sent from coordinator to partition leader,
> e)
> > > > > > > produce-request finally received by the partition leader,
> before
> > > this
> > > > > > > KIP e) step would be accepted causing a dangling txn; now it
> > would
> > > be
> > > > > > > rejected in step e) which is good. But from the client's point
> of
> > > > view
> > > > > > > now it becomes confusing since the `commitTransaction()`
> returns
> > > > > > > successfully, but the "future" throws an invalid-epoch error,
> and
> > > > they
> > > > > > > are not sure if the transaction did succeed or not. In fact, it
> > > > > > > "partially succeeded" with some msgs being rejected but others
> > > > > > > committed successfully.
> > > > > > >
> > > > > > > Of course the easy way to avoid this is, always call
> > > > > > > "producer.flush()" before commitTxn and that's what we do
> > > ourselves,
> > > > > > > and what we recommend users do. But I suspect not everyone does
> > it.
> > > > In
> > > > > > > fact I just checked the javadoc in KafkaProducer and our code
> > > snippet
> > > > > > > does not include a `flush()` call. So I'm thinking maybe we can
> > in
> > > > > > > side the `commitTxn` code to enforce flushing before sending
> the
> > > > > > > end-txn request.
> > > > > > >
> > > > > > > 2. I'd like to clarify a bit details on "just add partitions to
> > the
> > > > > > > transaction on the first produce request during a transaction".
> > My
> > > > > > > understanding is that the partition leader's cache has the
> > producer
> > > > id
> > > > > > > / sequence / epoch for the latest txn, either on-going or is
> > > > completed
> > > > > > > (upon receiving the marker request from coordinator). When a
> > > produce
> > > > > > > request is received, if
> > > > > > >
> > > > > > > * producer's epoch < cached epoch, or producer's epoch ==
> cached
> > > > epoch
> > > > > > > but the latest txn is completed, leader directly reject with
> > > > > > > invalid-epoch.
> > > > > > > * producer's epoch > cached epoch, park the the request and
> send
> > > > > > > add-partitions request to coordinator.
> > > > > > >
> > > > > > > In order to do it, does the coordinator need to bump the
> sequence
> > > and
> > > > > > > reset epoch to 0 when the next epoch is going to overflow? If
> no
> > > need
> > > > > > > to do so, then how we handle the (admittedly rare, but still
> may
> > > > > > > happen) epoch overflow situation?
> > > > > > >
> > > > > > > 3. I'm a bit concerned about adding a generic "ABORTABLE_ERROR"
> > > given
> > > > > > > we already have a pretty messy error classification and error
> > > > handling
> > > > > > > on the producer clients side --- I have a summary about the
> > issues
> > > > and
> > > > > > > a proposal to address this in
> > > > > > >
> > > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
> > > > > > > -- I understand we do not want to use "UNKNOWN_PRODUCER_ID"
> > anymore
> > > > > > > and in fact we intend to deprecate it in KIP-360 and eventually
> > > > remove
> > > > > > > it; but I'm wondering can we still use specific error codes.
> E.g.
> > > > what
> > > > > > > about "InvalidProducerEpochException" since for new clients,
> the
> > > > > > > actual reason this would actually be rejected is indeed because
> > the
> > > > > > > epoch on the coordinator caused the add-partitions-request from
> > the
> > > > > > > brokers to be rejected anyways?
> > > > > > >
> > > > > > > 4. It seems we put the producer request into purgatory before
> we
> > > ever
> > > > > > > append the records, while other producer's records may still be
> > > > > > > appended during the time; and that potentially may result in
> some
> > > > > > > re-ordering compared with reception order. I'm not super
> > concerned
> > > > > > > about it since Kafka does not guarantee reception ordering
> across
> > > > > > > producers anyways, but it may make the timestamps of records
> > > inside a
> > > > > > > partition to be more out-of-ordered. Are we aware of any
> > scenarios
> > > > > > > such as future enhancements on log compactions that may be
> > affected
> > > > by
> > > > > > > this effect?
> > > > > > >
> > > > > > > Below are just minor comments:
> > > > > > >
> > > > > > > 5. In "AddPartitionsToTxnTransaction" field of
> > > > > > > "AddPartitionsToTxnRequest" RPC, the versions of those inner
> > fields
> > > > > > > are "0-3" while I thought they should be "0+" still?
> > > > > > >
> > > > > > > 6. Regarding "we can place the request in a purgatory of sorts
> > and
> > > > > > > check if there is any state for the transaction on the
> broker": i
> > > > > > > think at this time when we just do the checks against the
> cached
> > > > > > > state, we do not need to put the request to purgatory yet?
> > > > > > >
> > > > > > > 7. This is related to 3) above. I feel using
> > > "InvalidRecordException"
> > > > > > > for older clients may also be a bit confusing, and also it is
> not
> > > > > > > fatal -- for old clients, it better to be fatal since this
> > > indicates
> > > > > > > the clients is doing something wrong and hence it should be
> > closed.
> > > > > > > And in general I'd prefer to use slightly more specific meaning
> > > error
> > > > > > > codes for clients. That being said, I also feel
> > > > > > > "InvalidProducerEpochException" is not suitable for old
> versioned
> > > > > > > clients, and we'd have to pick one that old clients recognize.
> > I'd
> > > > > > > prefer "InvalidTxnStateException" but that one is supposed to
> be
> > > > > > > returned from txn coordinators only today. I'd suggest we do a
> > > quick
> > > > > > > check in the current client's code path and see if that one
> would
> > > be
> > > > > > > handled if it's from a produce-response, and if yes, use this
> > one;
> > > > > > > otherwise, use "ProducerFencedException" which is much less
> > > > meaningful
> > > > > > > but it's still a fatal error.
> > > > > > >
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Guozhang
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
> > > > > > > <jo...@confluent.io.invalid> wrote:
> > > > > > > >
> > > > > > > > Yeah -- looks like we already have code to handle bumping the
> > > epoch
> > > > > and
> > > > > > > > when the epoch is Short.MAX_VALUE, we get a new producer ID.
> > > Since
> > > > > this
> > > > > > > is
> > > > > > > > already the behavior, do we want to change it further?
> > > > > > > >
> > > > > > > > Justine
> > > > > > > >
> > > > > > > > On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <
> > > > jolshan@confluent.io
> > > > > >
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hey all, just wanted to quickly update and say I've
> modified
> > > the
> > > > > KIP to
> > > > > > > > > explicitly mention that AddOffsetCommitsToTxnRequest will
> be
> > > > > replaced
> > > > > > > by
> > > > > > > > > a coordinator-side (inter-broker) AddPartitionsToTxn
> implicit
> > > > > request.
> > > > > > > This
> > > > > > > > > mirrors the user partitions and will implicitly add offset
> > > > > partitions
> > > > > > > to
> > > > > > > > > transactions when we commit offsets on them. We will
> > deprecate
> > > > > > > AddOffsetCommitsToTxnRequest
> > > > > > > > > for new clients.
> > > > > > > > >
> > > > > > > > > Also to address Artem's comments --
> > > > > > > > > I'm a bit unsure if the changes here will change the
> previous
> > > > > behavior
> > > > > > > for
> > > > > > > > > fencing producers. In the case you mention in the first
> > > > paragraph,
> > > > > are
> > > > > > > you
> > > > > > > > > saying we bump the epoch before we try to abort the
> > > transaction?
> > > > I
> > > > > > > think I
> > > > > > > > > need to understand the scenarios you mention a bit better.
> > > > > > > > >
> > > > > > > > > As for the second part -- I think it makes sense to have
> some
> > > > sort
> > > > > of
> > > > > > > > > "sentinel" epoch to signal epoch is about to overflow (I
> > think
> > > we
> > > > > sort
> > > > > > > of
> > > > > > > > > have this value in place in some ways) so we can codify it
> in
> > > the
> > > > > KIP.
> > > > > > > I'll
> > > > > > > > > look into that and try to update soon.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Justine.
> > > > > > > > >
> > > > > > > > > On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
> > > > > > > > > <al...@confluent.io.invalid> wrote:
> > > > > > > > >
> > > > > > > > >> It's good to know that KIP-588 addressed some of the
> issues.
> > > > > Looking
> > > > > > > at
> > > > > > > > >> the code, it still looks like there are some cases that
> > would
> > > > > result
> > > > > > > in
> > > > > > > > >> fatal error, e.g. PRODUCER_FENCED is issued by the
> > transaction
> > > > > > > coordinator
> > > > > > > > >> if epoch doesn't match, and the client treats it as a
> fatal
> > > > error
> > > > > > > (code in
> > > > > > > > >> TransactionManager request handling).  If we consider, for
> > > > > example,
> > > > > > > > >> committing a transaction that returns a timeout, but
> > actually
> > > > > > > succeeds,
> > > > > > > > >> trying to abort it or re-commit may result in
> > PRODUCER_FENCED
> > > > > error
> > > > > > > > >> (because of epoch bump).
> > > > > > > > >>
> > > > > > > > >> For failed commits, specifically, we need to know the
> actual
> > > > > outcome,
> > > > > > > > >> because if we return an error the application may think
> that
> > > the
> > > > > > > > >> transaction is aborted and redo the work, leading to
> > > duplicates.
> > > > > > > > >>
> > > > > > > > >> Re: overflowing epoch.  We could either do it on the TC
> and
> > > > return
> > > > > > > both
> > > > > > > > >> producer id and epoch (e.g. change the protocol), or
> signal
> > > the
> > > > > client
> > > > > > > > >> that
> > > > > > > > >> it needs to get a new producer id.  Checking for max epoch
> > > could
> > > > > be a
> > > > > > > > >> reasonable signal, the value to check should probably be
> > > present
> > > > > in
> > > > > > > the
> > > > > > > > >> KIP
> > > > > > > > >> as this is effectively a part of the contract.  Also, the
> TC
> > > > > should
> > > > > > > > >> probably return an error if the client didn't change
> > producer
> > > id
> > > > > after
> > > > > > > > >> hitting max epoch.
> > > > > > > > >>
> > > > > > > > >> -Artem
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
> > > > > > > > >> <jo...@confluent.io.invalid> wrote:
> > > > > > > > >>
> > > > > > > > >> > Thanks for the discussion Artem.
> > > > > > > > >> >
> > > > > > > > >> > With respect to the handling of fenced producers, we
> have
> > > some
> > > > > > > behavior
> > > > > > > > >> > already in place. As of KIP-588:
> > > > > > > > >> >
> > > > > > > > >> >
> > > > > > > > >>
> > > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> > > > > > > > >> > ,
> > > > > > > > >> > we handle timeouts more gracefully. The producer can
> > > recover.
> > > > > > > > >> >
> > > > > > > > >> > Produce requests can also recover from epoch fencing by
> > > > > aborting the
> > > > > > > > >> > transaction and starting over.
> > > > > > > > >> >
> > > > > > > > >> > What other cases were you considering that would cause
> us
> > to
> > > > > have a
> > > > > > > > >> fenced
> > > > > > > > >> > epoch but we'd want to recover?
> > > > > > > > >> >
> > > > > > > > >> > The first point about handling epoch overflows is fair.
> I
> > > > think
> > > > > > > there is
> > > > > > > > >> > some logic we'd need to consider. (ie, if we are one
> away
> > > from
> > > > > the
> > > > > > > max
> > > > > > > > >> > epoch, we need to reset the producer ID.) I'm still
> > > wondering
> > > > if
> > > > > > > there
> > > > > > > > >> is a
> > > > > > > > >> > way to direct this from the response, or if everything
> > > should
> > > > be
> > > > > > > done on
> > > > > > > > >> > the client side. Let me know if you have any thoughts
> > here.
> > > > > > > > >> >
> > > > > > > > >> > Thanks,
> > > > > > > > >> > Justine
> > > > > > > > >> >
> > > > > > > > >> > On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
> > > > > > > > >> > <al...@confluent.io.invalid> wrote:
> > > > > > > > >> >
> > > > > > > > >> > > There are some workflows in the client that are
> implied
> > by
> > > > > > > protocol
> > > > > > > > >> > > changes, e.g.:
> > > > > > > > >> > >
> > > > > > > > >> > > - for new clients, epoch changes with every
> transaction
> > > and
> > > > > can
> > > > > > > > >> overflow,
> > > > > > > > >> > > in old clients this condition was handled
> transparently,
> > > > > because
> > > > > > > epoch
> > > > > > > > >> > was
> > > > > > > > >> > > bumped in InitProducerId and it would return a new
> > > producer
> > > > > id if
> > > > > > > > >> epoch
> > > > > > > > >> > > overflows, the new clients would need to implement
> some
> > > > > workflow
> > > > > > > to
> > > > > > > > >> > refresh
> > > > > > > > >> > > producer id
> > > > > > > > >> > > - how to handle fenced producers, for new clients
> epoch
> > > > > changes
> > > > > > > with
> > > > > > > > >> > every
> > > > > > > > >> > > transaction, so in presence of failures during
> commits /
> > > > > aborts,
> > > > > > > the
> > > > > > > > >> > > producer could get easily fenced, old clients would
> > pretty
> > > > > much
> > > > > > > would
> > > > > > > > >> get
> > > > > > > > >> > > fenced when a new incarnation of the producer was
> > > > initialized
> > > > > with
> > > > > > > > >> > > InitProducerId so it's ok to treat as a fatal error,
> the
> > > new
> > > > > > > clients
> > > > > > > > >> > would
> > > > > > > > >> > > need to implement some workflow to handle that error,
> > > > > otherwise
> > > > > > > they
> > > > > > > > >> > could
> > > > > > > > >> > > get fenced by themselves
> > > > > > > > >> > > - in particular (as a subset of the previous issue),
> > what
> > > > > would
> > > > > > > the
> > > > > > > > >> > client
> > > > > > > > >> > > do if it got a timeout during commit?  commit could've
> > > > > succeeded
> > > > > > > or
> > > > > > > > >> > failed
> > > > > > > > >> > >
> > > > > > > > >> > > Not sure if this has to be defined in the KIP as
> > > > implementing
> > > > > > > those
> > > > > > > > >> > > probably wouldn't require protocol changes, but we
> have
> > > > > multiple
> > > > > > > > >> > > implementations of Kafka clients, so probably would be
> > > good
> > > > to
> > > > > > > have
> > > > > > > > >> some
> > > > > > > > >> > > client implementation guidance.  Could also be done
> as a
> > > > > separate
> > > > > > > doc.
> > > > > > > > >> > >
> > > > > > > > >> > > -Artem
> > > > > > > > >> > >
> > > > > > > > >> > > On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
> > > > > > > > >> > <jolshan@confluent.io.invalid
> > > > > > > > >> > > >
> > > > > > > > >> > > wrote:
> > > > > > > > >> > >
> > > > > > > > >> > > > Hey all, I've updated the KIP to incorporate Jason's
> > > > > > > suggestions.
> > > > > > > > >> > > >
> > > > > > > > >> > > >
> > > > > > > > >> > > >
> > > > > > > > >> > >
> > > > > > > > >> >
> > > > > > > > >>
> > > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > > > > >> > > >
> > > > > > > > >> > > >
> > > > > > > > >> > > > 1. Use AddPartitionsToTxn + verify flag to check on
> > old
> > > > > clients
> > > > > > > > >> > > > 2. Updated AddPartitionsToTxn API to support
> > transaction
> > > > > > > batching
> > > > > > > > >> > > > 3. Mention IBP bump
> > > > > > > > >> > > > 4. Mention auth change on new AddPartitionsToTxn
> > > version.
> > > > > > > > >> > > >
> > > > > > > > >> > > > I'm planning on opening a vote soon.
> > > > > > > > >> > > > Thanks,
> > > > > > > > >> > > > Justine
> > > > > > > > >> > > >
> > > > > > > > >> > > > On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <
> > > > > > > jolshan@confluent.io
> > > > > > > > >> >
> > > > > > > > >> > > > wrote:
> > > > > > > > >> > > >
> > > > > > > > >> > > > > Thanks Jason. Those changes make sense to me. I
> will
> > > > > update
> > > > > > > the
> > > > > > > > >> KIP.
> > > > > > > > >> > > > >
> > > > > > > > >> > > > >
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
> > > > > > > > >> > > > <ja...@confluent.io.invalid>
> > > > > > > > >> > > > > wrote:
> > > > > > > > >> > > > >
> > > > > > > > >> > > > >> Hey Justine,
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> > I was wondering about compatibility here. When
> we
> > > > send
> > > > > > > requests
> > > > > > > > >> > > > >> between brokers, we want to ensure that the
> > receiving
> > > > > broker
> > > > > > > > >> > > understands
> > > > > > > > >> > > > >> the request (specifically the new fields).
> > Typically
> > > > > this is
> > > > > > > done
> > > > > > > > >> > via
> > > > > > > > >> > > > >> IBP/metadata version.
> > > > > > > > >> > > > >> I'm trying to think if there is a way around it
> but
> > > I'm
> > > > > not
> > > > > > > sure
> > > > > > > > >> > there
> > > > > > > > >> > > > is.
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> Yes. I think we would gate usage of this behind
> an
> > > IBP
> > > > > bump.
> > > > > > > Does
> > > > > > > > >> > that
> > > > > > > > >> > > > >> seem
> > > > > > > > >> > > > >> reasonable?
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> > As for the improvements -- can you clarify how
> > the
> > > > > multiple
> > > > > > > > >> > > > >> transactional
> > > > > > > > >> > > > >> IDs would help here? Were you thinking of a case
> > > where
> > > > we
> > > > > > > > >> wait/batch
> > > > > > > > >> > > > >> multiple produce requests together? My
> > understanding
> > > > for
> > > > > now
> > > > > > > was
> > > > > > > > >> 1
> > > > > > > > >> > > > >> transactional ID and one validation per 1 produce
> > > > > request.
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> Each call to `AddPartitionsToTxn` is essentially
> a
> > > > write
> > > > > to
> > > > > > > the
> > > > > > > > >> > > > >> transaction
> > > > > > > > >> > > > >> log and must block on replication. The more we
> can
> > > fit
> > > > > into a
> > > > > > > > >> single
> > > > > > > > >> > > > >> request, the more writes we can do in parallel.
> The
> > > > > > > alternative
> > > > > > > > >> is
> > > > > > > > >> > to
> > > > > > > > >> > > > make
> > > > > > > > >> > > > >> use of more connections, but usually we prefer
> > > batching
> > > > > > > since the
> > > > > > > > >> > > > network
> > > > > > > > >> > > > >> stack is not really optimized for high
> > > > connection/request
> > > > > > > loads.
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> > Finally with respect to the authorizations, I
> > think
> > > > it
> > > > > > > makes
> > > > > > > > >> sense
> > > > > > > > >> > > to
> > > > > > > > >> > > > >> skip
> > > > > > > > >> > > > >> topic authorizations, but I'm a bit confused by
> the
> > > > > "leader
> > > > > > > ID"
> > > > > > > > >> > field.
> > > > > > > > >> > > > >> Wouldn't we just want to flag the request as
> from a
> > > > > broker
> > > > > > > (does
> > > > > > > > >> it
> > > > > > > > >> > > > matter
> > > > > > > > >> > > > >> which one?).
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> We could also make it version-based. For the next
> > > > > version, we
> > > > > > > > >> could
> > > > > > > > >> > > > >> require
> > > > > > > > >> > > > >> CLUSTER auth. So clients would not be able to use
> > the
> > > > API
> > > > > > > > >> anymore,
> > > > > > > > >> > > which
> > > > > > > > >> > > > >> is
> > > > > > > > >> > > > >> probably what we want.
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> -Jason
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> > > > > > > > >> > > > >> <jo...@confluent.io.invalid>
> > > > > > > > >> > > > >> wrote:
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> > As a follow up, I was just thinking about the
> > > > batching
> > > > > a
> > > > > > > bit
> > > > > > > > >> more.
> > > > > > > > >> > > > >> > I suppose if we have one request in flight and
> we
> > > > > queue up
> > > > > > > the
> > > > > > > > >> > other
> > > > > > > > >> > > > >> > produce requests in some sort of purgatory, we
> > > could
> > > > > send
> > > > > > > > >> > > information
> > > > > > > > >> > > > >> out
> > > > > > > > >> > > > >> > for all of them rather than one by one. So that
> > > would
> > > > > be a
> > > > > > > > >> benefit
> > > > > > > > >> > > of
> > > > > > > > >> > > > >> > batching partitions to add per transaction.
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >> > I'll need to think a bit more on the design of
> > this
> > > > > part
> > > > > > > of the
> > > > > > > > >> > KIP,
> > > > > > > > >> > > > and
> > > > > > > > >> > > > >> > will update the KIP in the next few days.
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >> > Thanks,
> > > > > > > > >> > > > >> > Justine
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >> > On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan
> <
> > > > > > > > >> > > jolshan@confluent.io>
> > > > > > > > >> > > > >> > wrote:
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >> > > Hey Jason -- thanks for the input -- I was
> just
> > > > > digging
> > > > > > > a bit
> > > > > > > > >> > > deeper
> > > > > > > > >> > > > >> into
> > > > > > > > >> > > > >> > > the design + implementation of the validation
> > > calls
> > > > > here
> > > > > > > and
> > > > > > > > >> > what
> > > > > > > > >> > > > you
> > > > > > > > >> > > > >> say
> > > > > > > > >> > > > >> > > makes sense.
> > > > > > > > >> > > > >> > >
> > > > > > > > >> > > > >> > > I was wondering about compatibility here.
> When
> > we
> > > > > send
> > > > > > > > >> requests
> > > > > > > > >> > > > >> > > between brokers, we want to ensure that the
> > > > receiving
> > > > > > > broker
> > > > > > > > >> > > > >> understands
> > > > > > > > >> > > > >> > > the request (specifically the new fields).
> > > > Typically
> > > > > > > this is
> > > > > > > > >> > done
> > > > > > > > >> > > > via
> > > > > > > > >> > > > >> > > IBP/metadata version.
> > > > > > > > >> > > > >> > > I'm trying to think if there is a way around
> it
> > > but
> > > > > I'm
> > > > > > > not
> > > > > > > > >> sure
> > > > > > > > >> > > > there
> > > > > > > > >> > > > >> > is.
> > > > > > > > >> > > > >> > >
> > > > > > > > >> > > > >> > > As for the improvements -- can you clarify
> how
> > > the
> > > > > > > multiple
> > > > > > > > >> > > > >> transactional
> > > > > > > > >> > > > >> > > IDs would help here? Were you thinking of a
> > case
> > > > > where we
> > > > > > > > >> > > wait/batch
> > > > > > > > >> > > > >> > > multiple produce requests together? My
> > > > understanding
> > > > > for
> > > > > > > now
> > > > > > > > >> > was 1
> > > > > > > > >> > > > >> > > transactional ID and one validation per 1
> > produce
> > > > > > > request.
> > > > > > > > >> > > > >> > >
> > > > > > > > >> > > > >> > > Finally with respect to the authorizations, I
> > > think
> > > > > it
> > > > > > > makes
> > > > > > > > >> > sense
> > > > > > > > >> > > > to
> > > > > > > > >> > > > >> > skip
> > > > > > > > >> > > > >> > > topic authorizations, but I'm a bit confused
> by
> > > the
> > > > > > > "leader
> > > > > > > > >> ID"
> > > > > > > > >> > > > field.
> > > > > > > > >> > > > >> > > Wouldn't we just want to flag the request as
> > > from a
> > > > > > > broker
> > > > > > > > >> (does
> > > > > > > > >> > > it
> > > > > > > > >> > > > >> > matter
> > > > > > > > >> > > > >> > > which one?).
> > > > > > > > >> > > > >> > >
> > > > > > > > >> > > > >> > > I think I want to adopt these suggestions,
> just
> > > had
> > > > > a few
> > > > > > > > >> > > questions
> > > > > > > > >> > > > on
> > > > > > > > >> > > > >> > the
> > > > > > > > >> > > > >> > > details.
> > > > > > > > >> > > > >> > >
> > > > > > > > >> > > > >> > > Thanks,
> > > > > > > > >> > > > >> > > Justine
> > > > > > > > >> > > > >> > >
> > > > > > > > >> > > > >> > > On Thu, Jan 5, 2023 at 5:05 PM Jason
> Gustafson
> > > > > > > > >> > > > >> > <ja...@confluent.io.invalid>
> > > > > > > > >> > > > >> > > wrote:
> > > > > > > > >> > > > >> > >
> > > > > > > > >> > > > >> > >> Hi Justine,
> > > > > > > > >> > > > >> > >>
> > > > > > > > >> > > > >> > >> Thanks for the proposal.
> > > > > > > > >> > > > >> > >>
> > > > > > > > >> > > > >> > >> I was thinking about the implementation a
> > little
> > > > > bit.
> > > > > > > In the
> > > > > > > > >> > > > current
> > > > > > > > >> > > > >> > >> proposal, the behavior depends on whether we
> > > have
> > > > an
> > > > > > > old or
> > > > > > > > >> new
> > > > > > > > >> > > > >> client.
> > > > > > > > >> > > > >> > >> For
> > > > > > > > >> > > > >> > >> old clients, we send `DescribeTransactions`
> > and
> > > > > verify
> > > > > > > the
> > > > > > > > >> > result
> > > > > > > > >> > > > and
> > > > > > > > >> > > > >> > for
> > > > > > > > >> > > > >> > >> new clients, we send `AddPartitionsToTxn`.
> We
> > > > might
> > > > > be
> > > > > > > able
> > > > > > > > >> to
> > > > > > > > >> > > > >> simplify
> > > > > > > > >> > > > >> > >> the
> > > > > > > > >> > > > >> > >> implementation if we can use the same
> request
> > > > type.
> > > > > For
> > > > > > > > >> > example,
> > > > > > > > >> > > > >> what if
> > > > > > > > >> > > > >> > >> we
> > > > > > > > >> > > > >> > >> bump the protocol version for
> > > `AddPartitionsToTxn`
> > > > > and
> > > > > > > add a
> > > > > > > > >> > > > >> > >> `validateOnly`
> > > > > > > > >> > > > >> > >> flag? For older versions, we can set
> > > > > > > `validateOnly=true` so
> > > > > > > > >> > that
> > > > > > > > >> > > > the
> > > > > > > > >> > > > >> > >> request only returns successfully if the
> > > partition
> > > > > had
> > > > > > > > >> already
> > > > > > > > >> > > been
> > > > > > > > >> > > > >> > added.
> > > > > > > > >> > > > >> > >> For new versions, we can set
> > > `validateOnly=false`
> > > > > and
> > > > > > > the
> > > > > > > > >> > > partition
> > > > > > > > >> > > > >> will
> > > > > > > > >> > > > >> > >> be
> > > > > > > > >> > > > >> > >> added to the transaction. The other slightly
> > > > > annoying
> > > > > > > thing
> > > > > > > > >> > that
> > > > > > > > >> > > > this
> > > > > > > > >> > > > >> > >> would
> > > > > > > > >> > > > >> > >> get around is the need to collect the
> > > transaction
> > > > > state
> > > > > > > for
> > > > > > > > >> all
> > > > > > > > >> > > > >> > partitions
> > > > > > > > >> > > > >> > >> even when we only care about a subset.
> > > > > > > > >> > > > >> > >>
> > > > > > > > >> > > > >> > >> Some additional improvements to consider:
> > > > > > > > >> > > > >> > >>
> > > > > > > > >> > > > >> > >> - We can give `AddPartitionsToTxn` better
> > batch
> > > > > support
> > > > > > > for
> > > > > > > > >> > > > >> inter-broker
> > > > > > > > >> > > > >> > >> usage. Currently we only allow one
> > > > > `TransactionalId` to
> > > > > > > be
> > > > > > > > >> > > > specified,
> > > > > > > > >> > > > >> > but
> > > > > > > > >> > > > >> > >> the broker may get some benefit being able
> to
> > > > batch
> > > > > > > across
> > > > > > > > >> > > multiple
> > > > > > > > >> > > > >> > >> transactions.
> > > > > > > > >> > > > >> > >> - Another small improvement is skipping
> topic
> > > > > > > authorization
> > > > > > > > >> > > checks
> > > > > > > > >> > > > >> for
> > > > > > > > >> > > > >> > >> `AddPartitionsToTxn` when the request is
> from
> > a
> > > > > broker.
> > > > > > > > >> Perhaps
> > > > > > > > >> > > we
> > > > > > > > >> > > > >> can
> > > > > > > > >> > > > >> > add
> > > > > > > > >> > > > >> > >> a field for the `LeaderId` or something like
> > > that
> > > > > and
> > > > > > > > >> require
> > > > > > > > >> > > > CLUSTER
> > > > > > > > >> > > > >> > >> permission when set.
> > > > > > > > >> > > > >> > >>
> > > > > > > > >> > > > >> > >> Best,
> > > > > > > > >> > > > >> > >> Jason
> > > > > > > > >> > > > >> > >>
> > > > > > > > >> > > > >> > >>
> > > > > > > > >> > > > >> > >>
> > > > > > > > >> > > > >> > >> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
> > > > > > > > >> > <jun@confluent.io.invalid
> > > > > > > > >> > > >
> > > > > > > > >> > > > >> > wrote:
> > > > > > > > >> > > > >> > >>
> > > > > > > > >> > > > >> > >> > Hi, Justine,
> > > > > > > > >> > > > >> > >> >
> > > > > > > > >> > > > >> > >> > Thanks for the explanation. It makes sense
> > to
> > > me
> > > > > now.
> > > > > > > > >> > > > >> > >> >
> > > > > > > > >> > > > >> > >> > Jun
> > > > > > > > >> > > > >> > >> >
> > > > > > > > >> > > > >> > >> > On Mon, Dec 19, 2022 at 1:42 PM Justine
> > Olshan
> > > > > > > > >> > > > >> > >> > <jo...@confluent.io.invalid>
> > > > > > > > >> > > > >> > >> > wrote:
> > > > > > > > >> > > > >> > >> >
> > > > > > > > >> > > > >> > >> > > Hi Jun,
> > > > > > > > >> > > > >> > >> > >
> > > > > > > > >> > > > >> > >> > > My understanding of the mechanism is
> that
> > > when
> > > > > we
> > > > > > > get to
> > > > > > > > >> > the
> > > > > > > > >> > > > last
> > > > > > > > >> > > > >> > >> epoch,
> > > > > > > > >> > > > >> > >> > we
> > > > > > > > >> > > > >> > >> > > increment to the fencing/last epoch and
> if
> > > any
> > > > > > > further
> > > > > > > > >> > > requests
> > > > > > > > >> > > > >> come
> > > > > > > > >> > > > >> > >> in
> > > > > > > > >> > > > >> > >> > for
> > > > > > > > >> > > > >> > >> > > this producer ID they are fenced. Then
> the
> > > > > producer
> > > > > > > > >> gets a
> > > > > > > > >> > > new
> > > > > > > > >> > > > ID
> > > > > > > > >> > > > >> > and
> > > > > > > > >> > > > >> > >> > > restarts with epoch/sequence 0. The
> fenced
> > > > epoch
> > > > > > > sticks
> > > > > > > > >> > > around
> > > > > > > > >> > > > >> for
> > > > > > > > >> > > > >> > the
> > > > > > > > >> > > > >> > >> > > duration of producer.id.expiration.ms
> and
> > > > > blocks
> > > > > > > any
> > > > > > > > >> late
> > > > > > > > >> > > > >> messages
> > > > > > > > >> > > > >> > >> > there.
> > > > > > > > >> > > > >> > >> > > The new ID will get to take advantage of
> > the
> > > > > > > improved
> > > > > > > > >> > > semantics
> > > > > > > > >> > > > >> > around
> > > > > > > > >> > > > >> > >> > > non-zero start sequences. So I think we
> > are
> > > > > covered.
> > > > > > > > >> > > > >> > >> > >
> > > > > > > > >> > > > >> > >> > > The only potential issue is overloading
> > the
> > > > > cache,
> > > > > > > but
> > > > > > > > >> > > > hopefully
> > > > > > > > >> > > > >> the
> > > > > > > > >> > > > >> > >> > > improvements (lowered
> > > > producer.id.expiration.ms
> > > > > )
> > > > > > > will
> > > > > > > > >> help
> > > > > > > > >> > > > with
> > > > > > > > >> > > > >> > that.
> > > > > > > > >> > > > >> > >> > Let
> > > > > > > > >> > > > >> > >> > > me know if you still have concerns.
> > > > > > > > >> > > > >> > >> > >
> > > > > > > > >> > > > >> > >> > > Thanks,
> > > > > > > > >> > > > >> > >> > > Justine
> > > > > > > > >> > > > >> > >> > >
> > > > > > > > >> > > > >> > >> > > On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
> > > > > > > > >> > > > >> <ju...@confluent.io.invalid>
> > > > > > > > >> > > > >> > >> > wrote:
> > > > > > > > >> > > > >> > >> > >
> > > > > > > > >> > > > >> > >> > > > Hi, Justine,
> > > > > > > > >> > > > >> > >> > > >
> > > > > > > > >> > > > >> > >> > > > Thanks for the explanation.
> > > > > > > > >> > > > >> > >> > > >
> > > > > > > > >> > > > >> > >> > > > 70. The proposed fencing logic doesn't
> > > apply
> > > > > when
> > > > > > > pid
> > > > > > > > >> > > > changes,
> > > > > > > > >> > > > >> is
> > > > > > > > >> > > > >> > >> that
> > > > > > > > >> > > > >> > >> > > > right? If so, I am not sure how
> complete
> > > we
> > > > > are
> > > > > > > > >> > addressing
> > > > > > > > >> > > > this
> > > > > > > > >> > > > >> > >> issue
> > > > > > > > >> > > > >> > >> > if
> > > > > > > > >> > > > >> > >> > > > the pid changes more frequently.
> > > > > > > > >> > > > >> > >> > > >
> > > > > > > > >> > > > >> > >> > > > Thanks,
> > > > > > > > >> > > > >> > >> > > >
> > > > > > > > >> > > > >> > >> > > > Jun
> > > > > > > > >> > > > >> > >> > > >
> > > > > > > > >> > > > >> > >> > > >
> > > > > > > > >> > > > >> > >> > > >
> > > > > > > > >> > > > >> > >> > > > On Fri, Dec 16, 2022 at 9:16 AM
> Justine
> > > > Olshan
> > > > > > > > >> > > > >> > >> > > > <jo...@confluent.io.invalid>
> > > > > > > > >> > > > >> > >> > > > wrote:
> > > > > > > > >> > > > >> > >> > > >
> > > > > > > > >> > > > >> > >> > > > > Hi Jun,
> > > > > > > > >> > > > >> > >> > > > >
> > > > > > > > >> > > > >> > >> > > > > Thanks for replying!
> > > > > > > > >> > > > >> > >> > > > >
> > > > > > > > >> > > > >> > >> > > > > 70.We already do the overflow
> > mechanism,
> > > > so
> > > > > my
> > > > > > > > >> change
> > > > > > > > >> > > would
> > > > > > > > >> > > > >> just
> > > > > > > > >> > > > >> > >> make
> > > > > > > > >> > > > >> > >> > > it
> > > > > > > > >> > > > >> > >> > > > > happen more often.
> > > > > > > > >> > > > >> > >> > > > > I was also not suggesting a new
> field
> > in
> > > > the
> > > > > > > log,
> > > > > > > > >> but
> > > > > > > > >> > in
> > > > > > > > >> > > > the
> > > > > > > > >> > > > >> > >> > response,
> > > > > > > > >> > > > >> > >> > > > > which would be gated by the client
> > > > version.
> > > > > > > Sorry if
> > > > > > > > >> > > > >> something
> > > > > > > > >> > > > >> > >> there
> > > > > > > > >> > > > >> > >> > is
> > > > > > > > >> > > > >> > >> > > > > unclear. I think we are starting to
> > > > diverge.
> > > > > > > > >> > > > >> > >> > > > > The goal of this KIP is to not
> change
> > to
> > > > the
> > > > > > > marker
> > > > > > > > >> > > format
> > > > > > > > >> > > > at
> > > > > > > > >> > > > >> > all.
> > > > > > > > >> > > > >> > >> > > > >
> > > > > > > > >> > > > >> > >> > > > > 71. Yes, I guess I was going under
> the
> > > > > > > assumption
> > > > > > > > >> that
> > > > > > > > >> > > the
> > > > > > > > >> > > > >> log
> > > > > > > > >> > > > >> > >> would
> > > > > > > > >> > > > >> > >> > > just
> > > > > > > > >> > > > >> > >> > > > > look at its last epoch and treat it
> as
> > > the
> > > > > > > current
> > > > > > > > >> > > epoch. I
> > > > > > > > >> > > > >> > >> suppose
> > > > > > > > >> > > > >> > >> > we
> > > > > > > > >> > > > >> > >> > > > can
> > > > > > > > >> > > > >> > >> > > > > have some special logic that if the
> > last
> > > > > epoch
> > > > > > > was
> > > > > > > > >> on a
> > > > > > > > >> > > > >> marker
> > > > > > > > >> > > > >> > we
> > > > > > > > >> > > > >> > >> > > > actually
> > > > > > > > >> > > > >> > >> > > > > expect the next epoch or something
> > like
> > > > > that. We
> > > > > > > > >> just
> > > > > > > > >> > > need
> > > > > > > > >> > > > to
> > > > > > > > >> > > > >> > >> > > distinguish
> > > > > > > > >> > > > >> > >> > > > > based on whether we had a
> commit/abort
> > > > > marker.
> > > > > > > > >> > > > >> > >> > > > >
> > > > > > > > >> > > > >> > >> > > > > 72.
> > > > > > > > >> > > > >> > >> > > > > > if the producer epoch hasn't been
> > > bumped
> > > > > on
> > > > > > > the
> > > > > > > > >> > > > >> > >> > > > > broker, it seems that the stucked
> > > message
> > > > > will
> > > > > > > fail
> > > > > > > > >> the
> > > > > > > > >> > > > >> sequence
> > > > > > > > >> > > > >> > >> > > > validation
> > > > > > > > >> > > > >> > >> > > > > and will be ignored. If the producer
> > > epoch
> > > > > has
> > > > > > > been
> > > > > > > > >> > > bumped,
> > > > > > > > >> > > > >> we
> > > > > > > > >> > > > >> > >> ignore
> > > > > > > > >> > > > >> > >> > > the
> > > > > > > > >> > > > >> > >> > > > > sequence check and the stuck message
> > > could
> > > > > be
> > > > > > > > >> appended
> > > > > > > > >> > to
> > > > > > > > >> > > > the
> > > > > > > > >> > > > >> > log.
> > > > > > > > >> > > > >> > >> > So,
> > > > > > > > >> > > > >> > >> > > is
> > > > > > > > >> > > > >> > >> > > > > the latter case that we want to
> guard?
> > > > > > > > >> > > > >> > >> > > > >
> > > > > > > > >> > > > >> > >> > > > > I'm not sure I follow that "the
> > message
> > > > will
> > > > > > > fail
> > > > > > > > >> the
> > > > > > > > >> > > > >> sequence
> > > > > > > > >> > > > >> > >> > > > validation".
> > > > > > > > >> > > > >> > >> > > > > In some of these cases, we had an
> > abort
> > > > > marker
> > > > > > > (due
> > > > > > > > >> to
> > > > > > > > >> > an
> > > > > > > > >> > > > >> error)
> > > > > > > > >> > > > >> > >> and
> > > > > > > > >> > > > >> > >> > > then
> > > > > > > > >> > > > >> > >> > > > > the late message comes in with the
> > > correct
> > > > > > > sequence
> > > > > > > > >> > > number.
> > > > > > > > >> > > > >> This
> > > > > > > > >> > > > >> > >> is a
> > > > > > > > >> > > > >> > >> > > > case
> > > > > > > > >> > > > >> > >> > > > > covered by the KIP.
> > > > > > > > >> > > > >> > >> > > > > The latter case is actually not
> > > something
> > > > > we've
> > > > > > > > >> > > considered
> > > > > > > > >> > > > >> > here. I
> > > > > > > > >> > > > >> > >> > > think
> > > > > > > > >> > > > >> > >> > > > > generally when we bump the epoch, we
> > are
> > > > > > > accepting
> > > > > > > > >> that
> > > > > > > > >> > > the
> > > > > > > > >> > > > >> > >> sequence
> > > > > > > > >> > > > >> > >> > > does
> > > > > > > > >> > > > >> > >> > > > > not need to be checked anymore. My
> > > > > > > understanding is
> > > > > > > > >> > also
> > > > > > > > >> > > > >> that we
> > > > > > > > >> > > > >> > >> > don't
> > > > > > > > >> > > > >> > >> > > > > typically bump epoch mid transaction
> > > > (based
> > > > > on a
> > > > > > > > >> quick
> > > > > > > > >> > > look
> > > > > > > > >> > > > >> at
> > > > > > > > >> > > > >> > the
> > > > > > > > >> > > > >> > >> > > code)
> > > > > > > > >> > > > >> > >> > > > > but let me know if that is the case.
> > > > > > > > >> > > > >> > >> > > > >
> > > > > > > > >> > > > >> > >> > > > > Thanks,
> > > > > > > > >> > > > >> > >> > > > > Justine
> > > > > > > > >> > > > >> > >> > > > >
> > > > > > > > >> > > > >> > >> > > > > On Thu, Dec 15, 2022 at 12:23 PM Jun
> > Rao
> > > > > > > > >> > > > >> > <jun@confluent.io.invalid
> > > > > > > > >> > > > >> > >> >
> > > > > > > > >> > > > >> > >> > > > wrote:
> > > > > > > > >> > > > >> > >> > > > >
> > > > > > > > >> > > > >> > >> > > > > > Hi, Justine,
> > > > > > > > >> > > > >> > >> > > > > >
> > > > > > > > >> > > > >> > >> > > > > > Thanks for the reply.
> > > > > > > > >> > > > >> > >> > > > > >
> > > > > > > > >> > > > >> > >> > > > > > 70. Assigning a new pid on int
> > > overflow
> > > > > seems
> > > > > > > a
> > > > > > > > >> bit
> > > > > > > > >> > > > hacky.
> > > > > > > > >> > > > >> If
> > > > > > > > >> > > > >> > we
> > > > > > > > >> > > > >> > >> > > need a
> > > > > > > > >> > > > >> > >> > > > > txn
> > > > > > > > >> > > > >> > >> > > > > > level id, it will be better to
> model
> > > > this
> > > > > > > > >> explicitly.
> > > > > > > > >> > > > >> Adding a
> > > > > > > > >> > > > >> > >> new
> > > > > > > > >> > > > >> > >> > > > field
> > > > > > > > >> > > > >> > >> > > > > > would require a bit more work
> since
> > it
> > > > > > > requires a
> > > > > > > > >> new
> > > > > > > > >> > > txn
> > > > > > > > >> > > > >> > marker
> > > > > > > > >> > > > >> > >> > > format
> > > > > > > > >> > > > >> > >> > > > > in
> > > > > > > > >> > > > >> > >> > > > > > the log. So, we probably need to
> > guard
> > > > it
> > > > > > > with an
> > > > > > > > >> IBP
> > > > > > > > >> > > or
> > > > > > > > >> > > > >> > >> metadata
> > > > > > > > >> > > > >> > >> > > > version
> > > > > > > > >> > > > >> > >> > > > > > and document the impact on
> downgrade
> > > > once
> > > > > the
> > > > > > > new
> > > > > > > > >> > > format
> > > > > > > > >> > > > is
> > > > > > > > >> > > > >> > >> written
> > > > > > > > >> > > > >> > >> > > to
> > > > > > > > >> > > > >> > >> > > > > the
> > > > > > > > >> > > > >> > >> > > > > > log.
> > > > > > > > >> > > > >> > >> > > > > >
> > > > > > > > >> > > > >> > >> > > > > > 71. Hmm, once the marker is
> written,
> > > the
> > > > > > > partition
> > > > > > > > >> > will
> > > > > > > > >> > > > >> expect
> > > > > > > > >> > > > >> > >> the
> > > > > > > > >> > > > >> > >> > > next
> > > > > > > > >> > > > >> > >> > > > > > append to be on the next epoch.
> Does
> > > > that
> > > > > > > cover
> > > > > > > > >> the
> > > > > > > > >> > > case
> > > > > > > > >> > > > >> you
> > > > > > > > >> > > > >> > >> > > mentioned?
> > > > > > > > >> > > > >> > >> > > > > >
> > > > > > > > >> > > > >> > >> > > > > > 72. Also, just to be clear on the
> > > > stucked
> > > > > > > message
> > > > > > > > >> > issue
> > > > > > > > >> > > > >> > >> described
> > > > > > > > >> > > > >> > >> > in
> > > > > > > > >> > > > >> > >> > > > the
> > > > > > > > >> > > > >> > >> > > > > > motivation. With EoS, we also
> > validate
> > > > the
> > > > > > > > >> sequence
> > > > > > > > >> > id
> > > > > > > > >> > > > for
> > > > > > > > >> > > > >> > >> > > idempotency.
> > > > > > > > >> > > > >> > >> > > > > So,
> > > > > > > > >> > > > >> > >> > > > > > with the current logic, if the
> > > producer
> > > > > epoch
> > > > > > > > >> hasn't
> > > > > > > > >> > > been
> > > > > > > > >> > > > >> > >> bumped on
> > > > > > > > >> > > > >> > >> > > the
> > > > > > > > >> > > > >> > >> > > > > > broker, it seems that the stucked
> > > > message
> > > > > will
> > > > > > > > >> fail
> > > > > > > > >> > the
> > > > > > > > >> > > > >> > sequence
> > > > > > > > >> > > > >> > >> > > > > validation
> > > > > > > > >> > > > >> > >> > > > > > and will be ignored. If the
> producer
> > > > > epoch has
> > > > > > > > >> been
> > > > > > > > >> > > > >> bumped, we
> > > > > > > > >> > > > >> > >> > ignore
> > > > > > > > >> > > > >> > >> > > > the
> > > > > > > > >> > > > >> > >> > > > > > sequence check and the stuck
> message
> > > > > could be
> > > > > > > > >> > appended
> > > > > > > > >> > > to
> > > > > > > > >> > > > >> the
> > > > > > > > >> > > > >> > >> log.
> > > > > > > > >> > > > >> > >> > > So,
> > > > > > > > >> > > > >> > >> > > > is
> > > > > > > > >> > > > >> > >> > > > > > the latter case that we want to
> > guard?
> > > > > > > > >> > > > >> > >> > > > > >
> > > > > > > > >> > > > >> > >> > > > > > Thanks,
> > > > > > > > >> > > > >> > >> > > > > >
> > > > > > > > >> > > > >> > >> > > > > > Jun
> > > > > > > > >> > > > >> > >> > > > > >
> > > > > > > > >> > > > >> > >> > > > > > On Wed, Dec 14, 2022 at 10:44 AM
> > > Justine
> > > > > > > Olshan
> > > > > > > > >> > > > >> > >> > > > > > <jo...@confluent.io.invalid>
> > wrote:
> > > > > > > > >> > > > >> > >> > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > Matthias — thanks again for
> taking
> > > > time
> > > > > to
> > > > > > > look
> > > > > > > > >> a
> > > > > > > > >> > > this.
> > > > > > > > >> > > > >> You
> > > > > > > > >> > > > >> > >> said:
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > My proposal was only focusing
> to
> > > > avoid
> > > > > > > > >> dangling
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > transactions if records are
> added
> > > > > without
> > > > > > > > >> > registered
> > > > > > > > >> > > > >> > >> partition.
> > > > > > > > >> > > > >> > >> > --
> > > > > > > > >> > > > >> > >> > > > > Maybe
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > you can add a few more details
> to
> > > the
> > > > > KIP
> > > > > > > about
> > > > > > > > >> > this
> > > > > > > > >> > > > >> > scenario
> > > > > > > > >> > > > >> > >> for
> > > > > > > > >> > > > >> > >> > > > > better
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > documentation purpose?
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > I'm not sure I understand what
> you
> > > > mean
> > > > > > > here.
> > > > > > > > >> The
> > > > > > > > >> > > > >> motivation
> > > > > > > > >> > > > >> > >> > > section
> > > > > > > > >> > > > >> > >> > > > > > > describes two scenarios about
> how
> > > the
> > > > > record
> > > > > > > > >> can be
> > > > > > > > >> > > > added
> > > > > > > > >> > > > >> > >> > without a
> > > > > > > > >> > > > >> > >> > > > > > > registered partition:
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > 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.
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > Another way hanging
> transactions
> > > can
> > > > > > > occur is
> > > > > > > > >> > that
> > > > > > > > >> > > a
> > > > > > > > >> > > > >> > client
> > > > > > > > >> > > > >> > >> is
> > > > > > > > >> > > > >> > >> > > > buggy
> > > > > > > > >> > > > >> > >> > > > > > and
> > > > > > > > >> > > > >> > >> > > > > > > may somehow try to write to a
> > > > partition
> > > > > > > before
> > > > > > > > >> it
> > > > > > > > >> > > adds
> > > > > > > > >> > > > >> the
> > > > > > > > >> > > > >> > >> > > partition
> > > > > > > > >> > > > >> > >> > > > to
> > > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > transaction.
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > For the first example of this
> > would
> > > it
> > > > > be
> > > > > > > > >> helpful
> > > > > > > > >> > to
> > > > > > > > >> > > > say
> > > > > > > > >> > > > >> > that
> > > > > > > > >> > > > >> > >> > this
> > > > > > > > >> > > > >> > >> > > > > > message
> > > > > > > > >> > > > >> > >> > > > > > > comes in after the abort, but
> > before
> > > > the
> > > > > > > > >> partition
> > > > > > > > >> > is
> > > > > > > > >> > > > >> added
> > > > > > > > >> > > > >> > to
> > > > > > > > >> > > > >> > >> > the
> > > > > > > > >> > > > >> > >> > > > next
> > > > > > > > >> > > > >> > >> > > > > > > transaction so it becomes
> > "hanging."
> > > > > > > Perhaps the
> > > > > > > > >> > next
> > > > > > > > >> > > > >> > sentence
> > > > > > > > >> > > > >> > >> > > > > describing
> > > > > > > > >> > > > >> > >> > > > > > > the message becoming part of the
> > > next
> > > > > > > > >> transaction
> > > > > > > > >> > (a
> > > > > > > > >> > > > >> > different
> > > > > > > > >> > > > >> > >> > > case)
> > > > > > > > >> > > > >> > >> > > > > was
> > > > > > > > >> > > > >> > >> > > > > > > not properly differentiated.
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > Jun — thanks for reading the
> KIP.
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > 70. The int typing was a
> concern.
> > > > > Currently
> > > > > > > we
> > > > > > > > >> > have a
> > > > > > > > >> > > > >> > >> mechanism
> > > > > > > > >> > > > >> > >> > in
> > > > > > > > >> > > > >> > >> > > > > place
> > > > > > > > >> > > > >> > >> > > > > > to
> > > > > > > > >> > > > >> > >> > > > > > > fence the final epoch when the
> > epoch
> > > > is
> > > > > > > about to
> > > > > > > > >> > > > overflow
> > > > > > > > >> > > > >> > and
> > > > > > > > >> > > > >> > >> > > assign
> > > > > > > > >> > > > >> > >> > > > a
> > > > > > > > >> > > > >> > >> > > > > > new
> > > > > > > > >> > > > >> > >> > > > > > > producer ID with epoch 0. Of
> > course,
> > > > > this
> > > > > > > is a
> > > > > > > > >> bit
> > > > > > > > >> > > > tricky
> > > > > > > > >> > > > >> > >> when it
> > > > > > > > >> > > > >> > >> > > > comes
> > > > > > > > >> > > > >> > >> > > > > > to
> > > > > > > > >> > > > >> > >> > > > > > > the response back to the client.
> > > > > > > > >> > > > >> > >> > > > > > > Making this a long could be
> > another
> > > > > option,
> > > > > > > but
> > > > > > > > >> I
> > > > > > > > >> > > > wonder
> > > > > > > > >> > > > >> are
> > > > > > > > >> > > > >> > >> > there
> > > > > > > > >> > > > >> > >> > > > any
> > > > > > > > >> > > > >> > >> > > > > > > implications on changing this
> > field
> > > if
> > > > > the
> > > > > > > > >> epoch is
> > > > > > > > >> > > > >> > persisted
> > > > > > > > >> > > > >> > >> to
> > > > > > > > >> > > > >> > >> > > > disk?
> > > > > > > > >> > > > >> > >> > > > > > I'd
> > > > > > > > >> > > > >> > >> > > > > > > need to check the usages.
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > 71.This was something Matthias
> > asked
> > > > > about
> > > > > > > as
> > > > > > > > >> > well. I
> > > > > > > > >> > > > was
> > > > > > > > >> > > > >> > >> > > > considering a
> > > > > > > > >> > > > >> > >> > > > > > > possible edge case where a
> produce
> > > > > request
> > > > > > > from
> > > > > > > > >> a
> > > > > > > > >> > new
> > > > > > > > >> > > > >> > >> transaction
> > > > > > > > >> > > > >> > >> > > > > somehow
> > > > > > > > >> > > > >> > >> > > > > > > gets sent right after the marker
> > is
> > > > > > > written, but
> > > > > > > > >> > > before
> > > > > > > > >> > > > >> the
> > > > > > > > >> > > > >> > >> > > producer
> > > > > > > > >> > > > >> > >> > > > is
> > > > > > > > >> > > > >> > >> > > > > > > alerted of the newly bumped
> epoch.
> > > In
> > > > > this
> > > > > > > > >> case, we
> > > > > > > > >> > > may
> > > > > > > > >> > > > >> > >> include
> > > > > > > > >> > > > >> > >> > > this
> > > > > > > > >> > > > >> > >> > > > > > record
> > > > > > > > >> > > > >> > >> > > > > > > when we don't want to. I suppose
> > we
> > > > > could
> > > > > > > try
> > > > > > > > >> to do
> > > > > > > > >> > > > >> > something
> > > > > > > > >> > > > >> > >> > > client
> > > > > > > > >> > > > >> > >> > > > > side
> > > > > > > > >> > > > >> > >> > > > > > > to bump the epoch after sending
> an
> > > > > endTxn as
> > > > > > > > >> well
> > > > > > > > >> > in
> > > > > > > > >> > > > this
> > > > > > > > >> > > > >> > >> > scenario
> > > > > > > > >> > > > >> > >> > > —
> > > > > > > > >> > > > >> > >> > > > > but
> > > > > > > > >> > > > >> > >> > > > > > I
> > > > > > > > >> > > > >> > >> > > > > > > wonder how it would work when
> the
> > > > > server is
> > > > > > > > >> > aborting
> > > > > > > > >> > > > >> based
> > > > > > > > >> > > > >> > on
> > > > > > > > >> > > > >> > >> a
> > > > > > > > >> > > > >> > >> > > > > > server-side
> > > > > > > > >> > > > >> > >> > > > > > > error. I could also be missing
> > > > > something and
> > > > > > > > >> this
> > > > > > > > >> > > > >> scenario
> > > > > > > > >> > > > >> > is
> > > > > > > > >> > > > >> > >> > > > actually
> > > > > > > > >> > > > >> > >> > > > > > not
> > > > > > > > >> > > > >> > >> > > > > > > possible.
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > Thanks again to everyone reading
> > and
> > > > > > > commenting.
> > > > > > > > >> > Let
> > > > > > > > >> > > me
> > > > > > > > >> > > > >> know
> > > > > > > > >> > > > >> > >> > about
> > > > > > > > >> > > > >> > >> > > > any
> > > > > > > > >> > > > >> > >> > > > > > > further questions or comments.
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > Justine
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > On Wed, Dec 14, 2022 at 9:41 AM
> > Jun
> > > > Rao
> > > > > > > > >> > > > >> > >> <jun@confluent.io.invalid
> > > > > > > > >> > > > >> > >> > >
> > > > > > > > >> > > > >> > >> > > > > > wrote:
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > Hi, Justine,
> > > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > Thanks for the KIP. A couple
> of
> > > > > comments.
> > > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > 70. Currently, the producer
> > epoch
> > > is
> > > > > an
> > > > > > > int.
> > > > > > > > >> I am
> > > > > > > > >> > > not
> > > > > > > > >> > > > >> sure
> > > > > > > > >> > > > >> > >> if
> > > > > > > > >> > > > >> > >> > > it's
> > > > > > > > >> > > > >> > >> > > > > > enough
> > > > > > > > >> > > > >> > >> > > > > > > > to accommodate all
> transactions
> > in
> > > > the
> > > > > > > > >> lifetime
> > > > > > > > >> > of
> > > > > > > > >> > > a
> > > > > > > > >> > > > >> > >> producer.
> > > > > > > > >> > > > >> > >> > > > Should
> > > > > > > > >> > > > >> > >> > > > > > we
> > > > > > > > >> > > > >> > >> > > > > > > > change that to a long or add a
> > new
> > > > > long
> > > > > > > field
> > > > > > > > >> > like
> > > > > > > > >> > > > >> txnId?
> > > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > 71. "it will write the prepare
> > > > commit
> > > > > > > message
> > > > > > > > >> > with
> > > > > > > > >> > > a
> > > > > > > > >> > > > >> > bumped
> > > > > > > > >> > > > >> > >> > epoch
> > > > > > > > >> > > > >> > >> > > > and
> > > > > > > > >> > > > >> > >> > > > > > > send
> > > > > > > > >> > > > >> > >> > > > > > > > WriteTxnMarkerRequests with
> the
> > > > bumped
> > > > > > > epoch."
> > > > > > > > >> > Hmm,
> > > > > > > > >> > > > the
> > > > > > > > >> > > > >> > >> epoch
> > > > > > > > >> > > > >> > >> > is
> > > > > > > > >> > > > >> > >> > > > > > > associated
> > > > > > > > >> > > > >> > >> > > > > > > > with the current txn right?
> So,
> > it
> > > > > seems
> > > > > > > > >> weird to
> > > > > > > > >> > > > >> write a
> > > > > > > > >> > > > >> > >> > commit
> > > > > > > > >> > > > >> > >> > > > > > message
> > > > > > > > >> > > > >> > >> > > > > > > > with a bumped epoch. Should we
> > > only
> > > > > bump
> > > > > > > up
> > > > > > > > >> the
> > > > > > > > >> > > epoch
> > > > > > > > >> > > > >> in
> > > > > > > > >> > > > >> > >> > > > > EndTxnResponse
> > > > > > > > >> > > > >> > >> > > > > > > and
> > > > > > > > >> > > > >> > >> > > > > > > > rename the field to sth like
> > > > > > > > >> nextProducerEpoch?
> > > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > Thanks,
> > > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > Jun
> > > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > On Mon, Dec 12, 2022 at 8:54
> PM
> > > > > Matthias
> > > > > > > J.
> > > > > > > > >> Sax <
> > > > > > > > >> > > > >> > >> > > mjsax@apache.org>
> > > > > > > > >> > > > >> > >> > > > > > > wrote:
> > > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > > Thanks for the background.
> > > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > > 20/30: SGTM. My proposal was
> > > only
> > > > > > > focusing
> > > > > > > > >> to
> > > > > > > > >> > > avoid
> > > > > > > > >> > > > >> > >> dangling
> > > > > > > > >> > > > >> > >> > > > > > > > > transactions if records are
> > > added
> > > > > > > without
> > > > > > > > >> > > > registered
> > > > > > > > >> > > > >> > >> > partition.
> > > > > > > > >> > > > >> > >> > > > --
> > > > > > > > >> > > > >> > >> > > > > > > Maybe
> > > > > > > > >> > > > >> > >> > > > > > > > > you can add a few more
> details
> > > to
> > > > > the
> > > > > > > KIP
> > > > > > > > >> about
> > > > > > > > >> > > > this
> > > > > > > > >> > > > >> > >> scenario
> > > > > > > > >> > > > >> > >> > > for
> > > > > > > > >> > > > >> > >> > > > > > > better
> > > > > > > > >> > > > >> > >> > > > > > > > > documentation purpose?
> > > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > > 40: I think you hit a fair
> > point
> > > > > about
> > > > > > > race
> > > > > > > > >> > > > >> conditions
> > > > > > > > >> > > > >> > or
> > > > > > > > >> > > > >> > >> > > client
> > > > > > > > >> > > > >> > >> > > > > bugs
> > > > > > > > >> > > > >> > >> > > > > > > > > (incorrectly not bumping the
> > > > > epoch). The
> > > > > > > > >> > > > >> > >> complexity/confusion
> > > > > > > > >> > > > >> > >> > > for
> > > > > > > > >> > > > >> > >> > > > > > using
> > > > > > > > >> > > > >> > >> > > > > > > > > the bumped epoch I see, is
> > > mainly
> > > > > for
> > > > > > > > >> internal
> > > > > > > > >> > > > >> > debugging,
> > > > > > > > >> > > > >> > >> ie,
> > > > > > > > >> > > > >> > >> > > > > > > inspecting
> > > > > > > > >> > > > >> > >> > > > > > > > > log segment dumps -- it
> seems
> > > > > harder to
> > > > > > > > >> reason
> > > > > > > > >> > > > about
> > > > > > > > >> > > > >> the
> > > > > > > > >> > > > >> > >> > system
> > > > > > > > >> > > > >> > >> > > > for
> > > > > > > > >> > > > >> > >> > > > > > us
> > > > > > > > >> > > > >> > >> > > > > > > > > humans. But if we get better
> > > > > > > guarantees, it
> > > > > > > > >> > would
> > > > > > > > >> > > > be
> > > > > > > > >> > > > >> > >> worth to
> > > > > > > > >> > > > >> > >> > > use
> > > > > > > > >> > > > >> > >> > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > bumped epoch.
> > > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > > 60: as I mentioned already,
> I
> > > > don't
> > > > > > > know the
> > > > > > > > >> > > broker
> > > > > > > > >> > > > >> > >> internals
> > > > > > > > >> > > > >> > >> > > to
> > > > > > > > >> > > > >> > >> > > > > > > provide
> > > > > > > > >> > > > >> > >> > > > > > > > > more input. So if nobody
> else
> > > > chimes
> > > > > > > in, we
> > > > > > > > >> > > should
> > > > > > > > >> > > > >> just
> > > > > > > > >> > > > >> > >> move
> > > > > > > > >> > > > >> > >> > > > > forward
> > > > > > > > >> > > > >> > >> > > > > > > > > with your proposal.
> > > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > > -Matthias
> > > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > > On 12/6/22 4:22 PM, Justine
> > > Olshan
> > > > > > > wrote:
> > > > > > > > >> > > > >> > >> > > > > > > > > > Hi all,
> > > > > > > > >> > > > >> > >> > > > > > > > > > After Artem's questions
> > about
> > > > > error
> > > > > > > > >> behavior,
> > > > > > > > >> > > > I've
> > > > > > > > >> > > > >> > >> > > re-evaluated
> > > > > > > > >> > > > >> > >> > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > > unknown producer ID
> > exception
> > > > and
> > > > > had
> > > > > > > some
> > > > > > > > >> > > > >> discussions
> > > > > > > > >> > > > >> > >> > > offline.
> > > > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > > > I think generally it makes
> > > sense
> > > > > to
> > > > > > > > >> simplify
> > > > > > > > >> > > > error
> > > > > > > > >> > > > >> > >> handling
> > > > > > > > >> > > > >> > >> > > in
> > > > > > > > >> > > > >> > >> > > > > > cases
> > > > > > > > >> > > > >> > >> > > > > > > > like
> > > > > > > > >> > > > >> > >> > > > > > > > > > this and the
> > > UNKNOWN_PRODUCER_ID
> > > > > error
> > > > > > > > >> has a
> > > > > > > > >> > > > pretty
> > > > > > > > >> > > > >> > long
> > > > > > > > >> > > > >> > >> > and
> > > > > > > > >> > > > >> > >> > > > > > > > complicated
> > > > > > > > >> > > > >> > >> > > > > > > > > > history. Because of this,
> I
> > > > > propose
> > > > > > > > >> adding a
> > > > > > > > >> > > new
> > > > > > > > >> > > > >> error
> > > > > > > > >> > > > >> > >> code
> > > > > > > > >> > > > >> > >> > > > > > > > > ABORTABLE_ERROR
> > > > > > > > >> > > > >> > >> > > > > > > > > > that when encountered by
> new
> > > > > clients
> > > > > > > > >> (gated
> > > > > > > > >> > by
> > > > > > > > >> > > > the
> > > > > > > > >> > > > >> > >> produce
> > > > > > > > >> > > > >> > >> > > > > request
> > > > > > > > >> > > > >> > >> > > > > > > > > version)
> > > > > > > > >> > > > >> > >> > > > > > > > > > will simply abort the
> > > > transaction.
> > > > > > > This
> > > > > > > > >> > allows
> > > > > > > > >> > > > the
> > > > > > > > >> > > > >> > >> server
> > > > > > > > >> > > > >> > >> > to
> > > > > > > > >> > > > >> > >> > > > have
> > > > > > > > >> > > > >> > >> > > > > > > some
> > > > > > > > >> > > > >> > >> > > > > > > > > say
> > > > > > > > >> > > > >> > >> > > > > > > > > > in whether the client
> aborts
> > > and
> > > > > makes
> > > > > > > > >> > handling
> > > > > > > > >> > > > >> much
> > > > > > > > >> > > > >> > >> > simpler.
> > > > > > > > >> > > > >> > >> > > > In
> > > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > > future, we can also use
> this
> > > > > error in
> > > > > > > > >> other
> > > > > > > > >> > > > >> situations
> > > > > > > > >> > > > >> > >> > where
> > > > > > > > >> > > > >> > >> > > we
> > > > > > > > >> > > > >> > >> > > > > > want
> > > > > > > > >> > > > >> > >> > > > > > > to
> > > > > > > > >> > > > >> > >> > > > > > > > > > abort the transactions. We
> > can
> > > > > even
> > > > > > > use on
> > > > > > > > >> > > other
> > > > > > > > >> > > > >> apis.
> > > > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > > > I've added this to the
> KIP.
> > > Let
> > > > me
> > > > > > > know if
> > > > > > > > >> > > there
> > > > > > > > >> > > > >> are
> > > > > > > > >> > > > >> > any
> > > > > > > > >> > > > >> > >> > > > > questions
> > > > > > > > >> > > > >> > >> > > > > > or
> > > > > > > > >> > > > >> > >> > > > > > > > > > issues.
> > > > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > > > Justine
> > > > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > > > On Fri, Dec 2, 2022 at
> 10:22
> > > AM
> > > > > > > Justine
> > > > > > > > >> > Olshan
> > > > > > > > >> > > <
> > > > > > > > >> > > > >> > >> > > > > > jolshan@confluent.io
> > > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > > wrote:
> > > > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > > >> Hey Matthias,
> > > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > > >> > > > >> > >> > > > > > > > > >> 20/30 — Maybe I also
> didn't
> > > > > express
> > > > > > > > >> myself
> > > > > > > > >> > > > >> clearly.
> > > > > > > > >> > > > >> > For
> > > > > > > > >> > > > >> > >> > > older
> > > > > > > > >> > > > >> > >> > > > > > > clients
> > > > > > > > >> > > > >> > >> > > > > > > > we
> > > > > > > > >> > > > >> > >> > > > > > > > > >> don't have a way to
> > > distinguish
> > > > > > > between a
> > > > > > > > >> > > > previous
> > > > > > > > >> > > > >> > and
> > > > > > > > >> > > > >> > >> the
> > > > > > > > >> > > > >> > >> > > > > current
> > > > > > > > >> > > > >> > >> > > > > > > > > >> transaction since we
> don't
> > > have
> > > > > the
> > > > > > > epoch
> > > > > > > > >> > > bump.
> > > > > > > > >> > > > >> This
> > > > > > > > >> > > > >> > >> means
> > > > > > > > >> > > > >> > >> > > > that
> > > > > > > > >> > > > >> > >> > > > > a
> > > > > > > > >> > > > >> > >> > > > > > > late
> > > > > > > > >> > > > >> > >> > > > > > > > > >> message from the previous
> > > > > transaction
> > > > > > > > >> may be
> > > > > > > > >> > > > >> added to
> > > > > > > > >> > > > >> > >> the
> > > > > > > > >> > > > >> > >> > > new
> > > > > > > > >> > > > >> > >> > > > > one.
> > > > > > > > >> > > > >> > >> > > > > > > > With
> > > > > > > > >> > > > >> > >> > > > > > > > > >> older clients — we can't
> > > > > guarantee
> > > > > > > this
> > > > > > > > >> > won't
> > > > > > > > >> > > > >> happen
> > > > > > > > >> > > > >> > >> if we
> > > > > > > > >> > > > >> > >> > > > > already
> > > > > > > > >> > > > >> > >> > > > > > > > sent
> > > > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call
> > (why
> > > we
> > > > > make
> > > > > > > > >> changes
> > > > > > > > >> > > for
> > > > > > > > >> > > > >> the
> > > > > > > > >> > > > >> > >> newer
> > > > > > > > >> > > > >> > >> > > > > client)
> > > > > > > > >> > > > >> > >> > > > > > > but
> > > > > > > > >> > > > >> > >> > > > > > > > > we
> > > > > > > > >> > > > >> > >> > > > > > > > > >> can at least gate some by
> > > > > ensuring
> > > > > > > that
> > > > > > > > >> the
> > > > > > > > >> > > > >> partition
> > > > > > > > >> > > > >> > >> has
> > > > > > > > >> > > > >> > >> > > been
> > > > > > > > >> > > > >> > >> > > > > > added
> > > > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >> transaction. The
> rationale
> > > here
> > > > > is
> > > > > > > that
> > > > > > > > >> > there
> > > > > > > > >> > > > are
> > > > > > > > >> > > > >> > >> likely
> > > > > > > > >> > > > >> > >> > > LESS
> > > > > > > > >> > > > >> > >> > > > > late
> > > > > > > > >> > > > >> > >> > > > > > > > > arrivals
> > > > > > > > >> > > > >> > >> > > > > > > > > >> as time goes on, so
> > hopefully
> > > > > most
> > > > > > > late
> > > > > > > > >> > > arrivals
> > > > > > > > >> > > > >> will
> > > > > > > > >> > > > >> > >> come
> > > > > > > > >> > > > >> > >> > > in
> > > > > > > > >> > > > >> > >> > > > > > BEFORE
> > > > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call.
> > > Those
> > > > > that
> > > > > > > > >> arrive
> > > > > > > > >> > > > before
> > > > > > > > >> > > > >> > will
> > > > > > > > >> > > > >> > >> be
> > > > > > > > >> > > > >> > >> > > > > properly
> > > > > > > > >> > > > >> > >> > > > > > > > gated
> > > > > > > > >> > > > >> > >> > > > > > > > > >> with the
> > describeTransactions
> > > > > > > approach.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > > >> > > > >> > >> > > > > > > > > >> If we take the approach
> you
> > > > > > > suggested,
> > > > > > > > >> ANY
> > > > > > > > >> > > late
> > > > > > > > >> > > > >> > arrival
> > > > > > > > >> > > > >> > >> > > from a
> > > > > > > > >> > > > >> > >> > > > > > > > previous
> > > > > > > > >> > > > >> > >> > > > > > > > > >> transaction will be
> added.
> > > And
> > > > we
> > > > > > > don't
> > > > > > > > >> want
> > > > > > > > >> > > > >> that. I
> > > > > > > > >> > > > >> > >> also
> > > > > > > > >> > > > >> > >> > > > don't
> > > > > > > > >> > > > >> > >> > > > > > see
> > > > > > > > >> > > > >> > >> > > > > > > > any
> > > > > > > > >> > > > >> > >> > > > > > > > > >> benefit in sending
> > > > > addPartitionsToTxn
> > > > > > > > >> over
> > > > > > > > >> > the
> > > > > > > > >> > > > >> > >> > describeTxns
> > > > > > > > >> > > > >> > >> > > > > call.
> > > > > > > > >> > > > >> > >> > > > > > > They
> > > > > > > > >> > > > >> > >> > > > > > > > > will
> > > > > > > > >> > > > >> > >> > > > > > > > > >> both be one extra RPC to
> > the
> > > > Txn
> > > > > > > > >> > coordinator.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > > >> > > > >> > >> > > > > > > > > >> To be clear — newer
> clients
> > > > will
> > > > > use
> > > > > > > > >> > > > >> > addPartitionsToTxn
> > > > > > > > >> > > > >> > >> > > > instead
> > > > > > > > >> > > > >> > >> > > > > of
> > > > > > > > >> > > > >> > >> > > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >> DescribeTxns.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > > >> > > > >> > >> > > > > > > > > >> 40)
> > > > > > > > >> > > > >> > >> > > > > > > > > >> My concern is that if we
> > have
> > > > > some
> > > > > > > delay
> > > > > > > > >> in
> > > > > > > > >> > > the
> > > > > > > > >> > > > >> > client
> > > > > > > > >> > > > >> > >> to
> > > > > > > > >> > > > >> > >> > > bump
> > > > > > > > >> > > > >> > >> > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > epoch,
> > > > > > > > >> > > > >> > >> > > > > > > > > >> it could continue to send
> > > epoch
> > > > > 73
> > > > > > > and
> > > > > > > > >> those
> > > > > > > > >> > > > >> records
> > > > > > > > >> > > > >> > >> would
> > > > > > > > >> > > > >> > >> > > not
> > > > > > > > >> > > > >> > >> > > > > be
> > > > > > > > >> > > > >> > >> > > > > > > > > fenced.
> > > > > > > > >> > > > >> > >> > > > > > > > > >> Perhaps this is not an
> > issue
> > > if
> > > > > we
> > > > > > > don't
> > > > > > > > >> > allow
> > > > > > > > >> > > > the
> > > > > > > > >> > > > >> > next
> > > > > > > > >> > > > >> > >> > > > produce
> > > > > > > > >> > > > >> > >> > > > > to
> > > > > > > > >> > > > >> > >> > > > > > > go
> > > > > > > > >> > > > >> > >> > > > > > > > > >> through before the EndTxn
> > > > request
> > > > > > > > >> returns.
> > > > > > > > >> > I'm
> > > > > > > > >> > > > >> also
> > > > > > > > >> > > > >> > >> > thinking
> > > > > > > > >> > > > >> > >> > > > > about
> > > > > > > > >> > > > >> > >> > > > > > > > > cases of
> > > > > > > > >> > > > >> > >> > > > > > > > > >> failure. I will need to
> > think
> > > > on
> > > > > > > this a
> > > > > > > > >> bit.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > > >> > > > >> > >> > > > > > > > > >> I wasn't sure if it was
> > that
> > > > > > > confusing.
> > > > > > > > >> But
> > > > > > > > >> > if
> > > > > > > > >> > > > we
> > > > > > > > >> > > > >> > >> think it
> > > > > > > > >> > > > >> > >> > > is,
> > > > > > > > >> > > > >> > >> > > > > we
> > > > > > > > >> > > > >> > >> > > > > > > can
> > > > > > > > >> > > > >> > >> > > > > > > > > >> investigate other ways.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > > >> > > > >> > >> > > > > > > > > >> 60)
> > > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > > >> > > > >> > >> > > > > > > > > >> I'm not sure these are
> the
> > > same
> > > > > > > > >> purgatories
> > > > > > > > >> > > > since
> > > > > > > > >> > > > >> one
> > > > > > > > >> > > > >> > >> is a
> > > > > > > > >> > > > >> > >> > > > > produce
> > > > > > > > >> > > > >> > >> > > > > > > > > >> purgatory (I was planning
> > on
> > > > > using a
> > > > > > > > >> > callback
> > > > > > > > >> > > > >> rather
> > > > > > > > >> > > > >> > >> than
> > > > > > > > >> > > > >> > >> > > > > > purgatory)
> > > > > > > > >> > > > >> > >> > > > > > > > and
> > > > > > > > >> > > > >> > >> > > > > > > > > >> the other is simply a
> > request
> > > > to
> > > > > > > append
> > > > > > > > >> to
> > > > > > > > >> > the
> > > > > > > > >> > > > >> log.
> > > > > > > > >> > > > >> > Not
> > > > > > > > >> > > > >> > >> > sure
> > > > > > > > >> > > > >> > >> > > > we
> > > > > > > > >> > > > >> > >> > > > > > have
> > > > > > > > >> > > > >> > >> > > > > > > > any
> > > > > > > > >> > > > >> > >> > > > > > > > > >> structure here for
> > ordering,
> > > > but
> > > > > my
> > > > > > > > >> > > > understanding
> > > > > > > > >> > > > >> is
> > > > > > > > >> > > > >> > >> that
> > > > > > > > >> > > > >> > >> > > the
> > > > > > > > >> > > > >> > >> > > > > > broker
> > > > > > > > >> > > > >> > >> > > > > > > > > could
> > > > > > > > >> > > > >> > >> > > > > > > > > >> handle the write request
> > > before
> > > > > it
> > > > > > > hears
> > > > > > > > >> > back
> > > > > > > > >> > > > from
> > > > > > > > >> > > > >> > the
> > > > > > > > >> > > > >> > >> Txn
> > > > > > > > >> > > > >> > >> > > > > > > > Coordinator.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > > >> > > > >> > >> > > > > > > > > >> Let me know if I
> > > misunderstood
> > > > > > > something
> > > > > > > > >> or
> > > > > > > > >> > > > >> something
> > > > > > > > >> > > > >> > >> was
> > > > > > > > >> > > > >> > >> > > > > unclear.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > > >> > > > >> > >> > > > > > > > > >> Justine
> > > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > > >> > > > >> > >> > > > > > > > > >> On Thu, Dec 1, 2022 at
> > 12:15
> > > PM
> > > > > > > Matthias
> > > > > > > > >> J.
> > > > > > > > >> > > Sax
> > > > > > > > >> > > > <
> > > > > > > > >> > > > >> > >> > > > > mjsax@apache.org
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > > wrote:
> > > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> Thanks for the details
> > > > Justine!
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> 20)
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> The client side change
> > for
> > > 2
> > > > is
> > > > > > > > >> removing
> > > > > > > > >> > the
> > > > > > > > >> > > > >> > >> > addPartitions
> > > > > > > > >> > > > >> > >> > > > to
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> transaction
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> call. We don't need to
> > make
> > > > > this
> > > > > > > from
> > > > > > > > >> the
> > > > > > > > >> > > > >> producer
> > > > > > > > >> > > > >> > to
> > > > > > > > >> > > > >> > >> > the
> > > > > > > > >> > > > >> > >> > > > txn
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> coordinator,
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> only server side.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> I think I did not
> express
> > > > myself
> > > > > > > > >> clearly. I
> > > > > > > > >> > > > >> > understand
> > > > > > > > >> > > > >> > >> > that
> > > > > > > > >> > > > >> > >> > > > we
> > > > > > > > >> > > > >> > >> > > > > > can
> > > > > > > > >> > > > >> > >> > > > > > > > (and
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> should) change the
> > producer
> > > to
> > > > > not
> > > > > > > send
> > > > > > > > >> the
> > > > > > > > >> > > > >> > >> > `addPartitions`
> > > > > > > > >> > > > >> > >> > > > > > request
> > > > > > > > >> > > > >> > >> > > > > > > > any
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> longer. But I don't
> thinks
> > > > it's
> > > > > > > > >> requirement
> > > > > > > > >> > > to
> > > > > > > > >> > > > >> > change
> > > > > > > > >> > > > >> > >> the
> > > > > > > > >> > > > >> > >> > > > > broker?
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> What I am trying to say
> > is:
> > > > as a
> > > > > > > > >> safe-guard
> > > > > > > > >> > > and
> > > > > > > > >> > > > >> > >> > improvement
> > > > > > > > >> > > > >> > >> > > > for
> > > > > > > > >> > > > >> > >> > > > > > > older
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> producers, the partition
> > > > leader
> > > > > can
> > > > > > > just
> > > > > > > > >> > send
> > > > > > > > >> > > > the
> > > > > > > > >> > > > >> > >> > > > > `addPartitions`
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> request to the
> > > TX-coordinator
> > > > > in any
> > > > > > > > >> case
> > > > > > > > >> > --
> > > > > > > > >> > > if
> > > > > > > > >> > > > >> the
> > > > > > > > >> > > > >> > >> old
> > > > > > > > >> > > > >> > >> > > > > producer
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> correctly did send the
> > > > > > > `addPartition`
> > > > > > > > >> > request
> > > > > > > > >> > > > to
> > > > > > > > >> > > > >> the
> > > > > > > > >> > > > >> > >> > > > > > TX-coordinator
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> already, the
> > TX-coordinator
> > > > can
> > > > > just
> > > > > > > > >> > "ignore"
> > > > > > > > >> > > > is
> > > > > > > > >> > > > >> as
> > > > > > > > >> > > > >> > >> > > > idempotent.
> > > > > > > > >> > > > >> > >> > > > > > > > > However,
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> if the old producer has
> a
> > > bug
> > > > > and
> > > > > > > did
> > > > > > > > >> > forget
> > > > > > > > >> > > to
> > > > > > > > >> > > > >> sent
> > > > > > > > >> > > > >> > >> the
> > > > > > > > >> > > > >> > >> > > > > > > > `addPartition`
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> request, we would now
> > ensure
> > > > > that
> > > > > > > the
> > > > > > > > >> > > partition
> > > > > > > > >> > > > >> is
> > > > > > > > >> > > > >> > >> indeed
> > > > > > > > >> > > > >> > >> > > > added
> > > > > > > > >> > > > >> > >> > > > > > to
> > > > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> TX and thus fix a
> > potential
> > > > > > > producer bug
> > > > > > > > >> > > (even
> > > > > > > > >> > > > >> if we
> > > > > > > > >> > > > >> > >> > don't
> > > > > > > > >> > > > >> > >> > > > get
> > > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> fencing via the bump
> > epoch).
> > > > --
> > > > > It
> > > > > > > > >> seems to
> > > > > > > > >> > > be
> > > > > > > > >> > > > a
> > > > > > > > >> > > > >> > good
> > > > > > > > >> > > > >> > >> > > > > > improvement?
> > > > > > > > >> > > > >> > >> > > > > > > Or
> > > > > > > > >> > > > >> > >> > > > > > > > > is
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> there a reason to not do
> > > this?
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> 30)
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> Transaction is ongoing
> =
> > > > > partition
> > > > > > > was
> > > > > > > > >> > added
> > > > > > > > >> > > > to
> > > > > > > > >> > > > >> > >> > > transaction
> > > > > > > > >> > > > >> > >> > > > > via
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> addPartitionsToTxn. We
> > > check
> > > > > this
> > > > > > > with
> > > > > > > > >> the
> > > > > > > > >> > > > >> > >> > > > > DescribeTransactions
> > > > > > > > >> > > > >> > >> > > > > > > > call.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> Let
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> me know if this wasn't
> > > > > sufficiently
> > > > > > > > >> > > explained
> > > > > > > > >> > > > >> here:
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> If we do what I propose
> in
> > > > > (20), we
> > > > > > > > >> don't
> > > > > > > > >> > > > really
> > > > > > > > >> > > > >> > need
> > > > > > > > >> > > > >> > >> to
> > > > > > > > >> > > > >> > >> > > make
> > > > > > > > >> > > > >> > >> > > > > > this
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> `DescribeTransaction`
> > call,
> > > as
> > > > > the
> > > > > > > > >> > partition
> > > > > > > > >> > > > >> leader
> > > > > > > > >> > > > >> > >> adds
> > > > > > > > >> > > > >> > >> > > the
> > > > > > > > >> > > > >> > >> > > > > > > > partition
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> for older clients and we
> > get
> > > > > this
> > > > > > > check
> > > > > > > > >> for
> > > > > > > > >> > > > free.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> 40)
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> The idea here is that
> if
> > > any
> > > > > > > messages
> > > > > > > > >> > > somehow
> > > > > > > > >> > > > >> come
> > > > > > > > >> > > > >> > in
> > > > > > > > >> > > > >> > >> > > before
> > > > > > > > >> > > > >> > >> > > > > we
> > > > > > > > >> > > > >> > >> > > > > > > get
> > > > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> new
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> epoch to the producer,
> > they
> > > > > will be
> > > > > > > > >> > fenced.
> > > > > > > > >> > > > >> > However,
> > > > > > > > >> > > > >> > >> if
> > > > > > > > >> > > > >> > >> > we
> > > > > > > > >> > > > >> > >> > > > > don't
> > > > > > > > >> > > > >> > >> > > > > > > > think
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> this
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> is necessary, it can be
> > > > > discussed
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> I agree that we should
> > have
> > > > > epoch
> > > > > > > > >> fencing.
> > > > > > > > >> > My
> > > > > > > > >> > > > >> > >> question is
> > > > > > > > >> > > > >> > >> > > > > > > different:
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> Assume we are at epoch
> 73,
> > > and
> > > > > we
> > > > > > > have
> > > > > > > > >> an
> > > > > > > > >> > > > ongoing
> > > > > > > > >> > > > >> > >> > > > transaction,
> > > > > > > > >> > > > >> > >> > > > > > that
> > > > > > > > >> > > > >> > >> > > > > > > > is
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> committed. It seems
> > natural
> > > to
> > > > > > > write the
> > > > > > > > >> > > > "prepare
> > > > > > > > >> > > > >> > >> commit"
> > > > > > > > >> > > > >> > >> > > > > marker
> > > > > > > > >> > > > >> > >> > > > > > > and
> > > > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest`
> > both
> > > > with
> > > > > > > epoch
> > > > > > > > >> 73,
> > > > > > > > >> > > too,
> > > > > > > > >> > > > >> as
> > > > > > > > >> > > > >> > it
> > > > > > > > >> > > > >> > >> > > belongs
> > > > > > > > >> > > > >> > >> > > > > to
> > > > > > > > >> > > > >> > >> > > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> current transaction. Of
> > > > course,
> > > > > we
> > > > > > > now
> > > > > > > > >> also
> > > > > > > > >> > > > bump
> > > > > > > > >> > > > >> the
> > > > > > > > >> > > > >> > >> > epoch
> > > > > > > > >> > > > >> > >> > > > and
> > > > > > > > >> > > > >> > >> > > > > > > expect
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> the next requests to
> have
> > > > epoch
> > > > > 74,
> > > > > > > and
> > > > > > > > >> > would
> > > > > > > > >> > > > >> reject
> > > > > > > > >> > > > >> > >> an
> > > > > > > > >> > > > >> > >> > > > request
> > > > > > > > >> > > > >> > >> > > > > > > with
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> epoch 73, as the
> > > corresponding
> > > > > TX
> > > > > > > for
> > > > > > > > >> epoch
> > > > > > > > >> > > 73
> > > > > > > > >> > > > >> was
> > > > > > > > >> > > > >> > >> > already
> > > > > > > > >> > > > >> > >> > > > > > > committed.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> It seems you propose to
> > > write
> > > > > the
> > > > > > > > >> "prepare
> > > > > > > > >> > > > commit
> > > > > > > > >> > > > >> > >> marker"
> > > > > > > > >> > > > >> > >> > > and
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest`
> > with
> > > > > epoch 74
> > > > > > > > >> > though,
> > > > > > > > >> > > > what
> > > > > > > > >> > > > >> > >> would
> > > > > > > > >> > > > >> > >> > > work,
> > > > > > > > >> > > > >> > >> > > > > but
> > > > > > > > >> > > > >> > >> > > > > > > it
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> seems confusing. Is
> there
> > a
> > > > > reason
> > > > > > > why
> > > > > > > > >> we
> > > > > > > > >> > > would
> > > > > > > > >> > > > >> use
> > > > > > > > >> > > > >> > >> the
> > > > > > > > >> > > > >> > >> > > > bumped
> > > > > > > > >> > > > >> > >> > > > > > > epoch
> > > > > > > > >> > > > >> > >> > > > > > > > 74
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> instead of the current
> > epoch
> > > > 73?
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> 60)
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> When we are checking if
> > the
> > > > > > > > >> transaction is
> > > > > > > > >> > > > >> ongoing,
> > > > > > > > >> > > > >> > >> we
> > > > > > > > >> > > > >> > >> > > need
> > > > > > > > >> > > > >> > >> > > > to
> > > > > > > > >> > > > >> > >> > > > > > > make
> > > > > > > > >> > > > >> > >> > > > > > > > a
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> round
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> trip from the leader
> > > > partition
> > > > > to
> > > > > > > the
> > > > > > > > >> > > > >> transaction
> > > > > > > > >> > > > >> > >> > > > coordinator.
> > > > > > > > >> > > > >> > >> > > > > > In
> > > > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> time
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> we are waiting for this
> > > > > message to
> > > > > > > come
> > > > > > > > >> > > back,
> > > > > > > > >> > > > in
> > > > > > > > >> > > > >> > >> theory
> > > > > > > > >> > > > >> > >> > we
> > > > > > > > >> > > > >> > >> > > > > could
> > > > > > > > >> > > > >> > >> > > > > > > > have
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> sent
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> a commit/abort call
> that
> > > > would
> > > > > > > make the
> > > > > > > > >> > > > original
> > > > > > > > >> > > > >> > >> result
> > > > > > > > >> > > > >> > >> > of
> > > > > > > > >> > > > >> > >> > > > the
> > > > > > > > >> > > > >> > >> > > > > > > check
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> out of
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> date. That is why we
> can
> > > > check
> > > > > the
> > > > > > > > >> leader
> > > > > > > > >> > > > state
> > > > > > > > >> > > > >> > >> before
> > > > > > > > >> > > > >> > >> > we
> > > > > > > > >> > > > >> > >> > > > > write
> > > > > > > > >> > > > >> > >> > > > > > to
> > > > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> log.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> Thanks. Got it.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> However, is this really
> an
> > > > > issue?
> > > > > > > We put
> > > > > > > > >> > the
> > > > > > > > >> > > > >> produce
> > > > > > > > >> > > > >> > >> > > request
> > > > > > > > >> > > > >> > >> > > > in
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> purgatory, so how could
> we
> > > > > process
> > > > > > > the
> > > > > > > > >> > > > >> > >> > > > `WriteTxnMarkerRequest`
> > > > > > > > >> > > > >> > >> > > > > > > first?
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> Don't we need to put the
> > > > > > > > >> > > > `WriteTxnMarkerRequest`
> > > > > > > > >> > > > >> > into
> > > > > > > > >> > > > >> > >> > > > > purgatory,
> > > > > > > > >> > > > >> > >> > > > > > > too,
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> for this case, and
> process
> > > > both
> > > > > > > request
> > > > > > > > >> > > > in-order?
> > > > > > > > >> > > > >> > >> (Again,
> > > > > > > > >> > > > >> > >> > > my
> > > > > > > > >> > > > >> > >> > > > > > broker
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> knowledge is limited and
> > > maybe
> > > > > we
> > > > > > > don't
> > > > > > > > >> > > > maintain
> > > > > > > > >> > > > >> > >> request
> > > > > > > > >> > > > >> > >> > > > order
> > > > > > > > >> > > > >> > >> > > > > > for
> > > > > > > > >> > > > >> > >> > > > > > > > this
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> case, what seems to be
> an
> > > > issue
> > > > > > > IMHO,
> > > > > > > > >> and I
> > > > > > > > >> > > am
> > > > > > > > >> > > > >> > >> wondering
> > > > > > > > >> > > > >> > >> > if
> > > > > > > > >> > > > >> > >> > > > > > > changing
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> request handling to
> > preserve
> > > > > order
> > > > > > > for
> > > > > > > > >> this
> > > > > > > > >> > > > case
> > > > > > > > >> > > > >> > >> might be
> > > > > > > > >> > > > >> > >> > > the
> > > > > > > > >> > > > >> > >> > > > > > > cleaner
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> solution?)
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> -Matthias
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> On 11/30/22 3:28 PM,
> Artem
> > > > > Livshits
> > > > > > > > >> wrote:
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> Hi Justine,
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> I think the interesting
> > > part
> > > > is
> > > > > > > not in
> > > > > > > > >> > this
> > > > > > > > >> > > > >> logic
> > > > > > > > >> > > > >> > >> > (because
> > > > > > > > >> > > > >> > >> > > > it
> > > > > > > > >> > > > >> > >> > > > > > > tries
> > > > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> figure out when
> > > > > > > UNKNOWN_PRODUCER_ID is
> > > > > > > > >> > > > retriable
> > > > > > > > >> > > > >> > and
> > > > > > > > >> > > > >> > >> if
> > > > > > > > >> > > > >> > >> > > it's
> > > > > > > > >> > > > >> > >> > > > > > > > > retryable,
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> it's definitely not
> > fatal),
> > > > but
> > > > > > > what
> > > > > > > > >> > happens
> > > > > > > > >> > > > >> when
> > > > > > > > >> > > > >> > >> this
> > > > > > > > >> > > > >> > >> > > logic
> > > > > > > > >> > > > >> > >> > > > > > > doesn't
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> return
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> 'true' and falls
> through.
> > > In
> > > > > the
> > > > > > > old
> > > > > > > > >> > > clients
> > > > > > > > >> > > > it
> > > > > > > > >> > > > >> > >> seems
> > > > > > > > >> > > > >> > >> > to
> > > > > > > > >> > > > >> > >> > > be
> > > > > > > > >> > > > >> > >> > > > > > > fatal,
> > > > > > > > >> > > > >> > >> > > > > > > > if
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> we
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> keep the behavior in
> the
> > > new
> > > > > > > clients,
> > > > > > > > >> I'd
> > > > > > > > >> > > > >> expect it
> > > > > > > > >> > > > >> > >> > would
> > > > > > > > >> > > > >> > >> > > be
> > > > > > > > >> > > > >> > >> > > > > > fatal
> > > > > > > > >> > > > >> > >> > > > > > > > as
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> well.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> -Artem
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>> On Tue, Nov 29, 2022 at
> > > 11:57
> > > > > AM
> > > > > > > > >> Justine
> > > > > > > > >> > > > Olshan
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > <jolshan@confluent.io.invalid
> > > > >
> > > > > > > wrote:
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> Hi Artem and Jeff,
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> Thanks for taking a
> look
> > > and
> > > > > > > sorry for
> > > > > > > > >> > the
> > > > > > > > >> > > > slow
> > > > > > > > >> > > > >> > >> > response.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> You both mentioned the
> > > > change
> > > > > to
> > > > > > > > >> handle
> > > > > > > > >> > > > >> > >> > > UNKNOWN_PRODUCER_ID
> > > > > > > > >> > > > >> > >> > > > > > > errors.
> > > > > > > > >> > > > >> > >> > > > > > > > > To
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> be
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> clear — this error
> code
> > > will
> > > > > only
> > > > > > > be
> > > > > > > > >> sent
> > > > > > > > >> > > > again
> > > > > > > > >> > > > >> > when
> > > > > > > > >> > > > >> > >> > the
> > > > > > > > >> > > > >> > >> > > > > > client's
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> request
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> version is high enough
> > to
> > > > > ensure
> > > > > > > we
> > > > > > > > >> > handle
> > > > > > > > >> > > it
> > > > > > > > >> > > > >> > >> > correctly.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> The current (Java)
> > client
> > > > > handles
> > > > > > > > >> this by
> > > > > > > > >> > > the
> > > > > > > > >> > > > >> > >> following
> > > > > > > > >> > > > >> > >> > > > > > (somewhat
> > > > > > > > >> > > > >> > >> > > > > > > > > long)
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> code snippet:
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> // An
> > UNKNOWN_PRODUCER_ID
> > > > > means
> > > > > > > that
> > > > > > > > >> we
> > > > > > > > >> > > have
> > > > > > > > >> > > > >> lost
> > > > > > > > >> > > > >> > >> the
> > > > > > > > >> > > > >> > >> > > > > producer
> > > > > > > > >> > > > >> > >> > > > > > > > state
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> on the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> broker. Depending on
> the
> > > log
> > > > > start
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> // offset, we may want
> > to
> > > > > retry
> > > > > > > > >> these, as
> > > > > > > > >> > > > >> > described
> > > > > > > > >> > > > >> > >> for
> > > > > > > > >> > > > >> > >> > > > each
> > > > > > > > >> > > > >> > >> > > > > > case
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> below. If
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> none of those apply,
> > then
> > > > for
> > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> // idempotent
> producer,
> > we
> > > > > will
> > > > > > > > >> locally
> > > > > > > > >> > > bump
> > > > > > > > >> > > > >> the
> > > > > > > > >> > > > >> > >> epoch
> > > > > > > > >> > > > >> > >> > > and
> > > > > > > > >> > > > >> > >> > > > > > reset
> > > > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> sequence numbers of
> > > > in-flight
> > > > > > > batches
> > > > > > > > >> > from
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> // sequence 0, then
> > retry
> > > > the
> > > > > > > failed
> > > > > > > > >> > batch,
> > > > > > > > >> > > > >> which
> > > > > > > > >> > > > >> > >> > should
> > > > > > > > >> > > > >> > >> > > > now
> > > > > > > > >> > > > >> > >> > > > > > > > succeed.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> For
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> the transactional
> > > producer,
> > > > > allow
> > > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> // batch to fail. When
> > > > > processing
> > > > > > > the
> > > > > > > > >> > > failed
> > > > > > > > >> > > > >> > batch,
> > > > > > > > >> > > > >> > >> we
> > > > > > > > >> > > > >> > >> > > will
> > > > > > > > >> > > > >> > >> > > > > > > > > transition
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> to
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> an abortable error and
> > > set a
> > > > > flag
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> // indicating that we
> > need
> > > > to
> > > > > > > bump the
> > > > > > > > >> > > epoch
> > > > > > > > >> > > > >> (if
> > > > > > > > >> > > > >> > >> > > supported
> > > > > > > > >> > > > >> > >> > > > by
> > > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> broker).
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> if (error ==
> > > > > > > > >> > Errors.*UNKNOWN_PRODUCER_ID*)
> > > > > > > > >> > > {
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>       if
> > > > > (response.logStartOffset
> > > > > > > ==
> > > > > > > > >> -1)
> > > > > > > > >> > {
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // We don't
> > know
> > > > > the log
> > > > > > > > >> start
> > > > > > > > >> > > > offset
> > > > > > > > >> > > > >> > with
> > > > > > > > >> > > > >> > >> > this
> > > > > > > > >> > > > >> > >> > > > > > > response.
> > > > > > > > >> > > > >> > >> > > > > > > > > We
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> should
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> just retry the request
> > > until
> > > > > we
> > > > > > > get
> > > > > > > > >> it.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // The
> > > > > > > UNKNOWN_PRODUCER_ID
> > > > > > > > >> > error
> > > > > > > > >> > > > code
> > > > > > > > >> > > > >> > was
> > > > > > > > >> > > > >> > >> > added
> > > > > > > > >> > > > >> > >> > > > > along
> > > > > > > > >> > > > >> > >> > > > > > > > with
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> the new
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> ProduceResponse which
> > > > > includes the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           //
> > > logStartOffset.
> > > > > So
> > > > > > > the
> > > > > > > > >> '-1'
> > > > > > > > >> > > > >> sentinel
> > > > > > > > >> > > > >> > is
> > > > > > > > >> > > > >> > >> > not
> > > > > > > > >> > > > >> > >> > > > for
> > > > > > > > >> > > > >> > >> > > > > > > > backward
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> compatibility.
> Instead,
> > it
> > > > is
> > > > > > > possible
> > > > > > > > >> > for
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // a broker
> to
> > > not
> > > > > know
> > > > > > > the
> > > > > > > > >> > > > >> > >> logStartOffset at
> > > > > > > > >> > > > >> > >> > > > when
> > > > > > > > >> > > > >> > >> > > > > it
> > > > > > > > >> > > > >> > >> > > > > > > is
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> returning
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> the response because
> the
> > > > > partition
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // may have
> > > moved
> > > > > away
> > > > > > > from
> > > > > > > > >> the
> > > > > > > > >> > > > >> broker
> > > > > > > > >> > > > >> > >> from
> > > > > > > > >> > > > >> > >> > the
> > > > > > > > >> > > > >> > >> > > > > time
> > > > > > > > >> > > > >> > >> > > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> error was
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> initially raised to
> the
> > > time
> > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // response
> > was
> > > > > being
> > > > > > > > >> > > constructed.
> > > > > > > > >> > > > In
> > > > > > > > >> > > > >> > >> these
> > > > > > > > >> > > > >> > >> > > > cases,
> > > > > > > > >> > > > >> > >> > > > > we
> > > > > > > > >> > > > >> > >> > > > > > > > > should
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> just
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> retry the request: we
> > are
> > > > > > > guaranteed
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // to
> > eventually
> > > > > get a
> > > > > > > > >> > > > logStartOffset
> > > > > > > > >> > > > >> > once
> > > > > > > > >> > > > >> > >> > > things
> > > > > > > > >> > > > >> > >> > > > > > > settle
> > > > > > > > >> > > > >> > >> > > > > > > > > down.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>       if
> > > > > > > > >> (batch.sequenceHasBeenReset()) {
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // When the
> > > first
> > > > > > > inflight
> > > > > > > > >> > batch
> > > > > > > > >> > > > >> fails
> > > > > > > > >> > > > >> > >> due to
> > > > > > > > >> > > > >> > >> > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > truncation
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> case,
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> then the sequences of
> > all
> > > > the
> > > > > > > other
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // in flight
> > > > batches
> > > > > > > would
> > > > > > > > >> have
> > > > > > > > >> > > > been
> > > > > > > > >> > > > >> > >> > restarted
> > > > > > > > >> > > > >> > >> > > > from
> > > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> beginning.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> However, when those
> > > > responses
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // come back
> > > from
> > > > > the
> > > > > > > > >> broker,
> > > > > > > > >> > > they
> > > > > > > > >> > > > >> would
> > > > > > > > >> > > > >> > >> also
> > > > > > > > >> > > > >> > >> > > > come
> > > > > > > > >> > > > >> > >> > > > > > with
> > > > > > > > >> > > > >> > >> > > > > > > > an
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > error.
> > > > In
> > > > > this
> > > > > > > > >> case,
> > > > > > > > >> > we
> > > > > > > > >> > > > >> should
> > > > > > > > >> > > > >> > >> not
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // reset the
> > > > > sequence
> > > > > > > > >> numbers
> > > > > > > > >> > to
> > > > > > > > >> > > > the
> > > > > > > > >> > > > >> > >> > beginning.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>       } else if
> > > > > > > > >> > > > >> > >> > > > >
> > > > > (lastAckedOffset(batch.topicPartition).orElse(
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > > > > > > > >> > > > >> > >> > > > response.logStartOffset) {
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // The head
> of
> > > the
> > > > > log
> > > > > > > has
> > > > > > > > >> been
> > > > > > > > >> > > > >> removed,
> > > > > > > > >> > > > >> > >> > > probably
> > > > > > > > >> > > > >> > >> > > > > due
> > > > > > > > >> > > > >> > >> > > > > > > to
> > > > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> retention time
> elapsing.
> > > In
> > > > > this
> > > > > > > case,
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // we expect
> > to
> > > > > lose the
> > > > > > > > >> > producer
> > > > > > > > >> > > > >> state.
> > > > > > > > >> > > > >> > >> For
> > > > > > > > >> > > > >> > >> > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > transactional
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> producer, reset the
> > > > sequences
> > > > > of
> > > > > > > all
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // inflight
> > > > batches
> > > > > to
> > > > > > > be
> > > > > > > > >> from
> > > > > > > > >> > > the
> > > > > > > > >> > > > >> > >> beginning
> > > > > > > > >> > > > >> > >> > > and
> > > > > > > > >> > > > >> > >> > > > > > retry
> > > > > > > > >> > > > >> > >> > > > > > > > > them,
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> so
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> that the transaction
> > does
> > > > not
> > > > > > > need to
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // be
> aborted.
> > > For
> > > > > the
> > > > > > > > >> > idempotent
> > > > > > > > >> > > > >> > >> producer,
> > > > > > > > >> > > > >> > >> > > bump
> > > > > > > > >> > > > >> > >> > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > epoch
> > > > > > > > >> > > > >> > >> > > > > > > > > to
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> avoid
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> reusing (sequence,
> > epoch)
> > > > > pairs
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           if
> > > > > (isTransactional()) {
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > >
> > > > > > > > >> > > >
> > > > > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > this.producerIdAndEpoch);
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           } else {
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > >
> > > > > > > > >> requestEpochBumpForPartition(batch.topicPartition);
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           }
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>       if
> > > > (!isTransactional())
> > > > > {
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // For the
> > > > > idempotent
> > > > > > > > >> producer,
> > > > > > > > >> > > > >> always
> > > > > > > > >> > > > >> > >> retry
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> errors. If the batch
> has
> > > the
> > > > > > > current
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // producer
> ID
> > > and
> > > > > > > epoch,
> > > > > > > > >> > > request a
> > > > > > > > >> > > > >> bump
> > > > > > > > >> > > > >> > >> of
> > > > > > > > >> > > > >> > >> > the
> > > > > > > > >> > > > >> > >> > > > > > epoch.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> Otherwise
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> just retry the
> produce.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > >
> > > > > > > requestEpochBumpForPartition(batch.topicPartition);
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> }
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> I was considering
> > keeping
> > > > this
> > > > > > > > >> behavior —
> > > > > > > > >> > > but
> > > > > > > > >> > > > >> am
> > > > > > > > >> > > > >> > >> open
> > > > > > > > >> > > > >> > >> > to
> > > > > > > > >> > > > >> > >> > > > > > > > simplifying
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> it.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> We are leaving changes
> > to
> > > > > older
> > > > > > > > >> clients
> > > > > > > > >> > off
> > > > > > > > >> > > > the
> > > > > > > > >> > > > >> > >> table
> > > > > > > > >> > > > >> > >> > > here
> > > > > > > > >> > > > >> > >> > > > > > since
> > > > > > > > >> > > > >> > >> > > > > > > it
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> caused
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> many issues for
> clients
> > in
> > > > the
> > > > > > > past.
> > > > > > > > >> > > > Previously
> > > > > > > > >> > > > >> > this
> > > > > > > > >> > > > >> > >> > was
> > > > > > > > >> > > > >> > >> > > a
> > > > > > > > >> > > > >> > >> > > > > > fatal
> > > > > > > > >> > > > >> > >> > > > > > > > > error
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> and
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> we didn't have the
> > > > mechanisms
> > > > > in
> > > > > > > > >> place to
> > > > > > > > >> > > > >> detect
> > > > > > > > >> > > > >> > >> when
> > > > > > > > >> > > > >> > >> > > this
> > > > > > > > >> > > > >> > >> > > > > was
> > > > > > > > >> > > > >> > >> > > > > > a
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> legitimate
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> case vs some bug or
> gap
> > in
> > > > the
> > > > > > > > >> protocol.
> > > > > > > > >> > > > >> Ensuring
> > > > > > > > >> > > > >> > >> each
> > > > > > > > >> > > > >> > >> > > > > > > transaction
> > > > > > > > >> > > > >> > >> > > > > > > > > has
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> its
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> own epoch should close
> > > this
> > > > > gap.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> And to address Jeff's
> > > second
> > > > > > > point:
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> *does the typical
> > produce
> > > > > request
> > > > > > > path
> > > > > > > > >> > > append
> > > > > > > > >> > > > >> > >> records
> > > > > > > > >> > > > >> > >> > to
> > > > > > > > >> > > > >> > >> > > > > local
> > > > > > > > >> > > > >> > >> > > > > > > log
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> along*
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> *with the
> > > > > currentTxnFirstOffset
> > > > > > > > >> > > information?
> > > > > > > > >> > > > I
> > > > > > > > >> > > > >> > would
> > > > > > > > >> > > > >> > >> > like
> > > > > > > > >> > > > >> > >> > > > to
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> understand*
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> *when the field is
> > written
> > > > to
> > > > > > > disk.*
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> Yes, the first produce
> > > > request
> > > > > > > > >> populates
> > > > > > > > >> > > this
> > > > > > > > >> > > > >> > field
> > > > > > > > >> > > > >> > >> and
> > > > > > > > >> > > > >> > >> > > > > writes
> > > > > > > > >> > > > >> > >> > > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> offset
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> as part of the record
> > > batch
> > > > > and
> > > > > > > also
> > > > > > > > >> to
> > > > > > > > >> > the
> > > > > > > > >> > > > >> > producer
> > > > > > > > >> > > > >> > >> > > state
> > > > > > > > >> > > > >> > >> > > > > > > > snapshot.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> When
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> we reload the records
> on
> > > > > restart
> > > > > > > > >> and/or
> > > > > > > > >> > > > >> > >> reassignment,
> > > > > > > > >> > > > >> > >> > we
> > > > > > > > >> > > > >> > >> > > > > > > repopulate
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> this
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> field with the
> snapshot
> > > from
> > > > > disk
> > > > > > > > >> along
> > > > > > > > >> > > with
> > > > > > > > >> > > > >> the
> > > > > > > > >> > > > >> > >> rest
> > > > > > > > >> > > > >> > >> > of
> > > > > > > > >> > > > >> > >> > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > producer
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> state.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> Let me know if there
> are
> > > > > further
> > > > > > > > >> comments
> > > > > > > > >> > > > >> and/or
> > > > > > > > >> > > > >> > >> > > questions.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> Thanks,
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> Justine
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> On Tue, Nov 22, 2022
> at
> > > 9:00
> > > > > PM
> > > > > > > Jeff
> > > > > > > > >> Kim
> > > > > > > > >> > > > >> > >> > > > > > > > >
> <jeff.kim@confluent.io.invalid
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> wrote:
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Hi Justine,
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Thanks for the KIP! I
> > > have
> > > > > two
> > > > > > > > >> > questions:
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> 1) For new clients,
> we
> > > can
> > > > > once
> > > > > > > again
> > > > > > > > >> > > return
> > > > > > > > >> > > > >> an
> > > > > > > > >> > > > >> > >> error
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> for sequences
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> that are non-zero
> when
> > > > there
> > > > > is
> > > > > > > no
> > > > > > > > >> > > producer
> > > > > > > > >> > > > >> state
> > > > > > > > >> > > > >> > >> > > present
> > > > > > > > >> > > > >> > >> > > > on
> > > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> server.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> This will indicate we
> > > > missed
> > > > > the
> > > > > > > 0
> > > > > > > > >> > > sequence
> > > > > > > > >> > > > >> and
> > > > > > > > >> > > > >> > we
> > > > > > > > >> > > > >> > >> > don't
> > > > > > > > >> > > > >> > >> > > > yet
> > > > > > > > >> > > > >> > >> > > > > > > want
> > > > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> write
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> to the log.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> I would like to
> > > understand
> > > > > the
> > > > > > > > >> current
> > > > > > > > >> > > > >> behavior
> > > > > > > > >> > > > >> > to
> > > > > > > > >> > > > >> > >> > > handle
> > > > > > > > >> > > > >> > >> > > > > > older
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> clients,
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> and if there are any
> > > > changes
> > > > > we
> > > > > > > are
> > > > > > > > >> > > making.
> > > > > > > > >> > > > >> Maybe
> > > > > > > > >> > > > >> > >> I'm
> > > > > > > > >> > > > >> > >> > > > > missing
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> something,
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> but we would want to
> > > > identify
> > > > > > > > >> whether we
> > > > > > > > >> > > > >> missed
> > > > > > > > >> > > > >> > >> the 0
> > > > > > > > >> > > > >> > >> > > > > sequence
> > > > > > > > >> > > > >> > >> > > > > > > for
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> older
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> clients, no?
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> 2) Upon returning
> from
> > > the
> > > > > > > > >> transaction
> > > > > > > > >> > > > >> > >> coordinator, we
> > > > > > > > >> > > > >> > >> > > can
> > > > > > > > >> > > > >> > >> > > > > set
> > > > > > > > >> > > > >> > >> > > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> transaction
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> as ongoing on the
> > leader
> > > by
> > > > > > > > >> populating
> > > > > > > > >> > > > >> > >> > > > currentTxnFirstOffset
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> through the typical
> > > produce
> > > > > > > request
> > > > > > > > >> > > > handling.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> does the typical
> > produce
> > > > > request
> > > > > > > path
> > > > > > > > >> > > append
> > > > > > > > >> > > > >> > >> records
> > > > > > > > >> > > > >> > >> > to
> > > > > > > > >> > > > >> > >> > > > > local
> > > > > > > > >> > > > >> > >> > > > > > > log
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> along
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> with the
> > > > > currentTxnFirstOffset
> > > > > > > > >> > > information?
> > > > > > > > >> > > > I
> > > > > > > > >> > > > >> > would
> > > > > > > > >> > > > >> > >> > like
> > > > > > > > >> > > > >> > >> > > > to
> > > > > > > > >> > > > >> > >> > > > > > > > > understand
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> when the field is
> > written
> > > > to
> > > > > > > disk.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Thanks,
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Jeff
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> On Tue, Nov 22, 2022
> at
> > > > 4:44
> > > > > PM
> > > > > > > Artem
> > > > > > > > >> > > > Livshits
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> <
> > alivshits@confluent.io
> > > > > .invalid>
> > > > > > > > >> wrote:
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> Hi Justine,
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> Thank you for the
> KIP.
> > > I
> > > > > have
> > > > > > > one
> > > > > > > > >> > > > question.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> 5) For new clients,
> we
> > > can
> > > > > once
> > > > > > > > >> again
> > > > > > > > >> > > > return
> > > > > > > > >> > > > >> an
> > > > > > > > >> > > > >> > >> error
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> I believe we had
> > > problems
> > > > > in the
> > > > > > > > >> past
> > > > > > > > >> > > with
> > > > > > > > >> > > > >> > >> returning
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> because it was
> > > considered
> > > > > fatal
> > > > > > > and
> > > > > > > > >> > > > required
> > > > > > > > >> > > > >> > >> client
> > > > > > > > >> > > > >> > >> > > > > restart.
> > > > > > > > >> > > > >> > >> > > > > > > It
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> would
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> be
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> good to spell out
> the
> > > new
> > > > > client
> > > > > > > > >> > behavior
> > > > > > > > >> > > > >> when
> > > > > > > > >> > > > >> > it
> > > > > > > > >> > > > >> > >> > > > receives
> > > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > error.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> -Artem
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> On Tue, Nov 22, 2022
> > at
> > > > > 10:00 AM
> > > > > > > > >> > Justine
> > > > > > > > >> > > > >> Olshan
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > <jo...@confluent.io.invalid>
> > > > > > > > >> wrote:
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks for taking a
> > > look
> > > > > > > Matthias.
> > > > > > > > >> > I've
> > > > > > > > >> > > > >> tried
> > > > > > > > >> > > > >> > to
> > > > > > > > >> > > > >> > >> > > answer
> > > > > > > > >> > > > >> > >> > > > > your
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> questions
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> below:
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 10)
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Right — so the
> > hanging
> > > > > > > transaction
> > > > > > > > >> > only
> > > > > > > > >> > > > >> occurs
> > > > > > > > >> > > > >> > >> when
> > > > > > > > >> > > > >> > >> > we
> > > > > > > > >> > > > >> > >> > > > > have
> > > > > > > > >> > > > >> > >> > > > > > a
> > > > > > > > >> > > > >> > >> > > > > > > > late
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> message
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> come in and the
> > > partition
> > > > > is
> > > > > > > never
> > > > > > > > >> > added
> > > > > > > > >> > > > to
> > > > > > > > >> > > > >> a
> > > > > > > > >> > > > >> > >> > > > transaction
> > > > > > > > >> > > > >> > >> > > > > > > again.
> > > > > > > > >> > > > >> > >> > > > > > > > > If
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> we
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> never add the
> > partition
> > > > to
> > > > > a
> > > > > > > > >> > > transaction,
> > > > > > > > >> > > > we
> > > > > > > > >> > > > >> > will
> > > > > > > > >> > > > >> > >> > > never
> > > > > > > > >> > > > >> > >> > > > > > write
> > > > > > > > >> > > > >> > >> > > > > > > a
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> marker
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> and
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> never advance the
> > LSO.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> If we do end up
> > adding
> > > > the
> > > > > > > > >> partition
> > > > > > > > >> > to
> > > > > > > > >> > > > the
> > > > > > > > >> > > > >> > >> > > transaction
> > > > > > > > >> > > > >> > >> > > > (I
> > > > > > > > >> > > > >> > >> > > > > > > > suppose
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> this
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> can
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> happen before or
> > after
> > > > the
> > > > > late
> > > > > > > > >> > message
> > > > > > > > >> > > > >> comes
> > > > > > > > >> > > > >> > in)
> > > > > > > > >> > > > >> > >> > then
> > > > > > > > >> > > > >> > >> > > > we
> > > > > > > > >> > > > >> > >> > > > > > will
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> include
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> late message in the
> > > next
> > > > > > > > >> (incorrect)
> > > > > > > > >> > > > >> > transaction.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> So perhaps it is
> > > clearer
> > > > to
> > > > > > > make
> > > > > > > > >> the
> > > > > > > > >> > > > >> > distinction
> > > > > > > > >> > > > >> > >> > > between
> > > > > > > > >> > > > >> > >> > > > > > > > messages
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> that
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> eventually get
> added
> > to
> > > > the
> > > > > > > > >> > transaction
> > > > > > > > >> > > > (but
> > > > > > > > >> > > > >> > the
> > > > > > > > >> > > > >> > >> > wrong
> > > > > > > > >> > > > >> > >> > > > > one)
> > > > > > > > >> > > > >> > >> > > > > > or
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> messages
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> that never get
> added
> > > and
> > > > > become
> > > > > > > > >> > hanging.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 20)
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> The client side
> > change
> > > > for
> > > > > 2 is
> > > > > > > > >> > removing
> > > > > > > > >> > > > the
> > > > > > > > >> > > > >> > >> > > > addPartitions
> > > > > > > > >> > > > >> > >> > > > > > to
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> transaction
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> call. We don't need
> > to
> > > > make
> > > > > > > this
> > > > > > > > >> from
> > > > > > > > >> > > the
> > > > > > > > >> > > > >> > >> producer
> > > > > > > > >> > > > >> > >> > to
> > > > > > > > >> > > > >> > >> > > > the
> > > > > > > > >> > > > >> > >> > > > > > txn
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> coordinator,
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> only server side.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> In my opinion, the
> > > issue
> > > > > with
> > > > > > > the
> > > > > > > > >> > > > >> > >> addPartitionsToTxn
> > > > > > > > >> > > > >> > >> > > > call
> > > > > > > > >> > > > >> > >> > > > > > for
> > > > > > > > >> > > > >> > >> > > > > > > > > older
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> clients
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> is that we don't
> have
> > > the
> > > > > epoch
> > > > > > > > >> bump,
> > > > > > > > >> > so
> > > > > > > > >> > > > we
> > > > > > > > >> > > > >> > don't
> > > > > > > > >> > > > >> > >> > know
> > > > > > > > >> > > > >> > >> > > > if
> > > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> message
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> belongs to the
> > previous
> > > > > > > > >> transaction or
> > > > > > > > >> > > > this
> > > > > > > > >> > > > >> > one.
> > > > > > > > >> > > > >> > >> We
> > > > > > > > >> > > > >> > >> > > need
> > > > > > > > >> > > > >> > >> > > > > to
> > > > > > > > >> > > > >> > >> > > > > > > > check
> > > > > > > > >> > > > >> > >> > > > > > > > > if
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> partition has been
> > > added
> > > > to
> > > > > > > this
> > > > > > > > >> > > > >> transaction.
> > > > > > > > >> > > > >> > Of
> > > > > > > > >> > > > >> > >> > > course,
> > > > > > > > >> > > > >> > >> > > > > > this
> > > > > > > > >> > > > >> > >> > > > > > > > > means
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> we
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> won't completely
> > cover
> > > > the
> > > > > case
> > > > > > > > >> where
> > > > > > > > >> > we
> > > > > > > > >> > > > >> have a
> > > > > > > > >> > > > >> > >> > really
> > > > > > > > >> > > > >> > >> > > > > late
> > > > > > > > >> > > > >> > >> > > > > > > > > message
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> and
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> we
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> have added the
> > > partition
> > > > to
> > > > > > > the new
> > > > > > > > >> > > > >> > transaction,
> > > > > > > > >> > > > >> > >> but
> > > > > > > > >> > > > >> > >> > > > > that's
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> unfortunately
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> something we will
> > need
> > > > the
> > > > > new
> > > > > > > > >> clients
> > > > > > > > >> > > to
> > > > > > > > >> > > > >> > cover.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 30)
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Transaction is
> > ongoing
> > > =
> > > > > > > partition
> > > > > > > > >> was
> > > > > > > > >> > > > >> added to
> > > > > > > > >> > > > >> > >> > > > > transaction
> > > > > > > > >> > > > >> > >> > > > > > > via
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> addPartitionsToTxn.
> > We
> > > > > check
> > > > > > > this
> > > > > > > > >> with
> > > > > > > > >> > > the
> > > > > > > > >> > > > >> > >> > > > > > > DescribeTransactions
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> call.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Let
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> me know if this
> > wasn't
> > > > > > > sufficiently
> > > > > > > > >> > > > >> explained
> > > > > > > > >> > > > >> > >> here:
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > >
> > > > > > > > >> > > > >> > >> > > > >
> > > > > > > > >> > > > >> > >> > > >
> > > > > > > > >> > > > >> > >> > >
> > > > > > > > >> > > > >> > >> >
> > > > > > > > >> > > > >> > >>
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >>
> > > > > > > > >> > > >
> > > > > > > > >> > >
> > > > > > > > >> >
> > > > > > > > >>
> > > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 40)
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> The idea here is
> that
> > > if
> > > > > any
> > > > > > > > >> messages
> > > > > > > > >> > > > >> somehow
> > > > > > > > >> > > > >> > >> come
> > > > > > > > >> > > > >> > >> > in
> > > > > > > > >> > > > >> > >> > > > > before
> > > > > > > > >> > > > >> > >> > > > > > > we
> > > > > > > > >> > > > >> > >> > > > > > > > > get
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> new
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> epoch to the
> > producer,
> > > > they
> > > > > > > will be
> > > > > > > > >> > > > fenced.
> > > > > > > > >> > > > >> > >> However,
> > > > > > > > >> > > > >> > >> > > if
> > > > > > > > >> > > > >> > >> > > > we
> > > > > > > > >> > > > >> > >> > > > > > > don't
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> think
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> this
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> is necessary, it
> can
> > be
> > > > > > > discussed
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 50)
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> It should be
> > > synchronous
> > > > > > > because
> > > > > > > > >> if we
> > > > > > > > >> > > > have
> > > > > > > > >> > > > >> an
> > > > > > > > >> > > > >> > >> event
> > > > > > > > >> > > > >> > >> > > > (ie,
> > > > > > > > >> > > > >> > >> > > > > an
> > > > > > > > >> > > > >> > >> > > > > > > > > error)
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> that
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> causes us to need
> to
> > > > abort
> > > > > the
> > > > > > > > >> > > > transaction,
> > > > > > > > >> > > > >> we
> > > > > > > > >> > > > >> > >> need
> > > > > > > > >> > > > >> > >> > to
> > > > > > > > >> > > > >> > >> > > > > know
> > > > > > > > >> > > > >> > >> > > > > > > > which
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> partitions to send
> > > > > transaction
> > > > > > > > >> markers
> > > > > > > > >> > > to.
> > > > > > > > >> > > > >> We
> > > > > > > > >> > > > >> > >> know
> > > > > > > > >> > > > >> > >> > the
> > > > > > > > >> > > > >> > >> > > > > > > > partitions
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> because
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> we added them to
> the
> > > > > > > coordinator
> > > > > > > > >> via
> > > > > > > > >> > the
> > > > > > > > >> > > > >> > >> > > > > addPartitionsToTxn
> > > > > > > > >> > > > >> > >> > > > > > > > call.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Previously we have
> > had
> > > > > > > asynchronous
> > > > > > > > >> > > calls
> > > > > > > > >> > > > in
> > > > > > > > >> > > > >> > the
> > > > > > > > >> > > > >> > >> > past
> > > > > > > > >> > > > >> > >> > > > (ie,
> > > > > > > > >> > > > >> > >> > > > > > > > writing
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> commit markers when
> > the
> > > > > > > > >> transaction is
> > > > > > > > >> > > > >> > completed)
> > > > > > > > >> > > > >> > >> > but
> > > > > > > > >> > > > >> > >> > > > > often
> > > > > > > > >> > > > >> > >> > > > > > > this
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> just
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> causes confusion as
> > we
> > > > > need to
> > > > > > > wait
> > > > > > > > >> > for
> > > > > > > > >> > > > some
> > > > > > > > >> > > > >> > >> > > operations
> > > > > > > > >> > > > >> > >> > > > to
> > > > > > > > >> > > > >> > >> > > > > > > > > complete.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> In
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> writing commit
> > markers
> > > > > case,
> > > > > > > > >> clients
> > > > > > > > >> > > often
> > > > > > > > >> > > > >> see
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> CONCURRENT_TRANSACTIONs
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> error messages and
> > that
> > > > > can be
> > > > > > > > >> > > confusing.
> > > > > > > > >> > > > >> For
> > > > > > > > >> > > > >> > >> that
> > > > > > > > >> > > > >> > >> > > > reason,
> > > > > > > > >> > > > >> > >> > > > > > it
> > > > > > > > >> > > > >> > >> > > > > > > > may
> > > > > > > > >> > > > >> > >> > > > > > > > > be
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> simpler to just
> have
> > > > > > > synchronous
> > > > > > > > >> > calls —
> > > > > > > > >> > > > >> > >> especially
> > > > > > > > >> > > > >> > >> > if
> > > > > > > > >> > > > >> > >> > > > we
> > > > > > > > >> > > > >> > >> > > > > > need
> > > > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> block
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> on
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> some operation's
> > > > completion
> > > > > > > anyway
> > > > > > > > >> > > before
> > > > > > > > >> > > > we
> > > > > > > > >> > > > >> > can
> > > > > > > > >> > > > >> > >> > start
> > > > > > > > >> > > > >> > >> > > > the
> > > > > > > > >> > > > >> > >> > > > > > > next
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> transaction. And
> > yes, I
> > > > > meant
> > > > > > > > >> > > > coordinator. I
> > > > > > > > >> > > > >> > will
> > > > > > > > >> > > > >> > >> > fix
> > > > > > > > >> > > > >> > >> > > > > that.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 60)
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> When we are
> checking
> > if
> > > > the
> > > > > > > > >> > transaction
> > > > > > > > >> > > is
> > > > > > > > >> > > > >> > >> ongoing,
> > > > > > > > >> > > > >> > >> > we
> > > > > > > > >> > > > >> > >> > > > > need
> > > > > > > > >> > > > >> > >> > > > > > to
> > > > > > > > >> > > > >> > >> > > > > > > > > make
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> a
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> round
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> trip from the
> leader
> > > > > partition
> > > > > > > to
> > > > > > > > >> the
> > > > > > > > >> > > > >> > transaction
> > > > > > > > >> > > > >> > >> > > > > > coordinator.
> > > > > > > > >> > > > >> > >> > > > > > > > In
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> time
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> we are waiting for
> > this
> > > > > > > message to
> > > > > > > > >> > come
> > > > > > > > >> > > > >> back,
> > > > > > > > >> > > > >> > in
> > > > > > > > >> > > > >> > >> > > theory
> > > > > > > > >> > > > >> > >> > > > we
> > > > > > > > >> > > > >> > >> > > > > > > could
> > > > > > > > >> > > > >> > >> > > > > > > > > >>> have
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> sent
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> a commit/abort call
> > > that
> > > > > would
> > > > > > > make
> > > > > > > > >> > the
> > > > > > > > >> > > > >> > original
> > > > > > > > >> > > > >> > >> > > result
> > > > > > > > >> > > > >> > >> > > > of
> > > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > check
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> out
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> of
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> date. That is why
> we
> > > can
> > > > > check
> > > > > > > the
> > > > > > > > >> > > leader
> > > > > > > > >> > > > >> state
> > > > > > > > >> > > > >> > >> > before
> > > > > > > > >> > > > >> > >> > > > we
> > > > > > > > >> > > > >> > >> > > > > > > write
> > > > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> log.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> I'm happy to update
> > the
> > > > > KIP if
> > > > > > > > >> some of
> > > > > > > > >> > > > these
> > > > > > > > >> > > > >> > >> things
> > > > > > > > >> > > > >> > >> > > were
> > > > > > > > >> > > > >> > >> > > > > not
> > > > > > > > >> > > > >> > >> > > > > > > > > clear.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks,
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Justine
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> On Mon, Nov 21,
> 2022
> > at
> > > > > 7:11 PM
> > > > > > > > >> > Matthias
> > > > > > > > >> > > > J.
> > > > > > > > >> > > > >> > Sax <
> > > > > > > > >> > > > >> > >> > > > > > > > mjsax@apache.org
> > > > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> wrote:
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Thanks for the
> KIP.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Couple of
> > > clarification
> > > > > > > questions
> > > > > > > > >> (I
> > > > > > > > >> > am
> > > > > > > > >> > > > >> not a
> > > > > > > > >> > > > >> > >> > broker
> > > > > > > > >> > > > >> > >> > > > > expert
> > > > > > > > >> > > > >> > >> > > > > > > do
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> maybe
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> some question are
> > > > obvious
> > > > > for
> > > > > > > > >> others,
> > > > > > > > >> > > but
> > > > > > > > >> > > > >> not
> > > > > > > > >> > > > >> > >> for
> > > > > > > > >> > > > >> > >> > me
> > > > > > > > >> > > > >> > >> > > > with
> > > > > > > > >> > > > >> > >> > > > > > my
> > > > > > > > >> > > > >> > >> > > > > > > > lack
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> of
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> broker knowledge).
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (10)
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> 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.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> What happens if
> the
> > > > > message
> > > > > > > come
> > > > > > > > >> in
> > > > > > > > >> > > > before
> > > > > > > > >> > > > >> the
> > > > > > > > >> > > > >> > >> next
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> addPartitionsToTxn
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> request? It seems
> > the
> > > > > broker
> > > > > > > > >> hosting
> > > > > > > > >> > > the
> > > > > > > > >> > > > >> data
> > > > > > > > >> > > > >> > >> > > > partitions
> > > > > > > > >> > > > >> > >> > > > > > > won't
> > > > > > > > >> > > > >> > >> > > > > > > > > know
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> anything about it
> > and
> > > > > append
> > > > > > > it to
> > > > > > > > >> > the
> > > > > > > > >> > > > >> > >> partition,
> > > > > > > > >> > > > >> > >> > > too?
> > > > > > > > >> > > > >> > >> > > > > What
> > > > > > > > >> > > > >> > >> > > > > > > is
> > > > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> difference between
> > > both
> > > > > cases?
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Also, it seems a
> TX
> > > > would
> > > > > only
> > > > > > > > >> hang,
> > > > > > > > >> > if
> > > > > > > > >> > > > >> there
> > > > > > > > >> > > > >> > >> is no
> > > > > > > > >> > > > >> > >> > > > > > following
> > > > > > > > >> > > > >> > >> > > > > > > > TX
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> that
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> is
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> either committer
> or
> > > > > aborted?
> > > > > > > Thus,
> > > > > > > > >> > for
> > > > > > > > >> > > > the
> > > > > > > > >> > > > >> > case
> > > > > > > > >> > > > >> > >> > > above,
> > > > > > > > >> > > > >> > >> > > > > the
> > > > > > > > >> > > > >> > >> > > > > > TX
> > > > > > > > >> > > > >> > >> > > > > > > > > might
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> actually not hang
> > (of
> > > > > course,
> > > > > > > we
> > > > > > > > >> > might
> > > > > > > > >> > > > get
> > > > > > > > >> > > > >> an
> > > > > > > > >> > > > >> > >> EOS
> > > > > > > > >> > > > >> > >> > > > > violation
> > > > > > > > >> > > > >> > >> > > > > > > if
> > > > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> first
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> TX was aborted and
> > the
> > > > > second
> > > > > > > > >> > > committed,
> > > > > > > > >> > > > or
> > > > > > > > >> > > > >> > the
> > > > > > > > >> > > > >> > >> > other
> > > > > > > > >> > > > >> > >> > > > way
> > > > > > > > >> > > > >> > >> > > > > > > > > around).
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (20)
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Of course, 1 and
> 2
> > > > > require
> > > > > > > > >> > client-side
> > > > > > > > >> > > > >> > >> changes, so
> > > > > > > > >> > > > >> > >> > > for
> > > > > > > > >> > > > >> > >> > > > > > older
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> clients,
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> those approaches
> > won’t
> > > > > apply.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> For (1) I
> understand
> > > > why a
> > > > > > > client
> > > > > > > > >> > > change
> > > > > > > > >> > > > is
> > > > > > > > >> > > > >> > >> > > necessary,
> > > > > > > > >> > > > >> > >> > > > > but
> > > > > > > > >> > > > >> > >> > > > > > > not
> > > > > > > > >> > > > >> > >> > > > > > > > > sure
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> why
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> we need a client
> > > change
> > > > > for
> > > > > > > (2).
> > > > > > > > >> Can
> > > > > > > > >> > > you
> > > > > > > > >> > > > >> > >> elaborate?
> > > > > > > > >> > > > >> > >> > > --
> > > > > > > > >> > > > >> > >> > > > > > Later
> > > > > > > > >> > > > >> > >> > > > > > > > you
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> explain
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> that we should
> send
> > a
> > > > > > > > >> > > > >> > >> DescribeTransactionRequest,
> > > > > > > > >> > > > >> > >> > > but I
> > > > > > > > >> > > > >> > >> > > > > am
> > > > > > > > >> > > > >> > >> > > > > > > not
> > > > > > > > >> > > > >> > >> > > > > > > > > sure
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> why?
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Can't we not just
> do
> > > an
> > > > > > > implicit
> > > > > > > > >> > > > >> > >> AddPartiitonToTx,
> > > > > > > > >> > > > >> > >> > > too?
> > > > > > > > >> > > > >> > >> > > > > If
> > > > > > > > >> > > > >> > >> > > > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > old
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> producer correctly
> > > > > registered
> > > > > > > the
> > > > > > > > >> > > > partition
> > > > > > > > >> > > > >> > >> > already,
> > > > > > > > >> > > > >> > >> > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> TX-coordinator
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> can just ignore it
> > as
> > > > > it's an
> > > > > > > > >> > > idempotent
> > > > > > > > >> > > > >> > >> operation?
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (30)
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> To cover older
> > > clients,
> > > > > we
> > > > > > > will
> > > > > > > > >> > > ensure a
> > > > > > > > >> > > > >> > >> > transaction
> > > > > > > > >> > > > >> > >> > > > is
> > > > > > > > >> > > > >> > >> > > > > > > > ongoing
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> before
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> we write to a
> > > > transaction
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Not sure what you
> > mean
> > > > by
> > > > > > > this?
> > > > > > > > >> Can
> > > > > > > > >> > you
> > > > > > > > >> > > > >> > >> elaborate?
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (40)
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> [the
> > TX-coordinator]
> > > > will
> > > > > > > write
> > > > > > > > >> the
> > > > > > > > >> > > > >> prepare
> > > > > > > > >> > > > >> > >> commit
> > > > > > > > >> > > > >> > >> > > > > message
> > > > > > > > >> > > > >> > >> > > > > > > > with
> > > > > > > > >> > > > >> > >> > > > > > > > > a
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> bumped
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> epoch and send
> > > > > > > > >> WriteTxnMarkerRequests
> > > > > > > > >> > > > with
> > > > > > > > >> > > > >> the
> > > > > > > > >> > > > >> > >> > bumped
> > > > > > > > >> > > > >> > >> > > > > > epoch.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Why do we use the
> > > bumped
> > > > > > > epoch for
> > > > > > > > >> > > both?
> > > > > > > > >> > > > It
> > > > > > > > >> > > > >> > >> seems
> > > > > > > > >> > > > >> > >> > > more
> > > > > > > > >> > > > >> > >> > > > > > > > intuitive
> > > > > > > > >> > > > >> > >> > > > > > > > > to
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> use
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> the current epoch,
> > and
> > > > > only
> > > > > > > return
> > > > > > > > >> > the
> > > > > > > > >> > > > >> bumped
> > > > > > > > >> > > > >> > >> epoch
> > > > > > > > >> > > > >> > >> > > to
> > > > > > > > >> > > > >> > >> > > > > the
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> producer?
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (50) "Implicit
> > > > > > > > >> > > AddPartitionToTransaction"
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Why does the
> > > implicitly
> > > > > sent
> > > > > > > > >> request
> > > > > > > > >> > > need
> > > > > > > > >> > > > >> to
> > > > > > > > >> > > > >> > be
> > > > > > > > >> > > > >> > >> > > > > > synchronous?
> > > > > > > > >> > > > >> > >> > > > > > > > The
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> KIP
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> also says
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> in case we need
> to
> > > > abort
> > > > > and
> > > > > > > > >> need to
> > > > > > > > >> > > > know
> > > > > > > > >> > > > >> > which
> > > > > > > > >> > > > >> > >> > > > > partitions
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> What do you mean
> by
> > > > this?
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> we don’t want to
> > > write
> > > > > to it
> > > > > > > > >> before
> > > > > > > > >> > we
> > > > > > > > >> > > > >> store
> > > > > > > > >> > > > >> > in
> > > > > > > > >> > > > >> > >> > the
> > > > > > > > >> > > > >> > >> > > > > > > > transaction
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> manager
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Do you mean
> > > > TX-coordinator
> > > > > > > > >> instead of
> > > > > > > > >> > > > >> > "manager"?
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (60)
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> For older clients
> > and
> > > > > ensuring
> > > > > > > > >> that
> > > > > > > > >> > the
> > > > > > > > >> > > > TX
> > > > > > > > >> > > > >> is
> > > > > > > > >> > > > >> > >> > > ongoing,
> > > > > > > > >> > > > >> > >> > > > > you
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>> describe a
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> race condition. I
> am
> > > not
> > > > > sure
> > > > > > > if I
> > > > > > > > >> > can
> > > > > > > > >> > > > >> follow
> > > > > > > > >> > > > >> > >> here.
> > > > > > > > >> > > > >> > >> > > Can
> > > > > > > > >> > > > >> > >> > > > > you
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> elaborate?
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> -Matthias
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> On 11/18/22 1:21
> PM,
> > > > > Justine
> > > > > > > > >> Olshan
> > > > > > > > >> > > > wrote:
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Hey all!
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> I'd like to
> start a
> > > > > > > discussion
> > > > > > > > >> on my
> > > > > > > > >> > > > >> proposal
> > > > > > > > >> > > > >> > >> to
> > > > > > > > >> > > > >> > >> > add
> > > > > > > > >> > > > >> > >> > > > > some
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> server-side
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> checks on
> > > transactions
> > > > to
> > > > > > > avoid
> > > > > > > > >> > > hanging
> > > > > > > > >> > > > >> > >> > > transactions.
> > > > > > > > >> > > > >> > >> > > > I
> > > > > > > > >> > > > >> > >> > > > > > know
> > > > > > > > >> > > > >> > >> > > > > > > > > this
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> has
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> been
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> an issue for some
> > > time,
> > > > > so I
> > > > > > > > >> really
> > > > > > > > >> > > hope
> > > > > > > > >> > > > >> this
> > > > > > > > >> > > > >> > >> KIP
> > > > > > > > >> > > > >> > >> > > will
> > > > > > > > >> > > > >> > >> > > > > be
> > > > > > > > >> > > > >> > >> > > > > > > > > helpful
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> for
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> many
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> users of EOS.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> The KIP includes
> > > > changes
> > > > > that
> > > > > > > > >> will
> > > > > > > > >> > be
> > > > > > > > >> > > > >> > >> compatible
> > > > > > > > >> > > > >> > >> > > with
> > > > > > > > >> > > > >> > >> > > > > old
> > > > > > > > >> > > > >> > >> > > > > > > > > clients
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> and
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> changes to
> improve
> > > > > > > performance
> > > > > > > > >> and
> > > > > > > > >> > > > >> > correctness
> > > > > > > > >> > > > >> > >> on
> > > > > > > > >> > > > >> > >> > > new
> > > > > > > > >> > > > >> > >> > > > > > > clients.
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Please take a
> look
> > > and
> > > > > leave
> > > > > > > any
> > > > > > > > >> > > > comments
> > > > > > > > >> > > > >> you
> > > > > > > > >> > > > >> > >> may
> > > > > > > > >> > > > >> > >> > > > have!
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> KIP:
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > >
> > > > > > > > >> > > > >> > >> > > > >
> > > > > > > > >> > > > >> > >> > > >
> > > > > > > > >> > > > >> > >> > >
> > > > > > > > >> > > > >> > >> >
> > > > > > > > >> > > > >> > >>
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >>
> > > > > > > > >> > > >
> > > > > > > > >> > >
> > > > > > > > >> >
> > > > > > > > >>
> > > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> JIRA:
> > > > > > > > >> > > > >> > >> > > >
> > > > > https://issues.apache.org/jira/browse/KAFKA-14402
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Thanks!
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Justine
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > > >> > > > >> > >> > > > > >
> > > > > > > > >> > > > >> > >> > > > >
> > > > > > > > >> > > > >> > >> > > >
> > > > > > > > >> > > > >> > >> > >
> > > > > > > > >> > > > >> > >> >
> > > > > > > > >> > > > >> > >>
> > > > > > > > >> > > > >> > >
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >
> > > > > > > > >> > > >
> > > > > > > > >> > >
> > > > > > > > >> >
> > > > > > > > >>
> > > > > > > > >
> > > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Artem Livshits <al...@confluent.io.INVALID>.
>  besides the poorly written client case

A poorly written client could create a lot of grief to people who run Kafka
brokers :-), so when deciding to make an error fatal I would see if there
is a reasonable recovery path rather than how often it could happen.  If we
have solid implementation of transactions (which I hope we'll do as a
result of this KIP), it would help to recover from a large class of errors
by just aborting a transaction, even if the cause of error is a race
condition or etc.

-Artem

On Fri, Jan 20, 2023 at 3:26 PM Justine Olshan <jo...@confluent.io.invalid>
wrote:

> Artem --
> I guess the discussion path we were going down is when we expect to see
> this error. I mentioned that it was hard to come up with cases for when the
> producer would still be around to receive the error besides the poorly
> written client case.
> If we don't expect to have a producer to receive the response, it sort of
> makes sense for it to be fatal.
>
> I had some discussion with Jason offline about the epoch being off cases
> and I'm not sure we could find a ton (outside of produce requests) where we
> could/should recover. I'd be happy to hear some examples though, maybe I'm
> missing something.
>
> Thanks,
> Justine
>
> On Fri, Jan 20, 2023 at 3:19 PM Artem Livshits
> <al...@confluent.io.invalid> wrote:
>
> > In general, I'd like to avoid fatal errors as much as possible, in some
> > sense fatal errors just push out recovery logic to the application which
> > either complicates the application or leads to disruption (we've seen
> cases
> > when a transient broker error could lead to work stoppage when
> applications
> > need to be manually restarted).  I think we should strive to define
> > recovery logic for most errors (and/or encapsulate it in the Kafka client
> > as much as possible).
> >
> > One benefit of transactions is that they simplify recovery from errors,
> > pretty much any error (that's not handled transparently by retries in
> Kafka
> > client) can be handled by the application via aborting the transaction
> and
> > repeating the transactional logic again.  One tricky error is an error
> > during commit, because we don't know the outcome.  For commit errors, the
> > recommendation should be to retry the commit until it returns the
> specific
> > result (committed or aborted).
> >
> > -Artem
> >
> > On Fri, Jan 20, 2023 at 2:52 PM Justine Olshan
> > <jo...@confluent.io.invalid>
> > wrote:
> >
> > > That's a fair point about other clients.
> > >
> > > I think the abortable error case is interesting because I'm curious how
> > > other clients would handle this. I assume they would need to implement
> > > handling for the error code unless they did something like "any unknown
> > > error codes/any codes that aren't x,y,z are retriable." I would hope
> that
> > > unknown error codes were fatal, and if the code was implemented it
> would
> > > abort the transaction. But I will think on this too.
> > >
> > > As for InvalidRecord -- you mentioned it was not fatal, but I'm taking
> a
> > > look through the code. We would see this on handling the produce
> > response.
> > > If I recall correctly, we check if errors are retriable. I think this
> > error
> > > would not be retriable. But I guess the concern here is that it is not
> > > enough for just that batch to fail. I guess I hadn't considered fully
> > > fencing the old producer but there are valid arguments here why we
> would
> > > want to.
> > >
> > > Thanks,
> > > Justine
> > >
> > > On Fri, Jan 20, 2023 at 2:35 PM Guozhang Wang <
> > guozhang.wang.us@gmail.com>
> > > wrote:
> > >
> > > > Thanks Justine for the replies! I agree with most of your thoughts.
> > > >
> > > > Just for 3/7), though I agree for our own AK producer, since we do
> > > > "nextRequest(boolean hasIncompleteBatches)", we guarantee the end-txn
> > > > would not be sent until we've effectively flushed, but I was
> referring
> > > > to any future bugs or other buggy clients that the same client may
> get
> > > > into this situation, in which case we should give the client a clear
> > > > msg that "you did something wrong, and hence now you should fatally
> > > > close yourself". What I'm concerned about is that, by seeing an
> > > > "abortable error" or in some rare cases an "invalid record", the
> > > > client could not realize "something that's really bad happened". So
> > > > it's not about adding a new error, it's mainly about those real buggy
> > > > situations causing such "should never happen" cases, the errors
> return
> > > > would not be informative enough.
> > > >
> > > > Thinking in other ways, if we believe that for most cases such error
> > > > codes would not reach the original clients since they would be
> > > > disconnected or even gone by that time, and only in some rare cases
> > > > they would still be seen by the sending clients, then why not make
> > > > them more fatal and more specific than generic.
> > > >
> > > > Guozhang
> > > >
> > > > On Fri, Jan 20, 2023 at 1:59 PM Justine Olshan
> > > > <jo...@confluent.io.invalid> wrote:
> > > > >
> > > > > Hey Guozhang. Thanks for taking a look and for the detailed
> comments!
> > > > I'll
> > > > > do my best to address below.
> > > > >
> > > > > 1. I see what you are saying here, but I think I need to look
> through
> > > the
> > > > > sequence of events you mention. Typically we've seen this issue in
> a
> > > few
> > > > > cases.
> > > > >
> > > > >  One is when we have a producer disconnect when trying to produce.
> > > > > Typically in these cases, we abort the transaction. We've seen that
> > > after
> > > > > the markers are written, the disconnection can sometimes cause the
> > > > request
> > > > > to get flushed to the broker. In this case, we don't need client
> > > handling
> > > > > because the producer we are responding to is gone. We just needed
> to
> > > make
> > > > > sure we didn't write to the log on the broker side. I'm trying to
> > think
> > > > of
> > > > > a case where we do have the client to return to. I'd think the same
> > > > client
> > > > > couldn't progress to committing the transaction unless the produce
> > > > request
> > > > > returned right? Of course, there is the incorrectly written clients
> > > case.
> > > > > I'll think on this a bit more and let you know if I come up with
> > > another
> > > > > scenario when we would return to an active client when the
> > transaction
> > > is
> > > > > no longer ongoing.
> > > > >
> > > > > I was not aware that we checked the result of a send after we
> commit
> > > > > though. I'll need to look into that a bit more.
> > > > >
> > > > > 2. There were some questions about this in the discussion. The plan
> > is
> > > to
> > > > > handle overflow with the mechanism we currently have in the
> producer.
> > > If
> > > > we
> > > > > try to bump and the epoch will overflow, we actually allocate a new
> > > > > producer ID. I need to confirm the fencing logic on the last epoch
> > (ie,
> > > > we
> > > > > probably shouldn't allow any records to be produced with the final
> > > epoch
> > > > > since we can never properly fence that one).
> > > > >
> > > > > 3. I can agree with you that the current error handling is messy. I
> > > > recall
> > > > > taking a look at your KIP a while back, but I think I mostly saw
> the
> > > > > section about how the errors were wrapped. Maybe I need to take
> > another
> > > > > look. As for abortable error, the idea was that the handling would
> be
> > > > > simple -- if this error is seen, the transaction should be aborted
> --
> > > no
> > > > > other logic about previous state or requests necessary. Is your
> > concern
> > > > > simply about adding new errors? We were hoping to have an error
> that
> > > > would
> > > > > have one meaning and many of the current errors have a history of
> > > meaning
> > > > > different things on different client versions. That was the main
> > > > motivation
> > > > > for adding a new error.
> > > > >
> > > > > 4. This is a good point about record timestamp reordering.
> Timestamps
> > > > don't
> > > > > affect compaction, but they do affect retention deletion. For that,
> > > kafka
> > > > > considers the largest timestamp in the segment, so I think a small
> > > amount
> > > > > of reordering (hopefully on the order of milliseconds or even
> > seconds)
> > > > will
> > > > > be ok. We take timestamps from clients so there is already a
> > > possibility
> > > > > for some drift and non-monotonically increasing timestamps.
> > > > >
> > > > > 5. Thanks for catching. The error is there, but it's actually that
> > > those
> > > > > fields should be 4+! Due to how the message generator works, I
> > actually
> > > > > have to redefine those fields inside the
> > > `"AddPartitionsToTxnTransaction`
> > > > > block for it to build correctly. I'll fix it to be correct.
> > > > >
> > > > > 6. Correct -- we will only add the request to purgatory if the
> cache
> > > has
> > > > no
> > > > > ongoing transaction. I can change the wording to make that clearer
> > that
> > > > we
> > > > > only place the request in purgatory if we need to contact the
> > > transaction
> > > > > coordinator.
> > > > >
> > > > > 7. We did take a look at some of the errors and it was hard to come
> > up
> > > > with
> > > > > a good one. I agree that InvalidTxnStateException is ideal except
> for
> > > the
> > > > > fact that it hasn't been returned on Produce requests before. The
> > error
> > > > > handling for clients is a bit vague (which is why I opened
> > KAFKA-14439
> > > > > <https://issues.apache.org/jira/browse/KAFKA-14439>), but the
> > decision
> > > > we
> > > > > made here was to only return errors that have been previously
> > returned
> > > to
> > > > > producers. As for not being fatal, I think part of the theory was
> > that
> > > in
> > > > > many cases, the producer would be disconnected. (See point 1) and
> > this
> > > > > would just be an error to return from the server. I did plan to
> think
> > > > about
> > > > > other cases, so let me know if you think of any as well!
> > > > >
> > > > > Lots to say! Let me know if you have further thoughts!
> > > > > Justine
> > > > >
> > > > > On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang <
> > > > guozhang.wang.us@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hello Justine,
> > > > > >
> > > > > > Thanks for the great write-up! I made a quick pass through it and
> > > here
> > > > > > are some thoughts (I have not been able to read through this
> thread
> > > so
> > > > > > pardon me if they have overlapped or subsumed by previous
> > comments):
> > > > > >
> > > > > > First are some meta ones:
> > > > > >
> > > > > > 1. I think we need to also improve the client's experience once
> we
> > > > > > have this defence in place. More concretely, say a user's
> producer
> > > > > > code is like following:
> > > > > >
> > > > > > future = producer.send();
> > > > > > // producer.flush();
> > > > > > producer.commitTransaction();
> > > > > > future.get();
> > > > > >
> > > > > > Which resulted in the order of a) produce-request sent by
> producer,
> > > b)
> > > > > > end-txn-request sent by producer, c) end-txn-response sent back,
> d)
> > > > > > txn-marker-request sent from coordinator to partition leader, e)
> > > > > > produce-request finally received by the partition leader, before
> > this
> > > > > > KIP e) step would be accepted causing a dangling txn; now it
> would
> > be
> > > > > > rejected in step e) which is good. But from the client's point of
> > > view
> > > > > > now it becomes confusing since the `commitTransaction()` returns
> > > > > > successfully, but the "future" throws an invalid-epoch error, and
> > > they
> > > > > > are not sure if the transaction did succeed or not. In fact, it
> > > > > > "partially succeeded" with some msgs being rejected but others
> > > > > > committed successfully.
> > > > > >
> > > > > > Of course the easy way to avoid this is, always call
> > > > > > "producer.flush()" before commitTxn and that's what we do
> > ourselves,
> > > > > > and what we recommend users do. But I suspect not everyone does
> it.
> > > In
> > > > > > fact I just checked the javadoc in KafkaProducer and our code
> > snippet
> > > > > > does not include a `flush()` call. So I'm thinking maybe we can
> in
> > > > > > side the `commitTxn` code to enforce flushing before sending the
> > > > > > end-txn request.
> > > > > >
> > > > > > 2. I'd like to clarify a bit details on "just add partitions to
> the
> > > > > > transaction on the first produce request during a transaction".
> My
> > > > > > understanding is that the partition leader's cache has the
> producer
> > > id
> > > > > > / sequence / epoch for the latest txn, either on-going or is
> > > completed
> > > > > > (upon receiving the marker request from coordinator). When a
> > produce
> > > > > > request is received, if
> > > > > >
> > > > > > * producer's epoch < cached epoch, or producer's epoch == cached
> > > epoch
> > > > > > but the latest txn is completed, leader directly reject with
> > > > > > invalid-epoch.
> > > > > > * producer's epoch > cached epoch, park the the request and send
> > > > > > add-partitions request to coordinator.
> > > > > >
> > > > > > In order to do it, does the coordinator need to bump the sequence
> > and
> > > > > > reset epoch to 0 when the next epoch is going to overflow? If no
> > need
> > > > > > to do so, then how we handle the (admittedly rare, but still may
> > > > > > happen) epoch overflow situation?
> > > > > >
> > > > > > 3. I'm a bit concerned about adding a generic "ABORTABLE_ERROR"
> > given
> > > > > > we already have a pretty messy error classification and error
> > > handling
> > > > > > on the producer clients side --- I have a summary about the
> issues
> > > and
> > > > > > a proposal to address this in
> > > > > >
> > > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
> > > > > > -- I understand we do not want to use "UNKNOWN_PRODUCER_ID"
> anymore
> > > > > > and in fact we intend to deprecate it in KIP-360 and eventually
> > > remove
> > > > > > it; but I'm wondering can we still use specific error codes. E.g.
> > > what
> > > > > > about "InvalidProducerEpochException" since for new clients, the
> > > > > > actual reason this would actually be rejected is indeed because
> the
> > > > > > epoch on the coordinator caused the add-partitions-request from
> the
> > > > > > brokers to be rejected anyways?
> > > > > >
> > > > > > 4. It seems we put the producer request into purgatory before we
> > ever
> > > > > > append the records, while other producer's records may still be
> > > > > > appended during the time; and that potentially may result in some
> > > > > > re-ordering compared with reception order. I'm not super
> concerned
> > > > > > about it since Kafka does not guarantee reception ordering across
> > > > > > producers anyways, but it may make the timestamps of records
> > inside a
> > > > > > partition to be more out-of-ordered. Are we aware of any
> scenarios
> > > > > > such as future enhancements on log compactions that may be
> affected
> > > by
> > > > > > this effect?
> > > > > >
> > > > > > Below are just minor comments:
> > > > > >
> > > > > > 5. In "AddPartitionsToTxnTransaction" field of
> > > > > > "AddPartitionsToTxnRequest" RPC, the versions of those inner
> fields
> > > > > > are "0-3" while I thought they should be "0+" still?
> > > > > >
> > > > > > 6. Regarding "we can place the request in a purgatory of sorts
> and
> > > > > > check if there is any state for the transaction on the broker": i
> > > > > > think at this time when we just do the checks against the cached
> > > > > > state, we do not need to put the request to purgatory yet?
> > > > > >
> > > > > > 7. This is related to 3) above. I feel using
> > "InvalidRecordException"
> > > > > > for older clients may also be a bit confusing, and also it is not
> > > > > > fatal -- for old clients, it better to be fatal since this
> > indicates
> > > > > > the clients is doing something wrong and hence it should be
> closed.
> > > > > > And in general I'd prefer to use slightly more specific meaning
> > error
> > > > > > codes for clients. That being said, I also feel
> > > > > > "InvalidProducerEpochException" is not suitable for old versioned
> > > > > > clients, and we'd have to pick one that old clients recognize.
> I'd
> > > > > > prefer "InvalidTxnStateException" but that one is supposed to be
> > > > > > returned from txn coordinators only today. I'd suggest we do a
> > quick
> > > > > > check in the current client's code path and see if that one would
> > be
> > > > > > handled if it's from a produce-response, and if yes, use this
> one;
> > > > > > otherwise, use "ProducerFencedException" which is much less
> > > meaningful
> > > > > > but it's still a fatal error.
> > > > > >
> > > > > >
> > > > > > Thanks,
> > > > > > Guozhang
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
> > > > > > <jo...@confluent.io.invalid> wrote:
> > > > > > >
> > > > > > > Yeah -- looks like we already have code to handle bumping the
> > epoch
> > > > and
> > > > > > > when the epoch is Short.MAX_VALUE, we get a new producer ID.
> > Since
> > > > this
> > > > > > is
> > > > > > > already the behavior, do we want to change it further?
> > > > > > >
> > > > > > > Justine
> > > > > > >
> > > > > > > On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <
> > > jolshan@confluent.io
> > > > >
> > > > > > wrote:
> > > > > > >
> > > > > > > > Hey all, just wanted to quickly update and say I've modified
> > the
> > > > KIP to
> > > > > > > > explicitly mention that AddOffsetCommitsToTxnRequest will be
> > > > replaced
> > > > > > by
> > > > > > > > a coordinator-side (inter-broker) AddPartitionsToTxn implicit
> > > > request.
> > > > > > This
> > > > > > > > mirrors the user partitions and will implicitly add offset
> > > > partitions
> > > > > > to
> > > > > > > > transactions when we commit offsets on them. We will
> deprecate
> > > > > > AddOffsetCommitsToTxnRequest
> > > > > > > > for new clients.
> > > > > > > >
> > > > > > > > Also to address Artem's comments --
> > > > > > > > I'm a bit unsure if the changes here will change the previous
> > > > behavior
> > > > > > for
> > > > > > > > fencing producers. In the case you mention in the first
> > > paragraph,
> > > > are
> > > > > > you
> > > > > > > > saying we bump the epoch before we try to abort the
> > transaction?
> > > I
> > > > > > think I
> > > > > > > > need to understand the scenarios you mention a bit better.
> > > > > > > >
> > > > > > > > As for the second part -- I think it makes sense to have some
> > > sort
> > > > of
> > > > > > > > "sentinel" epoch to signal epoch is about to overflow (I
> think
> > we
> > > > sort
> > > > > > of
> > > > > > > > have this value in place in some ways) so we can codify it in
> > the
> > > > KIP.
> > > > > > I'll
> > > > > > > > look into that and try to update soon.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Justine.
> > > > > > > >
> > > > > > > > On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
> > > > > > > > <al...@confluent.io.invalid> wrote:
> > > > > > > >
> > > > > > > >> It's good to know that KIP-588 addressed some of the issues.
> > > > Looking
> > > > > > at
> > > > > > > >> the code, it still looks like there are some cases that
> would
> > > > result
> > > > > > in
> > > > > > > >> fatal error, e.g. PRODUCER_FENCED is issued by the
> transaction
> > > > > > coordinator
> > > > > > > >> if epoch doesn't match, and the client treats it as a fatal
> > > error
> > > > > > (code in
> > > > > > > >> TransactionManager request handling).  If we consider, for
> > > > example,
> > > > > > > >> committing a transaction that returns a timeout, but
> actually
> > > > > > succeeds,
> > > > > > > >> trying to abort it or re-commit may result in
> PRODUCER_FENCED
> > > > error
> > > > > > > >> (because of epoch bump).
> > > > > > > >>
> > > > > > > >> For failed commits, specifically, we need to know the actual
> > > > outcome,
> > > > > > > >> because if we return an error the application may think that
> > the
> > > > > > > >> transaction is aborted and redo the work, leading to
> > duplicates.
> > > > > > > >>
> > > > > > > >> Re: overflowing epoch.  We could either do it on the TC and
> > > return
> > > > > > both
> > > > > > > >> producer id and epoch (e.g. change the protocol), or signal
> > the
> > > > client
> > > > > > > >> that
> > > > > > > >> it needs to get a new producer id.  Checking for max epoch
> > could
> > > > be a
> > > > > > > >> reasonable signal, the value to check should probably be
> > present
> > > > in
> > > > > > the
> > > > > > > >> KIP
> > > > > > > >> as this is effectively a part of the contract.  Also, the TC
> > > > should
> > > > > > > >> probably return an error if the client didn't change
> producer
> > id
> > > > after
> > > > > > > >> hitting max epoch.
> > > > > > > >>
> > > > > > > >> -Artem
> > > > > > > >>
> > > > > > > >>
> > > > > > > >> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
> > > > > > > >> <jo...@confluent.io.invalid> wrote:
> > > > > > > >>
> > > > > > > >> > Thanks for the discussion Artem.
> > > > > > > >> >
> > > > > > > >> > With respect to the handling of fenced producers, we have
> > some
> > > > > > behavior
> > > > > > > >> > already in place. As of KIP-588:
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >>
> > > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> > > > > > > >> > ,
> > > > > > > >> > we handle timeouts more gracefully. The producer can
> > recover.
> > > > > > > >> >
> > > > > > > >> > Produce requests can also recover from epoch fencing by
> > > > aborting the
> > > > > > > >> > transaction and starting over.
> > > > > > > >> >
> > > > > > > >> > What other cases were you considering that would cause us
> to
> > > > have a
> > > > > > > >> fenced
> > > > > > > >> > epoch but we'd want to recover?
> > > > > > > >> >
> > > > > > > >> > The first point about handling epoch overflows is fair. I
> > > think
> > > > > > there is
> > > > > > > >> > some logic we'd need to consider. (ie, if we are one away
> > from
> > > > the
> > > > > > max
> > > > > > > >> > epoch, we need to reset the producer ID.) I'm still
> > wondering
> > > if
> > > > > > there
> > > > > > > >> is a
> > > > > > > >> > way to direct this from the response, or if everything
> > should
> > > be
> > > > > > done on
> > > > > > > >> > the client side. Let me know if you have any thoughts
> here.
> > > > > > > >> >
> > > > > > > >> > Thanks,
> > > > > > > >> > Justine
> > > > > > > >> >
> > > > > > > >> > On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
> > > > > > > >> > <al...@confluent.io.invalid> wrote:
> > > > > > > >> >
> > > > > > > >> > > There are some workflows in the client that are implied
> by
> > > > > > protocol
> > > > > > > >> > > changes, e.g.:
> > > > > > > >> > >
> > > > > > > >> > > - for new clients, epoch changes with every transaction
> > and
> > > > can
> > > > > > > >> overflow,
> > > > > > > >> > > in old clients this condition was handled transparently,
> > > > because
> > > > > > epoch
> > > > > > > >> > was
> > > > > > > >> > > bumped in InitProducerId and it would return a new
> > producer
> > > > id if
> > > > > > > >> epoch
> > > > > > > >> > > overflows, the new clients would need to implement some
> > > > workflow
> > > > > > to
> > > > > > > >> > refresh
> > > > > > > >> > > producer id
> > > > > > > >> > > - how to handle fenced producers, for new clients epoch
> > > > changes
> > > > > > with
> > > > > > > >> > every
> > > > > > > >> > > transaction, so in presence of failures during commits /
> > > > aborts,
> > > > > > the
> > > > > > > >> > > producer could get easily fenced, old clients would
> pretty
> > > > much
> > > > > > would
> > > > > > > >> get
> > > > > > > >> > > fenced when a new incarnation of the producer was
> > > initialized
> > > > with
> > > > > > > >> > > InitProducerId so it's ok to treat as a fatal error, the
> > new
> > > > > > clients
> > > > > > > >> > would
> > > > > > > >> > > need to implement some workflow to handle that error,
> > > > otherwise
> > > > > > they
> > > > > > > >> > could
> > > > > > > >> > > get fenced by themselves
> > > > > > > >> > > - in particular (as a subset of the previous issue),
> what
> > > > would
> > > > > > the
> > > > > > > >> > client
> > > > > > > >> > > do if it got a timeout during commit?  commit could've
> > > > succeeded
> > > > > > or
> > > > > > > >> > failed
> > > > > > > >> > >
> > > > > > > >> > > Not sure if this has to be defined in the KIP as
> > > implementing
> > > > > > those
> > > > > > > >> > > probably wouldn't require protocol changes, but we have
> > > > multiple
> > > > > > > >> > > implementations of Kafka clients, so probably would be
> > good
> > > to
> > > > > > have
> > > > > > > >> some
> > > > > > > >> > > client implementation guidance.  Could also be done as a
> > > > separate
> > > > > > doc.
> > > > > > > >> > >
> > > > > > > >> > > -Artem
> > > > > > > >> > >
> > > > > > > >> > > On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
> > > > > > > >> > <jolshan@confluent.io.invalid
> > > > > > > >> > > >
> > > > > > > >> > > wrote:
> > > > > > > >> > >
> > > > > > > >> > > > Hey all, I've updated the KIP to incorporate Jason's
> > > > > > suggestions.
> > > > > > > >> > > >
> > > > > > > >> > > >
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >>
> > > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > > > >> > > >
> > > > > > > >> > > >
> > > > > > > >> > > > 1. Use AddPartitionsToTxn + verify flag to check on
> old
> > > > clients
> > > > > > > >> > > > 2. Updated AddPartitionsToTxn API to support
> transaction
> > > > > > batching
> > > > > > > >> > > > 3. Mention IBP bump
> > > > > > > >> > > > 4. Mention auth change on new AddPartitionsToTxn
> > version.
> > > > > > > >> > > >
> > > > > > > >> > > > I'm planning on opening a vote soon.
> > > > > > > >> > > > Thanks,
> > > > > > > >> > > > Justine
> > > > > > > >> > > >
> > > > > > > >> > > > On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <
> > > > > > jolshan@confluent.io
> > > > > > > >> >
> > > > > > > >> > > > wrote:
> > > > > > > >> > > >
> > > > > > > >> > > > > Thanks Jason. Those changes make sense to me. I will
> > > > update
> > > > > > the
> > > > > > > >> KIP.
> > > > > > > >> > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > > > On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
> > > > > > > >> > > > <ja...@confluent.io.invalid>
> > > > > > > >> > > > > wrote:
> > > > > > > >> > > > >
> > > > > > > >> > > > >> Hey Justine,
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> > I was wondering about compatibility here. When we
> > > send
> > > > > > requests
> > > > > > > >> > > > >> between brokers, we want to ensure that the
> receiving
> > > > broker
> > > > > > > >> > > understands
> > > > > > > >> > > > >> the request (specifically the new fields).
> Typically
> > > > this is
> > > > > > done
> > > > > > > >> > via
> > > > > > > >> > > > >> IBP/metadata version.
> > > > > > > >> > > > >> I'm trying to think if there is a way around it but
> > I'm
> > > > not
> > > > > > sure
> > > > > > > >> > there
> > > > > > > >> > > > is.
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> Yes. I think we would gate usage of this behind an
> > IBP
> > > > bump.
> > > > > > Does
> > > > > > > >> > that
> > > > > > > >> > > > >> seem
> > > > > > > >> > > > >> reasonable?
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> > As for the improvements -- can you clarify how
> the
> > > > multiple
> > > > > > > >> > > > >> transactional
> > > > > > > >> > > > >> IDs would help here? Were you thinking of a case
> > where
> > > we
> > > > > > > >> wait/batch
> > > > > > > >> > > > >> multiple produce requests together? My
> understanding
> > > for
> > > > now
> > > > > > was
> > > > > > > >> 1
> > > > > > > >> > > > >> transactional ID and one validation per 1 produce
> > > > request.
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> Each call to `AddPartitionsToTxn` is essentially a
> > > write
> > > > to
> > > > > > the
> > > > > > > >> > > > >> transaction
> > > > > > > >> > > > >> log and must block on replication. The more we can
> > fit
> > > > into a
> > > > > > > >> single
> > > > > > > >> > > > >> request, the more writes we can do in parallel. The
> > > > > > alternative
> > > > > > > >> is
> > > > > > > >> > to
> > > > > > > >> > > > make
> > > > > > > >> > > > >> use of more connections, but usually we prefer
> > batching
> > > > > > since the
> > > > > > > >> > > > network
> > > > > > > >> > > > >> stack is not really optimized for high
> > > connection/request
> > > > > > loads.
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> > Finally with respect to the authorizations, I
> think
> > > it
> > > > > > makes
> > > > > > > >> sense
> > > > > > > >> > > to
> > > > > > > >> > > > >> skip
> > > > > > > >> > > > >> topic authorizations, but I'm a bit confused by the
> > > > "leader
> > > > > > ID"
> > > > > > > >> > field.
> > > > > > > >> > > > >> Wouldn't we just want to flag the request as from a
> > > > broker
> > > > > > (does
> > > > > > > >> it
> > > > > > > >> > > > matter
> > > > > > > >> > > > >> which one?).
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> We could also make it version-based. For the next
> > > > version, we
> > > > > > > >> could
> > > > > > > >> > > > >> require
> > > > > > > >> > > > >> CLUSTER auth. So clients would not be able to use
> the
> > > API
> > > > > > > >> anymore,
> > > > > > > >> > > which
> > > > > > > >> > > > >> is
> > > > > > > >> > > > >> probably what we want.
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> -Jason
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> > > > > > > >> > > > >> <jo...@confluent.io.invalid>
> > > > > > > >> > > > >> wrote:
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> > As a follow up, I was just thinking about the
> > > batching
> > > > a
> > > > > > bit
> > > > > > > >> more.
> > > > > > > >> > > > >> > I suppose if we have one request in flight and we
> > > > queue up
> > > > > > the
> > > > > > > >> > other
> > > > > > > >> > > > >> > produce requests in some sort of purgatory, we
> > could
> > > > send
> > > > > > > >> > > information
> > > > > > > >> > > > >> out
> > > > > > > >> > > > >> > for all of them rather than one by one. So that
> > would
> > > > be a
> > > > > > > >> benefit
> > > > > > > >> > > of
> > > > > > > >> > > > >> > batching partitions to add per transaction.
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > I'll need to think a bit more on the design of
> this
> > > > part
> > > > > > of the
> > > > > > > >> > KIP,
> > > > > > > >> > > > and
> > > > > > > >> > > > >> > will update the KIP in the next few days.
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > Thanks,
> > > > > > > >> > > > >> > Justine
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan <
> > > > > > > >> > > jolshan@confluent.io>
> > > > > > > >> > > > >> > wrote:
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > > Hey Jason -- thanks for the input -- I was just
> > > > digging
> > > > > > a bit
> > > > > > > >> > > deeper
> > > > > > > >> > > > >> into
> > > > > > > >> > > > >> > > the design + implementation of the validation
> > calls
> > > > here
> > > > > > and
> > > > > > > >> > what
> > > > > > > >> > > > you
> > > > > > > >> > > > >> say
> > > > > > > >> > > > >> > > makes sense.
> > > > > > > >> > > > >> > >
> > > > > > > >> > > > >> > > I was wondering about compatibility here. When
> we
> > > > send
> > > > > > > >> requests
> > > > > > > >> > > > >> > > between brokers, we want to ensure that the
> > > receiving
> > > > > > broker
> > > > > > > >> > > > >> understands
> > > > > > > >> > > > >> > > the request (specifically the new fields).
> > > Typically
> > > > > > this is
> > > > > > > >> > done
> > > > > > > >> > > > via
> > > > > > > >> > > > >> > > IBP/metadata version.
> > > > > > > >> > > > >> > > I'm trying to think if there is a way around it
> > but
> > > > I'm
> > > > > > not
> > > > > > > >> sure
> > > > > > > >> > > > there
> > > > > > > >> > > > >> > is.
> > > > > > > >> > > > >> > >
> > > > > > > >> > > > >> > > As for the improvements -- can you clarify how
> > the
> > > > > > multiple
> > > > > > > >> > > > >> transactional
> > > > > > > >> > > > >> > > IDs would help here? Were you thinking of a
> case
> > > > where we
> > > > > > > >> > > wait/batch
> > > > > > > >> > > > >> > > multiple produce requests together? My
> > > understanding
> > > > for
> > > > > > now
> > > > > > > >> > was 1
> > > > > > > >> > > > >> > > transactional ID and one validation per 1
> produce
> > > > > > request.
> > > > > > > >> > > > >> > >
> > > > > > > >> > > > >> > > Finally with respect to the authorizations, I
> > think
> > > > it
> > > > > > makes
> > > > > > > >> > sense
> > > > > > > >> > > > to
> > > > > > > >> > > > >> > skip
> > > > > > > >> > > > >> > > topic authorizations, but I'm a bit confused by
> > the
> > > > > > "leader
> > > > > > > >> ID"
> > > > > > > >> > > > field.
> > > > > > > >> > > > >> > > Wouldn't we just want to flag the request as
> > from a
> > > > > > broker
> > > > > > > >> (does
> > > > > > > >> > > it
> > > > > > > >> > > > >> > matter
> > > > > > > >> > > > >> > > which one?).
> > > > > > > >> > > > >> > >
> > > > > > > >> > > > >> > > I think I want to adopt these suggestions, just
> > had
> > > > a few
> > > > > > > >> > > questions
> > > > > > > >> > > > on
> > > > > > > >> > > > >> > the
> > > > > > > >> > > > >> > > details.
> > > > > > > >> > > > >> > >
> > > > > > > >> > > > >> > > Thanks,
> > > > > > > >> > > > >> > > Justine
> > > > > > > >> > > > >> > >
> > > > > > > >> > > > >> > > On Thu, Jan 5, 2023 at 5:05 PM Jason Gustafson
> > > > > > > >> > > > >> > <ja...@confluent.io.invalid>
> > > > > > > >> > > > >> > > wrote:
> > > > > > > >> > > > >> > >
> > > > > > > >> > > > >> > >> Hi Justine,
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> > >> Thanks for the proposal.
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> > >> I was thinking about the implementation a
> little
> > > > bit.
> > > > > > In the
> > > > > > > >> > > > current
> > > > > > > >> > > > >> > >> proposal, the behavior depends on whether we
> > have
> > > an
> > > > > > old or
> > > > > > > >> new
> > > > > > > >> > > > >> client.
> > > > > > > >> > > > >> > >> For
> > > > > > > >> > > > >> > >> old clients, we send `DescribeTransactions`
> and
> > > > verify
> > > > > > the
> > > > > > > >> > result
> > > > > > > >> > > > and
> > > > > > > >> > > > >> > for
> > > > > > > >> > > > >> > >> new clients, we send `AddPartitionsToTxn`. We
> > > might
> > > > be
> > > > > > able
> > > > > > > >> to
> > > > > > > >> > > > >> simplify
> > > > > > > >> > > > >> > >> the
> > > > > > > >> > > > >> > >> implementation if we can use the same request
> > > type.
> > > > For
> > > > > > > >> > example,
> > > > > > > >> > > > >> what if
> > > > > > > >> > > > >> > >> we
> > > > > > > >> > > > >> > >> bump the protocol version for
> > `AddPartitionsToTxn`
> > > > and
> > > > > > add a
> > > > > > > >> > > > >> > >> `validateOnly`
> > > > > > > >> > > > >> > >> flag? For older versions, we can set
> > > > > > `validateOnly=true` so
> > > > > > > >> > that
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > >> request only returns successfully if the
> > partition
> > > > had
> > > > > > > >> already
> > > > > > > >> > > been
> > > > > > > >> > > > >> > added.
> > > > > > > >> > > > >> > >> For new versions, we can set
> > `validateOnly=false`
> > > > and
> > > > > > the
> > > > > > > >> > > partition
> > > > > > > >> > > > >> will
> > > > > > > >> > > > >> > >> be
> > > > > > > >> > > > >> > >> added to the transaction. The other slightly
> > > > annoying
> > > > > > thing
> > > > > > > >> > that
> > > > > > > >> > > > this
> > > > > > > >> > > > >> > >> would
> > > > > > > >> > > > >> > >> get around is the need to collect the
> > transaction
> > > > state
> > > > > > for
> > > > > > > >> all
> > > > > > > >> > > > >> > partitions
> > > > > > > >> > > > >> > >> even when we only care about a subset.
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> > >> Some additional improvements to consider:
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> > >> - We can give `AddPartitionsToTxn` better
> batch
> > > > support
> > > > > > for
> > > > > > > >> > > > >> inter-broker
> > > > > > > >> > > > >> > >> usage. Currently we only allow one
> > > > `TransactionalId` to
> > > > > > be
> > > > > > > >> > > > specified,
> > > > > > > >> > > > >> > but
> > > > > > > >> > > > >> > >> the broker may get some benefit being able to
> > > batch
> > > > > > across
> > > > > > > >> > > multiple
> > > > > > > >> > > > >> > >> transactions.
> > > > > > > >> > > > >> > >> - Another small improvement is skipping topic
> > > > > > authorization
> > > > > > > >> > > checks
> > > > > > > >> > > > >> for
> > > > > > > >> > > > >> > >> `AddPartitionsToTxn` when the request is from
> a
> > > > broker.
> > > > > > > >> Perhaps
> > > > > > > >> > > we
> > > > > > > >> > > > >> can
> > > > > > > >> > > > >> > add
> > > > > > > >> > > > >> > >> a field for the `LeaderId` or something like
> > that
> > > > and
> > > > > > > >> require
> > > > > > > >> > > > CLUSTER
> > > > > > > >> > > > >> > >> permission when set.
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> > >> Best,
> > > > > > > >> > > > >> > >> Jason
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> > >> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
> > > > > > > >> > <jun@confluent.io.invalid
> > > > > > > >> > > >
> > > > > > > >> > > > >> > wrote:
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> > >> > Hi, Justine,
> > > > > > > >> > > > >> > >> >
> > > > > > > >> > > > >> > >> > Thanks for the explanation. It makes sense
> to
> > me
> > > > now.
> > > > > > > >> > > > >> > >> >
> > > > > > > >> > > > >> > >> > Jun
> > > > > > > >> > > > >> > >> >
> > > > > > > >> > > > >> > >> > On Mon, Dec 19, 2022 at 1:42 PM Justine
> Olshan
> > > > > > > >> > > > >> > >> > <jo...@confluent.io.invalid>
> > > > > > > >> > > > >> > >> > wrote:
> > > > > > > >> > > > >> > >> >
> > > > > > > >> > > > >> > >> > > Hi Jun,
> > > > > > > >> > > > >> > >> > >
> > > > > > > >> > > > >> > >> > > My understanding of the mechanism is that
> > when
> > > > we
> > > > > > get to
> > > > > > > >> > the
> > > > > > > >> > > > last
> > > > > > > >> > > > >> > >> epoch,
> > > > > > > >> > > > >> > >> > we
> > > > > > > >> > > > >> > >> > > increment to the fencing/last epoch and if
> > any
> > > > > > further
> > > > > > > >> > > requests
> > > > > > > >> > > > >> come
> > > > > > > >> > > > >> > >> in
> > > > > > > >> > > > >> > >> > for
> > > > > > > >> > > > >> > >> > > this producer ID they are fenced. Then the
> > > > producer
> > > > > > > >> gets a
> > > > > > > >> > > new
> > > > > > > >> > > > ID
> > > > > > > >> > > > >> > and
> > > > > > > >> > > > >> > >> > > restarts with epoch/sequence 0. The fenced
> > > epoch
> > > > > > sticks
> > > > > > > >> > > around
> > > > > > > >> > > > >> for
> > > > > > > >> > > > >> > the
> > > > > > > >> > > > >> > >> > > duration of producer.id.expiration.ms and
> > > > blocks
> > > > > > any
> > > > > > > >> late
> > > > > > > >> > > > >> messages
> > > > > > > >> > > > >> > >> > there.
> > > > > > > >> > > > >> > >> > > The new ID will get to take advantage of
> the
> > > > > > improved
> > > > > > > >> > > semantics
> > > > > > > >> > > > >> > around
> > > > > > > >> > > > >> > >> > > non-zero start sequences. So I think we
> are
> > > > covered.
> > > > > > > >> > > > >> > >> > >
> > > > > > > >> > > > >> > >> > > The only potential issue is overloading
> the
> > > > cache,
> > > > > > but
> > > > > > > >> > > > hopefully
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> > > improvements (lowered
> > > producer.id.expiration.ms
> > > > )
> > > > > > will
> > > > > > > >> help
> > > > > > > >> > > > with
> > > > > > > >> > > > >> > that.
> > > > > > > >> > > > >> > >> > Let
> > > > > > > >> > > > >> > >> > > me know if you still have concerns.
> > > > > > > >> > > > >> > >> > >
> > > > > > > >> > > > >> > >> > > Thanks,
> > > > > > > >> > > > >> > >> > > Justine
> > > > > > > >> > > > >> > >> > >
> > > > > > > >> > > > >> > >> > > On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
> > > > > > > >> > > > >> <ju...@confluent.io.invalid>
> > > > > > > >> > > > >> > >> > wrote:
> > > > > > > >> > > > >> > >> > >
> > > > > > > >> > > > >> > >> > > > Hi, Justine,
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > > >> > >> > > > Thanks for the explanation.
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > > >> > >> > > > 70. The proposed fencing logic doesn't
> > apply
> > > > when
> > > > > > pid
> > > > > > > >> > > > changes,
> > > > > > > >> > > > >> is
> > > > > > > >> > > > >> > >> that
> > > > > > > >> > > > >> > >> > > > right? If so, I am not sure how complete
> > we
> > > > are
> > > > > > > >> > addressing
> > > > > > > >> > > > this
> > > > > > > >> > > > >> > >> issue
> > > > > > > >> > > > >> > >> > if
> > > > > > > >> > > > >> > >> > > > the pid changes more frequently.
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > > >> > >> > > > Thanks,
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > > >> > >> > > > Jun
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > > >> > >> > > > On Fri, Dec 16, 2022 at 9:16 AM Justine
> > > Olshan
> > > > > > > >> > > > >> > >> > > > <jo...@confluent.io.invalid>
> > > > > > > >> > > > >> > >> > > > wrote:
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > > >> > >> > > > > Hi Jun,
> > > > > > > >> > > > >> > >> > > > >
> > > > > > > >> > > > >> > >> > > > > Thanks for replying!
> > > > > > > >> > > > >> > >> > > > >
> > > > > > > >> > > > >> > >> > > > > 70.We already do the overflow
> mechanism,
> > > so
> > > > my
> > > > > > > >> change
> > > > > > > >> > > would
> > > > > > > >> > > > >> just
> > > > > > > >> > > > >> > >> make
> > > > > > > >> > > > >> > >> > > it
> > > > > > > >> > > > >> > >> > > > > happen more often.
> > > > > > > >> > > > >> > >> > > > > I was also not suggesting a new field
> in
> > > the
> > > > > > log,
> > > > > > > >> but
> > > > > > > >> > in
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > >> > response,
> > > > > > > >> > > > >> > >> > > > > which would be gated by the client
> > > version.
> > > > > > Sorry if
> > > > > > > >> > > > >> something
> > > > > > > >> > > > >> > >> there
> > > > > > > >> > > > >> > >> > is
> > > > > > > >> > > > >> > >> > > > > unclear. I think we are starting to
> > > diverge.
> > > > > > > >> > > > >> > >> > > > > The goal of this KIP is to not change
> to
> > > the
> > > > > > marker
> > > > > > > >> > > format
> > > > > > > >> > > > at
> > > > > > > >> > > > >> > all.
> > > > > > > >> > > > >> > >> > > > >
> > > > > > > >> > > > >> > >> > > > > 71. Yes, I guess I was going under the
> > > > > > assumption
> > > > > > > >> that
> > > > > > > >> > > the
> > > > > > > >> > > > >> log
> > > > > > > >> > > > >> > >> would
> > > > > > > >> > > > >> > >> > > just
> > > > > > > >> > > > >> > >> > > > > look at its last epoch and treat it as
> > the
> > > > > > current
> > > > > > > >> > > epoch. I
> > > > > > > >> > > > >> > >> suppose
> > > > > > > >> > > > >> > >> > we
> > > > > > > >> > > > >> > >> > > > can
> > > > > > > >> > > > >> > >> > > > > have some special logic that if the
> last
> > > > epoch
> > > > > > was
> > > > > > > >> on a
> > > > > > > >> > > > >> marker
> > > > > > > >> > > > >> > we
> > > > > > > >> > > > >> > >> > > > actually
> > > > > > > >> > > > >> > >> > > > > expect the next epoch or something
> like
> > > > that. We
> > > > > > > >> just
> > > > > > > >> > > need
> > > > > > > >> > > > to
> > > > > > > >> > > > >> > >> > > distinguish
> > > > > > > >> > > > >> > >> > > > > based on whether we had a commit/abort
> > > > marker.
> > > > > > > >> > > > >> > >> > > > >
> > > > > > > >> > > > >> > >> > > > > 72.
> > > > > > > >> > > > >> > >> > > > > > if the producer epoch hasn't been
> > bumped
> > > > on
> > > > > > the
> > > > > > > >> > > > >> > >> > > > > broker, it seems that the stucked
> > message
> > > > will
> > > > > > fail
> > > > > > > >> the
> > > > > > > >> > > > >> sequence
> > > > > > > >> > > > >> > >> > > > validation
> > > > > > > >> > > > >> > >> > > > > and will be ignored. If the producer
> > epoch
> > > > has
> > > > > > been
> > > > > > > >> > > bumped,
> > > > > > > >> > > > >> we
> > > > > > > >> > > > >> > >> ignore
> > > > > > > >> > > > >> > >> > > the
> > > > > > > >> > > > >> > >> > > > > sequence check and the stuck message
> > could
> > > > be
> > > > > > > >> appended
> > > > > > > >> > to
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > log.
> > > > > > > >> > > > >> > >> > So,
> > > > > > > >> > > > >> > >> > > is
> > > > > > > >> > > > >> > >> > > > > the latter case that we want to guard?
> > > > > > > >> > > > >> > >> > > > >
> > > > > > > >> > > > >> > >> > > > > I'm not sure I follow that "the
> message
> > > will
> > > > > > fail
> > > > > > > >> the
> > > > > > > >> > > > >> sequence
> > > > > > > >> > > > >> > >> > > > validation".
> > > > > > > >> > > > >> > >> > > > > In some of these cases, we had an
> abort
> > > > marker
> > > > > > (due
> > > > > > > >> to
> > > > > > > >> > an
> > > > > > > >> > > > >> error)
> > > > > > > >> > > > >> > >> and
> > > > > > > >> > > > >> > >> > > then
> > > > > > > >> > > > >> > >> > > > > the late message comes in with the
> > correct
> > > > > > sequence
> > > > > > > >> > > number.
> > > > > > > >> > > > >> This
> > > > > > > >> > > > >> > >> is a
> > > > > > > >> > > > >> > >> > > > case
> > > > > > > >> > > > >> > >> > > > > covered by the KIP.
> > > > > > > >> > > > >> > >> > > > > The latter case is actually not
> > something
> > > > we've
> > > > > > > >> > > considered
> > > > > > > >> > > > >> > here. I
> > > > > > > >> > > > >> > >> > > think
> > > > > > > >> > > > >> > >> > > > > generally when we bump the epoch, we
> are
> > > > > > accepting
> > > > > > > >> that
> > > > > > > >> > > the
> > > > > > > >> > > > >> > >> sequence
> > > > > > > >> > > > >> > >> > > does
> > > > > > > >> > > > >> > >> > > > > not need to be checked anymore. My
> > > > > > understanding is
> > > > > > > >> > also
> > > > > > > >> > > > >> that we
> > > > > > > >> > > > >> > >> > don't
> > > > > > > >> > > > >> > >> > > > > typically bump epoch mid transaction
> > > (based
> > > > on a
> > > > > > > >> quick
> > > > > > > >> > > look
> > > > > > > >> > > > >> at
> > > > > > > >> > > > >> > the
> > > > > > > >> > > > >> > >> > > code)
> > > > > > > >> > > > >> > >> > > > > but let me know if that is the case.
> > > > > > > >> > > > >> > >> > > > >
> > > > > > > >> > > > >> > >> > > > > Thanks,
> > > > > > > >> > > > >> > >> > > > > Justine
> > > > > > > >> > > > >> > >> > > > >
> > > > > > > >> > > > >> > >> > > > > On Thu, Dec 15, 2022 at 12:23 PM Jun
> Rao
> > > > > > > >> > > > >> > <jun@confluent.io.invalid
> > > > > > > >> > > > >> > >> >
> > > > > > > >> > > > >> > >> > > > wrote:
> > > > > > > >> > > > >> > >> > > > >
> > > > > > > >> > > > >> > >> > > > > > Hi, Justine,
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> > > > >> > >> > > > > > Thanks for the reply.
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> > > > >> > >> > > > > > 70. Assigning a new pid on int
> > overflow
> > > > seems
> > > > > > a
> > > > > > > >> bit
> > > > > > > >> > > > hacky.
> > > > > > > >> > > > >> If
> > > > > > > >> > > > >> > we
> > > > > > > >> > > > >> > >> > > need a
> > > > > > > >> > > > >> > >> > > > > txn
> > > > > > > >> > > > >> > >> > > > > > level id, it will be better to model
> > > this
> > > > > > > >> explicitly.
> > > > > > > >> > > > >> Adding a
> > > > > > > >> > > > >> > >> new
> > > > > > > >> > > > >> > >> > > > field
> > > > > > > >> > > > >> > >> > > > > > would require a bit more work since
> it
> > > > > > requires a
> > > > > > > >> new
> > > > > > > >> > > txn
> > > > > > > >> > > > >> > marker
> > > > > > > >> > > > >> > >> > > format
> > > > > > > >> > > > >> > >> > > > > in
> > > > > > > >> > > > >> > >> > > > > > the log. So, we probably need to
> guard
> > > it
> > > > > > with an
> > > > > > > >> IBP
> > > > > > > >> > > or
> > > > > > > >> > > > >> > >> metadata
> > > > > > > >> > > > >> > >> > > > version
> > > > > > > >> > > > >> > >> > > > > > and document the impact on downgrade
> > > once
> > > > the
> > > > > > new
> > > > > > > >> > > format
> > > > > > > >> > > > is
> > > > > > > >> > > > >> > >> written
> > > > > > > >> > > > >> > >> > > to
> > > > > > > >> > > > >> > >> > > > > the
> > > > > > > >> > > > >> > >> > > > > > log.
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> > > > >> > >> > > > > > 71. Hmm, once the marker is written,
> > the
> > > > > > partition
> > > > > > > >> > will
> > > > > > > >> > > > >> expect
> > > > > > > >> > > > >> > >> the
> > > > > > > >> > > > >> > >> > > next
> > > > > > > >> > > > >> > >> > > > > > append to be on the next epoch. Does
> > > that
> > > > > > cover
> > > > > > > >> the
> > > > > > > >> > > case
> > > > > > > >> > > > >> you
> > > > > > > >> > > > >> > >> > > mentioned?
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> > > > >> > >> > > > > > 72. Also, just to be clear on the
> > > stucked
> > > > > > message
> > > > > > > >> > issue
> > > > > > > >> > > > >> > >> described
> > > > > > > >> > > > >> > >> > in
> > > > > > > >> > > > >> > >> > > > the
> > > > > > > >> > > > >> > >> > > > > > motivation. With EoS, we also
> validate
> > > the
> > > > > > > >> sequence
> > > > > > > >> > id
> > > > > > > >> > > > for
> > > > > > > >> > > > >> > >> > > idempotency.
> > > > > > > >> > > > >> > >> > > > > So,
> > > > > > > >> > > > >> > >> > > > > > with the current logic, if the
> > producer
> > > > epoch
> > > > > > > >> hasn't
> > > > > > > >> > > been
> > > > > > > >> > > > >> > >> bumped on
> > > > > > > >> > > > >> > >> > > the
> > > > > > > >> > > > >> > >> > > > > > broker, it seems that the stucked
> > > message
> > > > will
> > > > > > > >> fail
> > > > > > > >> > the
> > > > > > > >> > > > >> > sequence
> > > > > > > >> > > > >> > >> > > > > validation
> > > > > > > >> > > > >> > >> > > > > > and will be ignored. If the producer
> > > > epoch has
> > > > > > > >> been
> > > > > > > >> > > > >> bumped, we
> > > > > > > >> > > > >> > >> > ignore
> > > > > > > >> > > > >> > >> > > > the
> > > > > > > >> > > > >> > >> > > > > > sequence check and the stuck message
> > > > could be
> > > > > > > >> > appended
> > > > > > > >> > > to
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> log.
> > > > > > > >> > > > >> > >> > > So,
> > > > > > > >> > > > >> > >> > > > is
> > > > > > > >> > > > >> > >> > > > > > the latter case that we want to
> guard?
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> > > > >> > >> > > > > > Thanks,
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> > > > >> > >> > > > > > Jun
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> > > > >> > >> > > > > > On Wed, Dec 14, 2022 at 10:44 AM
> > Justine
> > > > > > Olshan
> > > > > > > >> > > > >> > >> > > > > > <jo...@confluent.io.invalid>
> wrote:
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> > > > >> > >> > > > > > > Matthias — thanks again for taking
> > > time
> > > > to
> > > > > > look
> > > > > > > >> a
> > > > > > > >> > > this.
> > > > > > > >> > > > >> You
> > > > > > > >> > > > >> > >> said:
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > My proposal was only focusing to
> > > avoid
> > > > > > > >> dangling
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > transactions if records are added
> > > > without
> > > > > > > >> > registered
> > > > > > > >> > > > >> > >> partition.
> > > > > > > >> > > > >> > >> > --
> > > > > > > >> > > > >> > >> > > > > Maybe
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > you can add a few more details to
> > the
> > > > KIP
> > > > > > about
> > > > > > > >> > this
> > > > > > > >> > > > >> > scenario
> > > > > > > >> > > > >> > >> for
> > > > > > > >> > > > >> > >> > > > > better
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > documentation purpose?
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > I'm not sure I understand what you
> > > mean
> > > > > > here.
> > > > > > > >> The
> > > > > > > >> > > > >> motivation
> > > > > > > >> > > > >> > >> > > section
> > > > > > > >> > > > >> > >> > > > > > > describes two scenarios about how
> > the
> > > > record
> > > > > > > >> can be
> > > > > > > >> > > > added
> > > > > > > >> > > > >> > >> > without a
> > > > > > > >> > > > >> > >> > > > > > > registered partition:
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > 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.
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > Another way hanging transactions
> > can
> > > > > > occur is
> > > > > > > >> > that
> > > > > > > >> > > a
> > > > > > > >> > > > >> > client
> > > > > > > >> > > > >> > >> is
> > > > > > > >> > > > >> > >> > > > buggy
> > > > > > > >> > > > >> > >> > > > > > and
> > > > > > > >> > > > >> > >> > > > > > > may somehow try to write to a
> > > partition
> > > > > > before
> > > > > > > >> it
> > > > > > > >> > > adds
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> > > partition
> > > > > > > >> > > > >> > >> > > > to
> > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > transaction.
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > For the first example of this
> would
> > it
> > > > be
> > > > > > > >> helpful
> > > > > > > >> > to
> > > > > > > >> > > > say
> > > > > > > >> > > > >> > that
> > > > > > > >> > > > >> > >> > this
> > > > > > > >> > > > >> > >> > > > > > message
> > > > > > > >> > > > >> > >> > > > > > > comes in after the abort, but
> before
> > > the
> > > > > > > >> partition
> > > > > > > >> > is
> > > > > > > >> > > > >> added
> > > > > > > >> > > > >> > to
> > > > > > > >> > > > >> > >> > the
> > > > > > > >> > > > >> > >> > > > next
> > > > > > > >> > > > >> > >> > > > > > > transaction so it becomes
> "hanging."
> > > > > > Perhaps the
> > > > > > > >> > next
> > > > > > > >> > > > >> > sentence
> > > > > > > >> > > > >> > >> > > > > describing
> > > > > > > >> > > > >> > >> > > > > > > the message becoming part of the
> > next
> > > > > > > >> transaction
> > > > > > > >> > (a
> > > > > > > >> > > > >> > different
> > > > > > > >> > > > >> > >> > > case)
> > > > > > > >> > > > >> > >> > > > > was
> > > > > > > >> > > > >> > >> > > > > > > not properly differentiated.
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > Jun — thanks for reading the KIP.
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > 70. The int typing was a concern.
> > > > Currently
> > > > > > we
> > > > > > > >> > have a
> > > > > > > >> > > > >> > >> mechanism
> > > > > > > >> > > > >> > >> > in
> > > > > > > >> > > > >> > >> > > > > place
> > > > > > > >> > > > >> > >> > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > fence the final epoch when the
> epoch
> > > is
> > > > > > about to
> > > > > > > >> > > > overflow
> > > > > > > >> > > > >> > and
> > > > > > > >> > > > >> > >> > > assign
> > > > > > > >> > > > >> > >> > > > a
> > > > > > > >> > > > >> > >> > > > > > new
> > > > > > > >> > > > >> > >> > > > > > > producer ID with epoch 0. Of
> course,
> > > > this
> > > > > > is a
> > > > > > > >> bit
> > > > > > > >> > > > tricky
> > > > > > > >> > > > >> > >> when it
> > > > > > > >> > > > >> > >> > > > comes
> > > > > > > >> > > > >> > >> > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > the response back to the client.
> > > > > > > >> > > > >> > >> > > > > > > Making this a long could be
> another
> > > > option,
> > > > > > but
> > > > > > > >> I
> > > > > > > >> > > > wonder
> > > > > > > >> > > > >> are
> > > > > > > >> > > > >> > >> > there
> > > > > > > >> > > > >> > >> > > > any
> > > > > > > >> > > > >> > >> > > > > > > implications on changing this
> field
> > if
> > > > the
> > > > > > > >> epoch is
> > > > > > > >> > > > >> > persisted
> > > > > > > >> > > > >> > >> to
> > > > > > > >> > > > >> > >> > > > disk?
> > > > > > > >> > > > >> > >> > > > > > I'd
> > > > > > > >> > > > >> > >> > > > > > > need to check the usages.
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > 71.This was something Matthias
> asked
> > > > about
> > > > > > as
> > > > > > > >> > well. I
> > > > > > > >> > > > was
> > > > > > > >> > > > >> > >> > > > considering a
> > > > > > > >> > > > >> > >> > > > > > > possible edge case where a produce
> > > > request
> > > > > > from
> > > > > > > >> a
> > > > > > > >> > new
> > > > > > > >> > > > >> > >> transaction
> > > > > > > >> > > > >> > >> > > > > somehow
> > > > > > > >> > > > >> > >> > > > > > > gets sent right after the marker
> is
> > > > > > written, but
> > > > > > > >> > > before
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> > > producer
> > > > > > > >> > > > >> > >> > > > is
> > > > > > > >> > > > >> > >> > > > > > > alerted of the newly bumped epoch.
> > In
> > > > this
> > > > > > > >> case, we
> > > > > > > >> > > may
> > > > > > > >> > > > >> > >> include
> > > > > > > >> > > > >> > >> > > this
> > > > > > > >> > > > >> > >> > > > > > record
> > > > > > > >> > > > >> > >> > > > > > > when we don't want to. I suppose
> we
> > > > could
> > > > > > try
> > > > > > > >> to do
> > > > > > > >> > > > >> > something
> > > > > > > >> > > > >> > >> > > client
> > > > > > > >> > > > >> > >> > > > > side
> > > > > > > >> > > > >> > >> > > > > > > to bump the epoch after sending an
> > > > endTxn as
> > > > > > > >> well
> > > > > > > >> > in
> > > > > > > >> > > > this
> > > > > > > >> > > > >> > >> > scenario
> > > > > > > >> > > > >> > >> > > —
> > > > > > > >> > > > >> > >> > > > > but
> > > > > > > >> > > > >> > >> > > > > > I
> > > > > > > >> > > > >> > >> > > > > > > wonder how it would work when the
> > > > server is
> > > > > > > >> > aborting
> > > > > > > >> > > > >> based
> > > > > > > >> > > > >> > on
> > > > > > > >> > > > >> > >> a
> > > > > > > >> > > > >> > >> > > > > > server-side
> > > > > > > >> > > > >> > >> > > > > > > error. I could also be missing
> > > > something and
> > > > > > > >> this
> > > > > > > >> > > > >> scenario
> > > > > > > >> > > > >> > is
> > > > > > > >> > > > >> > >> > > > actually
> > > > > > > >> > > > >> > >> > > > > > not
> > > > > > > >> > > > >> > >> > > > > > > possible.
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > Thanks again to everyone reading
> and
> > > > > > commenting.
> > > > > > > >> > Let
> > > > > > > >> > > me
> > > > > > > >> > > > >> know
> > > > > > > >> > > > >> > >> > about
> > > > > > > >> > > > >> > >> > > > any
> > > > > > > >> > > > >> > >> > > > > > > further questions or comments.
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > Justine
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > On Wed, Dec 14, 2022 at 9:41 AM
> Jun
> > > Rao
> > > > > > > >> > > > >> > >> <jun@confluent.io.invalid
> > > > > > > >> > > > >> > >> > >
> > > > > > > >> > > > >> > >> > > > > > wrote:
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > Hi, Justine,
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > Thanks for the KIP. A couple of
> > > > comments.
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > 70. Currently, the producer
> epoch
> > is
> > > > an
> > > > > > int.
> > > > > > > >> I am
> > > > > > > >> > > not
> > > > > > > >> > > > >> sure
> > > > > > > >> > > > >> > >> if
> > > > > > > >> > > > >> > >> > > it's
> > > > > > > >> > > > >> > >> > > > > > enough
> > > > > > > >> > > > >> > >> > > > > > > > to accommodate all transactions
> in
> > > the
> > > > > > > >> lifetime
> > > > > > > >> > of
> > > > > > > >> > > a
> > > > > > > >> > > > >> > >> producer.
> > > > > > > >> > > > >> > >> > > > Should
> > > > > > > >> > > > >> > >> > > > > > we
> > > > > > > >> > > > >> > >> > > > > > > > change that to a long or add a
> new
> > > > long
> > > > > > field
> > > > > > > >> > like
> > > > > > > >> > > > >> txnId?
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > 71. "it will write the prepare
> > > commit
> > > > > > message
> > > > > > > >> > with
> > > > > > > >> > > a
> > > > > > > >> > > > >> > bumped
> > > > > > > >> > > > >> > >> > epoch
> > > > > > > >> > > > >> > >> > > > and
> > > > > > > >> > > > >> > >> > > > > > > send
> > > > > > > >> > > > >> > >> > > > > > > > WriteTxnMarkerRequests with the
> > > bumped
> > > > > > epoch."
> > > > > > > >> > Hmm,
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > >> epoch
> > > > > > > >> > > > >> > >> > is
> > > > > > > >> > > > >> > >> > > > > > > associated
> > > > > > > >> > > > >> > >> > > > > > > > with the current txn right? So,
> it
> > > > seems
> > > > > > > >> weird to
> > > > > > > >> > > > >> write a
> > > > > > > >> > > > >> > >> > commit
> > > > > > > >> > > > >> > >> > > > > > message
> > > > > > > >> > > > >> > >> > > > > > > > with a bumped epoch. Should we
> > only
> > > > bump
> > > > > > up
> > > > > > > >> the
> > > > > > > >> > > epoch
> > > > > > > >> > > > >> in
> > > > > > > >> > > > >> > >> > > > > EndTxnResponse
> > > > > > > >> > > > >> > >> > > > > > > and
> > > > > > > >> > > > >> > >> > > > > > > > rename the field to sth like
> > > > > > > >> nextProducerEpoch?
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > Thanks,
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > Jun
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > On Mon, Dec 12, 2022 at 8:54 PM
> > > > Matthias
> > > > > > J.
> > > > > > > >> Sax <
> > > > > > > >> > > > >> > >> > > mjsax@apache.org>
> > > > > > > >> > > > >> > >> > > > > > > wrote:
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > Thanks for the background.
> > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > 20/30: SGTM. My proposal was
> > only
> > > > > > focusing
> > > > > > > >> to
> > > > > > > >> > > avoid
> > > > > > > >> > > > >> > >> dangling
> > > > > > > >> > > > >> > >> > > > > > > > > transactions if records are
> > added
> > > > > > without
> > > > > > > >> > > > registered
> > > > > > > >> > > > >> > >> > partition.
> > > > > > > >> > > > >> > >> > > > --
> > > > > > > >> > > > >> > >> > > > > > > Maybe
> > > > > > > >> > > > >> > >> > > > > > > > > you can add a few more details
> > to
> > > > the
> > > > > > KIP
> > > > > > > >> about
> > > > > > > >> > > > this
> > > > > > > >> > > > >> > >> scenario
> > > > > > > >> > > > >> > >> > > for
> > > > > > > >> > > > >> > >> > > > > > > better
> > > > > > > >> > > > >> > >> > > > > > > > > documentation purpose?
> > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > 40: I think you hit a fair
> point
> > > > about
> > > > > > race
> > > > > > > >> > > > >> conditions
> > > > > > > >> > > > >> > or
> > > > > > > >> > > > >> > >> > > client
> > > > > > > >> > > > >> > >> > > > > bugs
> > > > > > > >> > > > >> > >> > > > > > > > > (incorrectly not bumping the
> > > > epoch). The
> > > > > > > >> > > > >> > >> complexity/confusion
> > > > > > > >> > > > >> > >> > > for
> > > > > > > >> > > > >> > >> > > > > > using
> > > > > > > >> > > > >> > >> > > > > > > > > the bumped epoch I see, is
> > mainly
> > > > for
> > > > > > > >> internal
> > > > > > > >> > > > >> > debugging,
> > > > > > > >> > > > >> > >> ie,
> > > > > > > >> > > > >> > >> > > > > > > inspecting
> > > > > > > >> > > > >> > >> > > > > > > > > log segment dumps -- it seems
> > > > harder to
> > > > > > > >> reason
> > > > > > > >> > > > about
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> > system
> > > > > > > >> > > > >> > >> > > > for
> > > > > > > >> > > > >> > >> > > > > > us
> > > > > > > >> > > > >> > >> > > > > > > > > humans. But if we get better
> > > > > > guarantees, it
> > > > > > > >> > would
> > > > > > > >> > > > be
> > > > > > > >> > > > >> > >> worth to
> > > > > > > >> > > > >> > >> > > use
> > > > > > > >> > > > >> > >> > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > bumped epoch.
> > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > 60: as I mentioned already, I
> > > don't
> > > > > > know the
> > > > > > > >> > > broker
> > > > > > > >> > > > >> > >> internals
> > > > > > > >> > > > >> > >> > > to
> > > > > > > >> > > > >> > >> > > > > > > provide
> > > > > > > >> > > > >> > >> > > > > > > > > more input. So if nobody else
> > > chimes
> > > > > > in, we
> > > > > > > >> > > should
> > > > > > > >> > > > >> just
> > > > > > > >> > > > >> > >> move
> > > > > > > >> > > > >> > >> > > > > forward
> > > > > > > >> > > > >> > >> > > > > > > > > with your proposal.
> > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > -Matthias
> > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > On 12/6/22 4:22 PM, Justine
> > Olshan
> > > > > > wrote:
> > > > > > > >> > > > >> > >> > > > > > > > > > Hi all,
> > > > > > > >> > > > >> > >> > > > > > > > > > After Artem's questions
> about
> > > > error
> > > > > > > >> behavior,
> > > > > > > >> > > > I've
> > > > > > > >> > > > >> > >> > > re-evaluated
> > > > > > > >> > > > >> > >> > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > > unknown producer ID
> exception
> > > and
> > > > had
> > > > > > some
> > > > > > > >> > > > >> discussions
> > > > > > > >> > > > >> > >> > > offline.
> > > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > > I think generally it makes
> > sense
> > > > to
> > > > > > > >> simplify
> > > > > > > >> > > > error
> > > > > > > >> > > > >> > >> handling
> > > > > > > >> > > > >> > >> > > in
> > > > > > > >> > > > >> > >> > > > > > cases
> > > > > > > >> > > > >> > >> > > > > > > > like
> > > > > > > >> > > > >> > >> > > > > > > > > > this and the
> > UNKNOWN_PRODUCER_ID
> > > > error
> > > > > > > >> has a
> > > > > > > >> > > > pretty
> > > > > > > >> > > > >> > long
> > > > > > > >> > > > >> > >> > and
> > > > > > > >> > > > >> > >> > > > > > > > complicated
> > > > > > > >> > > > >> > >> > > > > > > > > > history. Because of this, I
> > > > propose
> > > > > > > >> adding a
> > > > > > > >> > > new
> > > > > > > >> > > > >> error
> > > > > > > >> > > > >> > >> code
> > > > > > > >> > > > >> > >> > > > > > > > > ABORTABLE_ERROR
> > > > > > > >> > > > >> > >> > > > > > > > > > that when encountered by new
> > > > clients
> > > > > > > >> (gated
> > > > > > > >> > by
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > >> produce
> > > > > > > >> > > > >> > >> > > > > request
> > > > > > > >> > > > >> > >> > > > > > > > > version)
> > > > > > > >> > > > >> > >> > > > > > > > > > will simply abort the
> > > transaction.
> > > > > > This
> > > > > > > >> > allows
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > >> server
> > > > > > > >> > > > >> > >> > to
> > > > > > > >> > > > >> > >> > > > have
> > > > > > > >> > > > >> > >> > > > > > > some
> > > > > > > >> > > > >> > >> > > > > > > > > say
> > > > > > > >> > > > >> > >> > > > > > > > > > in whether the client aborts
> > and
> > > > makes
> > > > > > > >> > handling
> > > > > > > >> > > > >> much
> > > > > > > >> > > > >> > >> > simpler.
> > > > > > > >> > > > >> > >> > > > In
> > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > > future, we can also use this
> > > > error in
> > > > > > > >> other
> > > > > > > >> > > > >> situations
> > > > > > > >> > > > >> > >> > where
> > > > > > > >> > > > >> > >> > > we
> > > > > > > >> > > > >> > >> > > > > > want
> > > > > > > >> > > > >> > >> > > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > > abort the transactions. We
> can
> > > > even
> > > > > > use on
> > > > > > > >> > > other
> > > > > > > >> > > > >> apis.
> > > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > > I've added this to the KIP.
> > Let
> > > me
> > > > > > know if
> > > > > > > >> > > there
> > > > > > > >> > > > >> are
> > > > > > > >> > > > >> > any
> > > > > > > >> > > > >> > >> > > > > questions
> > > > > > > >> > > > >> > >> > > > > > or
> > > > > > > >> > > > >> > >> > > > > > > > > > issues.
> > > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > > Justine
> > > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > > On Fri, Dec 2, 2022 at 10:22
> > AM
> > > > > > Justine
> > > > > > > >> > Olshan
> > > > > > > >> > > <
> > > > > > > >> > > > >> > >> > > > > > jolshan@confluent.io
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > wrote:
> > > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > >> Hey Matthias,
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >> 20/30 — Maybe I also didn't
> > > > express
> > > > > > > >> myself
> > > > > > > >> > > > >> clearly.
> > > > > > > >> > > > >> > For
> > > > > > > >> > > > >> > >> > > older
> > > > > > > >> > > > >> > >> > > > > > > clients
> > > > > > > >> > > > >> > >> > > > > > > > we
> > > > > > > >> > > > >> > >> > > > > > > > > >> don't have a way to
> > distinguish
> > > > > > between a
> > > > > > > >> > > > previous
> > > > > > > >> > > > >> > and
> > > > > > > >> > > > >> > >> the
> > > > > > > >> > > > >> > >> > > > > current
> > > > > > > >> > > > >> > >> > > > > > > > > >> transaction since we don't
> > have
> > > > the
> > > > > > epoch
> > > > > > > >> > > bump.
> > > > > > > >> > > > >> This
> > > > > > > >> > > > >> > >> means
> > > > > > > >> > > > >> > >> > > > that
> > > > > > > >> > > > >> > >> > > > > a
> > > > > > > >> > > > >> > >> > > > > > > late
> > > > > > > >> > > > >> > >> > > > > > > > > >> message from the previous
> > > > transaction
> > > > > > > >> may be
> > > > > > > >> > > > >> added to
> > > > > > > >> > > > >> > >> the
> > > > > > > >> > > > >> > >> > > new
> > > > > > > >> > > > >> > >> > > > > one.
> > > > > > > >> > > > >> > >> > > > > > > > With
> > > > > > > >> > > > >> > >> > > > > > > > > >> older clients — we can't
> > > > guarantee
> > > > > > this
> > > > > > > >> > won't
> > > > > > > >> > > > >> happen
> > > > > > > >> > > > >> > >> if we
> > > > > > > >> > > > >> > >> > > > > already
> > > > > > > >> > > > >> > >> > > > > > > > sent
> > > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call
> (why
> > we
> > > > make
> > > > > > > >> changes
> > > > > > > >> > > for
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> newer
> > > > > > > >> > > > >> > >> > > > > client)
> > > > > > > >> > > > >> > >> > > > > > > but
> > > > > > > >> > > > >> > >> > > > > > > > > we
> > > > > > > >> > > > >> > >> > > > > > > > > >> can at least gate some by
> > > > ensuring
> > > > > > that
> > > > > > > >> the
> > > > > > > >> > > > >> partition
> > > > > > > >> > > > >> > >> has
> > > > > > > >> > > > >> > >> > > been
> > > > > > > >> > > > >> > >> > > > > > added
> > > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >> transaction. The rationale
> > here
> > > > is
> > > > > > that
> > > > > > > >> > there
> > > > > > > >> > > > are
> > > > > > > >> > > > >> > >> likely
> > > > > > > >> > > > >> > >> > > LESS
> > > > > > > >> > > > >> > >> > > > > late
> > > > > > > >> > > > >> > >> > > > > > > > > arrivals
> > > > > > > >> > > > >> > >> > > > > > > > > >> as time goes on, so
> hopefully
> > > > most
> > > > > > late
> > > > > > > >> > > arrivals
> > > > > > > >> > > > >> will
> > > > > > > >> > > > >> > >> come
> > > > > > > >> > > > >> > >> > > in
> > > > > > > >> > > > >> > >> > > > > > BEFORE
> > > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call.
> > Those
> > > > that
> > > > > > > >> arrive
> > > > > > > >> > > > before
> > > > > > > >> > > > >> > will
> > > > > > > >> > > > >> > >> be
> > > > > > > >> > > > >> > >> > > > > properly
> > > > > > > >> > > > >> > >> > > > > > > > gated
> > > > > > > >> > > > >> > >> > > > > > > > > >> with the
> describeTransactions
> > > > > > approach.
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >> If we take the approach you
> > > > > > suggested,
> > > > > > > >> ANY
> > > > > > > >> > > late
> > > > > > > >> > > > >> > arrival
> > > > > > > >> > > > >> > >> > > from a
> > > > > > > >> > > > >> > >> > > > > > > > previous
> > > > > > > >> > > > >> > >> > > > > > > > > >> transaction will be added.
> > And
> > > we
> > > > > > don't
> > > > > > > >> want
> > > > > > > >> > > > >> that. I
> > > > > > > >> > > > >> > >> also
> > > > > > > >> > > > >> > >> > > > don't
> > > > > > > >> > > > >> > >> > > > > > see
> > > > > > > >> > > > >> > >> > > > > > > > any
> > > > > > > >> > > > >> > >> > > > > > > > > >> benefit in sending
> > > > addPartitionsToTxn
> > > > > > > >> over
> > > > > > > >> > the
> > > > > > > >> > > > >> > >> > describeTxns
> > > > > > > >> > > > >> > >> > > > > call.
> > > > > > > >> > > > >> > >> > > > > > > They
> > > > > > > >> > > > >> > >> > > > > > > > > will
> > > > > > > >> > > > >> > >> > > > > > > > > >> both be one extra RPC to
> the
> > > Txn
> > > > > > > >> > coordinator.
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >> To be clear — newer clients
> > > will
> > > > use
> > > > > > > >> > > > >> > addPartitionsToTxn
> > > > > > > >> > > > >> > >> > > > instead
> > > > > > > >> > > > >> > >> > > > > of
> > > > > > > >> > > > >> > >> > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >> DescribeTxns.
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >> 40)
> > > > > > > >> > > > >> > >> > > > > > > > > >> My concern is that if we
> have
> > > > some
> > > > > > delay
> > > > > > > >> in
> > > > > > > >> > > the
> > > > > > > >> > > > >> > client
> > > > > > > >> > > > >> > >> to
> > > > > > > >> > > > >> > >> > > bump
> > > > > > > >> > > > >> > >> > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > epoch,
> > > > > > > >> > > > >> > >> > > > > > > > > >> it could continue to send
> > epoch
> > > > 73
> > > > > > and
> > > > > > > >> those
> > > > > > > >> > > > >> records
> > > > > > > >> > > > >> > >> would
> > > > > > > >> > > > >> > >> > > not
> > > > > > > >> > > > >> > >> > > > > be
> > > > > > > >> > > > >> > >> > > > > > > > > fenced.
> > > > > > > >> > > > >> > >> > > > > > > > > >> Perhaps this is not an
> issue
> > if
> > > > we
> > > > > > don't
> > > > > > > >> > allow
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > next
> > > > > > > >> > > > >> > >> > > > produce
> > > > > > > >> > > > >> > >> > > > > to
> > > > > > > >> > > > >> > >> > > > > > > go
> > > > > > > >> > > > >> > >> > > > > > > > > >> through before the EndTxn
> > > request
> > > > > > > >> returns.
> > > > > > > >> > I'm
> > > > > > > >> > > > >> also
> > > > > > > >> > > > >> > >> > thinking
> > > > > > > >> > > > >> > >> > > > > about
> > > > > > > >> > > > >> > >> > > > > > > > > cases of
> > > > > > > >> > > > >> > >> > > > > > > > > >> failure. I will need to
> think
> > > on
> > > > > > this a
> > > > > > > >> bit.
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >> I wasn't sure if it was
> that
> > > > > > confusing.
> > > > > > > >> But
> > > > > > > >> > if
> > > > > > > >> > > > we
> > > > > > > >> > > > >> > >> think it
> > > > > > > >> > > > >> > >> > > is,
> > > > > > > >> > > > >> > >> > > > > we
> > > > > > > >> > > > >> > >> > > > > > > can
> > > > > > > >> > > > >> > >> > > > > > > > > >> investigate other ways.
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >> 60)
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >> I'm not sure these are the
> > same
> > > > > > > >> purgatories
> > > > > > > >> > > > since
> > > > > > > >> > > > >> one
> > > > > > > >> > > > >> > >> is a
> > > > > > > >> > > > >> > >> > > > > produce
> > > > > > > >> > > > >> > >> > > > > > > > > >> purgatory (I was planning
> on
> > > > using a
> > > > > > > >> > callback
> > > > > > > >> > > > >> rather
> > > > > > > >> > > > >> > >> than
> > > > > > > >> > > > >> > >> > > > > > purgatory)
> > > > > > > >> > > > >> > >> > > > > > > > and
> > > > > > > >> > > > >> > >> > > > > > > > > >> the other is simply a
> request
> > > to
> > > > > > append
> > > > > > > >> to
> > > > > > > >> > the
> > > > > > > >> > > > >> log.
> > > > > > > >> > > > >> > Not
> > > > > > > >> > > > >> > >> > sure
> > > > > > > >> > > > >> > >> > > > we
> > > > > > > >> > > > >> > >> > > > > > have
> > > > > > > >> > > > >> > >> > > > > > > > any
> > > > > > > >> > > > >> > >> > > > > > > > > >> structure here for
> ordering,
> > > but
> > > > my
> > > > > > > >> > > > understanding
> > > > > > > >> > > > >> is
> > > > > > > >> > > > >> > >> that
> > > > > > > >> > > > >> > >> > > the
> > > > > > > >> > > > >> > >> > > > > > broker
> > > > > > > >> > > > >> > >> > > > > > > > > could
> > > > > > > >> > > > >> > >> > > > > > > > > >> handle the write request
> > before
> > > > it
> > > > > > hears
> > > > > > > >> > back
> > > > > > > >> > > > from
> > > > > > > >> > > > >> > the
> > > > > > > >> > > > >> > >> Txn
> > > > > > > >> > > > >> > >> > > > > > > > Coordinator.
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >> Let me know if I
> > misunderstood
> > > > > > something
> > > > > > > >> or
> > > > > > > >> > > > >> something
> > > > > > > >> > > > >> > >> was
> > > > > > > >> > > > >> > >> > > > > unclear.
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >> Justine
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >> On Thu, Dec 1, 2022 at
> 12:15
> > PM
> > > > > > Matthias
> > > > > > > >> J.
> > > > > > > >> > > Sax
> > > > > > > >> > > > <
> > > > > > > >> > > > >> > >> > > > > mjsax@apache.org
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > wrote:
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >>> Thanks for the details
> > > Justine!
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> 20)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> The client side change
> for
> > 2
> > > is
> > > > > > > >> removing
> > > > > > > >> > the
> > > > > > > >> > > > >> > >> > addPartitions
> > > > > > > >> > > > >> > >> > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > >>> transaction
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> call. We don't need to
> make
> > > > this
> > > > > > from
> > > > > > > >> the
> > > > > > > >> > > > >> producer
> > > > > > > >> > > > >> > to
> > > > > > > >> > > > >> > >> > the
> > > > > > > >> > > > >> > >> > > > txn
> > > > > > > >> > > > >> > >> > > > > > > > > >>> coordinator,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> only server side.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>> I think I did not express
> > > myself
> > > > > > > >> clearly. I
> > > > > > > >> > > > >> > understand
> > > > > > > >> > > > >> > >> > that
> > > > > > > >> > > > >> > >> > > > we
> > > > > > > >> > > > >> > >> > > > > > can
> > > > > > > >> > > > >> > >> > > > > > > > (and
> > > > > > > >> > > > >> > >> > > > > > > > > >>> should) change the
> producer
> > to
> > > > not
> > > > > > send
> > > > > > > >> the
> > > > > > > >> > > > >> > >> > `addPartitions`
> > > > > > > >> > > > >> > >> > > > > > request
> > > > > > > >> > > > >> > >> > > > > > > > any
> > > > > > > >> > > > >> > >> > > > > > > > > >>> longer. But I don't thinks
> > > it's
> > > > > > > >> requirement
> > > > > > > >> > > to
> > > > > > > >> > > > >> > change
> > > > > > > >> > > > >> > >> the
> > > > > > > >> > > > >> > >> > > > > broker?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>> What I am trying to say
> is:
> > > as a
> > > > > > > >> safe-guard
> > > > > > > >> > > and
> > > > > > > >> > > > >> > >> > improvement
> > > > > > > >> > > > >> > >> > > > for
> > > > > > > >> > > > >> > >> > > > > > > older
> > > > > > > >> > > > >> > >> > > > > > > > > >>> producers, the partition
> > > leader
> > > > can
> > > > > > just
> > > > > > > >> > send
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > >> > > > > `addPartitions`
> > > > > > > >> > > > >> > >> > > > > > > > > >>> request to the
> > TX-coordinator
> > > > in any
> > > > > > > >> case
> > > > > > > >> > --
> > > > > > > >> > > if
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> old
> > > > > > > >> > > > >> > >> > > > > producer
> > > > > > > >> > > > >> > >> > > > > > > > > >>> correctly did send the
> > > > > > `addPartition`
> > > > > > > >> > request
> > > > > > > >> > > > to
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> > > > > > TX-coordinator
> > > > > > > >> > > > >> > >> > > > > > > > > >>> already, the
> TX-coordinator
> > > can
> > > > just
> > > > > > > >> > "ignore"
> > > > > > > >> > > > is
> > > > > > > >> > > > >> as
> > > > > > > >> > > > >> > >> > > > idempotent.
> > > > > > > >> > > > >> > >> > > > > > > > > However,
> > > > > > > >> > > > >> > >> > > > > > > > > >>> if the old producer has a
> > bug
> > > > and
> > > > > > did
> > > > > > > >> > forget
> > > > > > > >> > > to
> > > > > > > >> > > > >> sent
> > > > > > > >> > > > >> > >> the
> > > > > > > >> > > > >> > >> > > > > > > > `addPartition`
> > > > > > > >> > > > >> > >> > > > > > > > > >>> request, we would now
> ensure
> > > > that
> > > > > > the
> > > > > > > >> > > partition
> > > > > > > >> > > > >> is
> > > > > > > >> > > > >> > >> indeed
> > > > > > > >> > > > >> > >> > > > added
> > > > > > > >> > > > >> > >> > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> TX and thus fix a
> potential
> > > > > > producer bug
> > > > > > > >> > > (even
> > > > > > > >> > > > >> if we
> > > > > > > >> > > > >> > >> > don't
> > > > > > > >> > > > >> > >> > > > get
> > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> fencing via the bump
> epoch).
> > > --
> > > > It
> > > > > > > >> seems to
> > > > > > > >> > > be
> > > > > > > >> > > > a
> > > > > > > >> > > > >> > good
> > > > > > > >> > > > >> > >> > > > > > improvement?
> > > > > > > >> > > > >> > >> > > > > > > Or
> > > > > > > >> > > > >> > >> > > > > > > > > is
> > > > > > > >> > > > >> > >> > > > > > > > > >>> there a reason to not do
> > this?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> 30)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> Transaction is ongoing =
> > > > partition
> > > > > > was
> > > > > > > >> > added
> > > > > > > >> > > > to
> > > > > > > >> > > > >> > >> > > transaction
> > > > > > > >> > > > >> > >> > > > > via
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> addPartitionsToTxn. We
> > check
> > > > this
> > > > > > with
> > > > > > > >> the
> > > > > > > >> > > > >> > >> > > > > DescribeTransactions
> > > > > > > >> > > > >> > >> > > > > > > > call.
> > > > > > > >> > > > >> > >> > > > > > > > > >>> Let
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> me know if this wasn't
> > > > sufficiently
> > > > > > > >> > > explained
> > > > > > > >> > > > >> here:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>> If we do what I propose in
> > > > (20), we
> > > > > > > >> don't
> > > > > > > >> > > > really
> > > > > > > >> > > > >> > need
> > > > > > > >> > > > >> > >> to
> > > > > > > >> > > > >> > >> > > make
> > > > > > > >> > > > >> > >> > > > > > this
> > > > > > > >> > > > >> > >> > > > > > > > > >>> `DescribeTransaction`
> call,
> > as
> > > > the
> > > > > > > >> > partition
> > > > > > > >> > > > >> leader
> > > > > > > >> > > > >> > >> adds
> > > > > > > >> > > > >> > >> > > the
> > > > > > > >> > > > >> > >> > > > > > > > partition
> > > > > > > >> > > > >> > >> > > > > > > > > >>> for older clients and we
> get
> > > > this
> > > > > > check
> > > > > > > >> for
> > > > > > > >> > > > free.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> 40)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> The idea here is that if
> > any
> > > > > > messages
> > > > > > > >> > > somehow
> > > > > > > >> > > > >> come
> > > > > > > >> > > > >> > in
> > > > > > > >> > > > >> > >> > > before
> > > > > > > >> > > > >> > >> > > > > we
> > > > > > > >> > > > >> > >> > > > > > > get
> > > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> new
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> epoch to the producer,
> they
> > > > will be
> > > > > > > >> > fenced.
> > > > > > > >> > > > >> > However,
> > > > > > > >> > > > >> > >> if
> > > > > > > >> > > > >> > >> > we
> > > > > > > >> > > > >> > >> > > > > don't
> > > > > > > >> > > > >> > >> > > > > > > > think
> > > > > > > >> > > > >> > >> > > > > > > > > >>> this
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> is necessary, it can be
> > > > discussed
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>> I agree that we should
> have
> > > > epoch
> > > > > > > >> fencing.
> > > > > > > >> > My
> > > > > > > >> > > > >> > >> question is
> > > > > > > >> > > > >> > >> > > > > > > different:
> > > > > > > >> > > > >> > >> > > > > > > > > >>> Assume we are at epoch 73,
> > and
> > > > we
> > > > > > have
> > > > > > > >> an
> > > > > > > >> > > > ongoing
> > > > > > > >> > > > >> > >> > > > transaction,
> > > > > > > >> > > > >> > >> > > > > > that
> > > > > > > >> > > > >> > >> > > > > > > > is
> > > > > > > >> > > > >> > >> > > > > > > > > >>> committed. It seems
> natural
> > to
> > > > > > write the
> > > > > > > >> > > > "prepare
> > > > > > > >> > > > >> > >> commit"
> > > > > > > >> > > > >> > >> > > > > marker
> > > > > > > >> > > > >> > >> > > > > > > and
> > > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest`
> both
> > > with
> > > > > > epoch
> > > > > > > >> 73,
> > > > > > > >> > > too,
> > > > > > > >> > > > >> as
> > > > > > > >> > > > >> > it
> > > > > > > >> > > > >> > >> > > belongs
> > > > > > > >> > > > >> > >> > > > > to
> > > > > > > >> > > > >> > >> > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> current transaction. Of
> > > course,
> > > > we
> > > > > > now
> > > > > > > >> also
> > > > > > > >> > > > bump
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> > epoch
> > > > > > > >> > > > >> > >> > > > and
> > > > > > > >> > > > >> > >> > > > > > > expect
> > > > > > > >> > > > >> > >> > > > > > > > > >>> the next requests to have
> > > epoch
> > > > 74,
> > > > > > and
> > > > > > > >> > would
> > > > > > > >> > > > >> reject
> > > > > > > >> > > > >> > >> an
> > > > > > > >> > > > >> > >> > > > request
> > > > > > > >> > > > >> > >> > > > > > > with
> > > > > > > >> > > > >> > >> > > > > > > > > >>> epoch 73, as the
> > corresponding
> > > > TX
> > > > > > for
> > > > > > > >> epoch
> > > > > > > >> > > 73
> > > > > > > >> > > > >> was
> > > > > > > >> > > > >> > >> > already
> > > > > > > >> > > > >> > >> > > > > > > committed.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>> It seems you propose to
> > write
> > > > the
> > > > > > > >> "prepare
> > > > > > > >> > > > commit
> > > > > > > >> > > > >> > >> marker"
> > > > > > > >> > > > >> > >> > > and
> > > > > > > >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest`
> with
> > > > epoch 74
> > > > > > > >> > though,
> > > > > > > >> > > > what
> > > > > > > >> > > > >> > >> would
> > > > > > > >> > > > >> > >> > > work,
> > > > > > > >> > > > >> > >> > > > > but
> > > > > > > >> > > > >> > >> > > > > > > it
> > > > > > > >> > > > >> > >> > > > > > > > > >>> seems confusing. Is there
> a
> > > > reason
> > > > > > why
> > > > > > > >> we
> > > > > > > >> > > would
> > > > > > > >> > > > >> use
> > > > > > > >> > > > >> > >> the
> > > > > > > >> > > > >> > >> > > > bumped
> > > > > > > >> > > > >> > >> > > > > > > epoch
> > > > > > > >> > > > >> > >> > > > > > > > 74
> > > > > > > >> > > > >> > >> > > > > > > > > >>> instead of the current
> epoch
> > > 73?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> 60)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> When we are checking if
> the
> > > > > > > >> transaction is
> > > > > > > >> > > > >> ongoing,
> > > > > > > >> > > > >> > >> we
> > > > > > > >> > > > >> > >> > > need
> > > > > > > >> > > > >> > >> > > > to
> > > > > > > >> > > > >> > >> > > > > > > make
> > > > > > > >> > > > >> > >> > > > > > > > a
> > > > > > > >> > > > >> > >> > > > > > > > > >>> round
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> trip from the leader
> > > partition
> > > > to
> > > > > > the
> > > > > > > >> > > > >> transaction
> > > > > > > >> > > > >> > >> > > > coordinator.
> > > > > > > >> > > > >> > >> > > > > > In
> > > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> time
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> we are waiting for this
> > > > message to
> > > > > > come
> > > > > > > >> > > back,
> > > > > > > >> > > > in
> > > > > > > >> > > > >> > >> theory
> > > > > > > >> > > > >> > >> > we
> > > > > > > >> > > > >> > >> > > > > could
> > > > > > > >> > > > >> > >> > > > > > > > have
> > > > > > > >> > > > >> > >> > > > > > > > > >>> sent
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> a commit/abort call that
> > > would
> > > > > > make the
> > > > > > > >> > > > original
> > > > > > > >> > > > >> > >> result
> > > > > > > >> > > > >> > >> > of
> > > > > > > >> > > > >> > >> > > > the
> > > > > > > >> > > > >> > >> > > > > > > check
> > > > > > > >> > > > >> > >> > > > > > > > > >>> out of
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> date. That is why we can
> > > check
> > > > the
> > > > > > > >> leader
> > > > > > > >> > > > state
> > > > > > > >> > > > >> > >> before
> > > > > > > >> > > > >> > >> > we
> > > > > > > >> > > > >> > >> > > > > write
> > > > > > > >> > > > >> > >> > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> log.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>> Thanks. Got it.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>> However, is this really an
> > > > issue?
> > > > > > We put
> > > > > > > >> > the
> > > > > > > >> > > > >> produce
> > > > > > > >> > > > >> > >> > > request
> > > > > > > >> > > > >> > >> > > > in
> > > > > > > >> > > > >> > >> > > > > > > > > >>> purgatory, so how could we
> > > > process
> > > > > > the
> > > > > > > >> > > > >> > >> > > > `WriteTxnMarkerRequest`
> > > > > > > >> > > > >> > >> > > > > > > first?
> > > > > > > >> > > > >> > >> > > > > > > > > >>> Don't we need to put the
> > > > > > > >> > > > `WriteTxnMarkerRequest`
> > > > > > > >> > > > >> > into
> > > > > > > >> > > > >> > >> > > > > purgatory,
> > > > > > > >> > > > >> > >> > > > > > > too,
> > > > > > > >> > > > >> > >> > > > > > > > > >>> for this case, and process
> > > both
> > > > > > request
> > > > > > > >> > > > in-order?
> > > > > > > >> > > > >> > >> (Again,
> > > > > > > >> > > > >> > >> > > my
> > > > > > > >> > > > >> > >> > > > > > broker
> > > > > > > >> > > > >> > >> > > > > > > > > >>> knowledge is limited and
> > maybe
> > > > we
> > > > > > don't
> > > > > > > >> > > > maintain
> > > > > > > >> > > > >> > >> request
> > > > > > > >> > > > >> > >> > > > order
> > > > > > > >> > > > >> > >> > > > > > for
> > > > > > > >> > > > >> > >> > > > > > > > this
> > > > > > > >> > > > >> > >> > > > > > > > > >>> case, what seems to be an
> > > issue
> > > > > > IMHO,
> > > > > > > >> and I
> > > > > > > >> > > am
> > > > > > > >> > > > >> > >> wondering
> > > > > > > >> > > > >> > >> > if
> > > > > > > >> > > > >> > >> > > > > > > changing
> > > > > > > >> > > > >> > >> > > > > > > > > >>> request handling to
> preserve
> > > > order
> > > > > > for
> > > > > > > >> this
> > > > > > > >> > > > case
> > > > > > > >> > > > >> > >> might be
> > > > > > > >> > > > >> > >> > > the
> > > > > > > >> > > > >> > >> > > > > > > cleaner
> > > > > > > >> > > > >> > >> > > > > > > > > >>> solution?)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>> -Matthias
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>> On 11/30/22 3:28 PM, Artem
> > > > Livshits
> > > > > > > >> wrote:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> Hi Justine,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> I think the interesting
> > part
> > > is
> > > > > > not in
> > > > > > > >> > this
> > > > > > > >> > > > >> logic
> > > > > > > >> > > > >> > >> > (because
> > > > > > > >> > > > >> > >> > > > it
> > > > > > > >> > > > >> > >> > > > > > > tries
> > > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> figure out when
> > > > > > UNKNOWN_PRODUCER_ID is
> > > > > > > >> > > > retriable
> > > > > > > >> > > > >> > and
> > > > > > > >> > > > >> > >> if
> > > > > > > >> > > > >> > >> > > it's
> > > > > > > >> > > > >> > >> > > > > > > > > retryable,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> it's definitely not
> fatal),
> > > but
> > > > > > what
> > > > > > > >> > happens
> > > > > > > >> > > > >> when
> > > > > > > >> > > > >> > >> this
> > > > > > > >> > > > >> > >> > > logic
> > > > > > > >> > > > >> > >> > > > > > > doesn't
> > > > > > > >> > > > >> > >> > > > > > > > > >>> return
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> 'true' and falls through.
> > In
> > > > the
> > > > > > old
> > > > > > > >> > > clients
> > > > > > > >> > > > it
> > > > > > > >> > > > >> > >> seems
> > > > > > > >> > > > >> > >> > to
> > > > > > > >> > > > >> > >> > > be
> > > > > > > >> > > > >> > >> > > > > > > fatal,
> > > > > > > >> > > > >> > >> > > > > > > > if
> > > > > > > >> > > > >> > >> > > > > > > > > >>> we
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> keep the behavior in the
> > new
> > > > > > clients,
> > > > > > > >> I'd
> > > > > > > >> > > > >> expect it
> > > > > > > >> > > > >> > >> > would
> > > > > > > >> > > > >> > >> > > be
> > > > > > > >> > > > >> > >> > > > > > fatal
> > > > > > > >> > > > >> > >> > > > > > > > as
> > > > > > > >> > > > >> > >> > > > > > > > > >>> well.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> -Artem
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> On Tue, Nov 29, 2022 at
> > 11:57
> > > > AM
> > > > > > > >> Justine
> > > > > > > >> > > > Olshan
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > <jolshan@confluent.io.invalid
> > > >
> > > > > > wrote:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> Hi Artem and Jeff,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> Thanks for taking a look
> > and
> > > > > > sorry for
> > > > > > > >> > the
> > > > > > > >> > > > slow
> > > > > > > >> > > > >> > >> > response.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> You both mentioned the
> > > change
> > > > to
> > > > > > > >> handle
> > > > > > > >> > > > >> > >> > > UNKNOWN_PRODUCER_ID
> > > > > > > >> > > > >> > >> > > > > > > errors.
> > > > > > > >> > > > >> > >> > > > > > > > > To
> > > > > > > >> > > > >> > >> > > > > > > > > >>> be
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> clear — this error code
> > will
> > > > only
> > > > > > be
> > > > > > > >> sent
> > > > > > > >> > > > again
> > > > > > > >> > > > >> > when
> > > > > > > >> > > > >> > >> > the
> > > > > > > >> > > > >> > >> > > > > > client's
> > > > > > > >> > > > >> > >> > > > > > > > > >>> request
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> version is high enough
> to
> > > > ensure
> > > > > > we
> > > > > > > >> > handle
> > > > > > > >> > > it
> > > > > > > >> > > > >> > >> > correctly.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> The current (Java)
> client
> > > > handles
> > > > > > > >> this by
> > > > > > > >> > > the
> > > > > > > >> > > > >> > >> following
> > > > > > > >> > > > >> > >> > > > > > (somewhat
> > > > > > > >> > > > >> > >> > > > > > > > > long)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> code snippet:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> // An
> UNKNOWN_PRODUCER_ID
> > > > means
> > > > > > that
> > > > > > > >> we
> > > > > > > >> > > have
> > > > > > > >> > > > >> lost
> > > > > > > >> > > > >> > >> the
> > > > > > > >> > > > >> > >> > > > > producer
> > > > > > > >> > > > >> > >> > > > > > > > state
> > > > > > > >> > > > >> > >> > > > > > > > > >>> on the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> broker. Depending on the
> > log
> > > > start
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> // offset, we may want
> to
> > > > retry
> > > > > > > >> these, as
> > > > > > > >> > > > >> > described
> > > > > > > >> > > > >> > >> for
> > > > > > > >> > > > >> > >> > > > each
> > > > > > > >> > > > >> > >> > > > > > case
> > > > > > > >> > > > >> > >> > > > > > > > > >>> below. If
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> none of those apply,
> then
> > > for
> > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> // idempotent producer,
> we
> > > > will
> > > > > > > >> locally
> > > > > > > >> > > bump
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> epoch
> > > > > > > >> > > > >> > >> > > and
> > > > > > > >> > > > >> > >> > > > > > reset
> > > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> sequence numbers of
> > > in-flight
> > > > > > batches
> > > > > > > >> > from
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> // sequence 0, then
> retry
> > > the
> > > > > > failed
> > > > > > > >> > batch,
> > > > > > > >> > > > >> which
> > > > > > > >> > > > >> > >> > should
> > > > > > > >> > > > >> > >> > > > now
> > > > > > > >> > > > >> > >> > > > > > > > succeed.
> > > > > > > >> > > > >> > >> > > > > > > > > >>> For
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> the transactional
> > producer,
> > > > allow
> > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> // batch to fail. When
> > > > processing
> > > > > > the
> > > > > > > >> > > failed
> > > > > > > >> > > > >> > batch,
> > > > > > > >> > > > >> > >> we
> > > > > > > >> > > > >> > >> > > will
> > > > > > > >> > > > >> > >> > > > > > > > > transition
> > > > > > > >> > > > >> > >> > > > > > > > > >>> to
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> an abortable error and
> > set a
> > > > flag
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> // indicating that we
> need
> > > to
> > > > > > bump the
> > > > > > > >> > > epoch
> > > > > > > >> > > > >> (if
> > > > > > > >> > > > >> > >> > > supported
> > > > > > > >> > > > >> > >> > > > by
> > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> broker).
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> if (error ==
> > > > > > > >> > Errors.*UNKNOWN_PRODUCER_ID*)
> > > > > > > >> > > {
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>       if
> > > > (response.logStartOffset
> > > > > > ==
> > > > > > > >> -1)
> > > > > > > >> > {
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // We don't
> know
> > > > the log
> > > > > > > >> start
> > > > > > > >> > > > offset
> > > > > > > >> > > > >> > with
> > > > > > > >> > > > >> > >> > this
> > > > > > > >> > > > >> > >> > > > > > > response.
> > > > > > > >> > > > >> > >> > > > > > > > > We
> > > > > > > >> > > > >> > >> > > > > > > > > >>> should
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> just retry the request
> > until
> > > > we
> > > > > > get
> > > > > > > >> it.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // The
> > > > > > UNKNOWN_PRODUCER_ID
> > > > > > > >> > error
> > > > > > > >> > > > code
> > > > > > > >> > > > >> > was
> > > > > > > >> > > > >> > >> > added
> > > > > > > >> > > > >> > >> > > > > along
> > > > > > > >> > > > >> > >> > > > > > > > with
> > > > > > > >> > > > >> > >> > > > > > > > > >>> the new
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> ProduceResponse which
> > > > includes the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           //
> > logStartOffset.
> > > > So
> > > > > > the
> > > > > > > >> '-1'
> > > > > > > >> > > > >> sentinel
> > > > > > > >> > > > >> > is
> > > > > > > >> > > > >> > >> > not
> > > > > > > >> > > > >> > >> > > > for
> > > > > > > >> > > > >> > >> > > > > > > > backward
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> compatibility. Instead,
> it
> > > is
> > > > > > possible
> > > > > > > >> > for
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // a broker to
> > not
> > > > know
> > > > > > the
> > > > > > > >> > > > >> > >> logStartOffset at
> > > > > > > >> > > > >> > >> > > > when
> > > > > > > >> > > > >> > >> > > > > it
> > > > > > > >> > > > >> > >> > > > > > > is
> > > > > > > >> > > > >> > >> > > > > > > > > >>> returning
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> the response because the
> > > > partition
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // may have
> > moved
> > > > away
> > > > > > from
> > > > > > > >> the
> > > > > > > >> > > > >> broker
> > > > > > > >> > > > >> > >> from
> > > > > > > >> > > > >> > >> > the
> > > > > > > >> > > > >> > >> > > > > time
> > > > > > > >> > > > >> > >> > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> error was
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> initially raised to the
> > time
> > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // response
> was
> > > > being
> > > > > > > >> > > constructed.
> > > > > > > >> > > > In
> > > > > > > >> > > > >> > >> these
> > > > > > > >> > > > >> > >> > > > cases,
> > > > > > > >> > > > >> > >> > > > > we
> > > > > > > >> > > > >> > >> > > > > > > > > should
> > > > > > > >> > > > >> > >> > > > > > > > > >>> just
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> retry the request: we
> are
> > > > > > guaranteed
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // to
> eventually
> > > > get a
> > > > > > > >> > > > logStartOffset
> > > > > > > >> > > > >> > once
> > > > > > > >> > > > >> > >> > > things
> > > > > > > >> > > > >> > >> > > > > > > settle
> > > > > > > >> > > > >> > >> > > > > > > > > down.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>       if
> > > > > > > >> (batch.sequenceHasBeenReset()) {
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // When the
> > first
> > > > > > inflight
> > > > > > > >> > batch
> > > > > > > >> > > > >> fails
> > > > > > > >> > > > >> > >> due to
> > > > > > > >> > > > >> > >> > > the
> > > > > > > >> > > > >> > >> > > > > > > > > truncation
> > > > > > > >> > > > >> > >> > > > > > > > > >>> case,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> then the sequences of
> all
> > > the
> > > > > > other
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // in flight
> > > batches
> > > > > > would
> > > > > > > >> have
> > > > > > > >> > > > been
> > > > > > > >> > > > >> > >> > restarted
> > > > > > > >> > > > >> > >> > > > from
> > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> beginning.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> However, when those
> > > responses
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // come back
> > from
> > > > the
> > > > > > > >> broker,
> > > > > > > >> > > they
> > > > > > > >> > > > >> would
> > > > > > > >> > > > >> > >> also
> > > > > > > >> > > > >> > >> > > > come
> > > > > > > >> > > > >> > >> > > > > > with
> > > > > > > >> > > > >> > >> > > > > > > > an
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> error.
> > > In
> > > > this
> > > > > > > >> case,
> > > > > > > >> > we
> > > > > > > >> > > > >> should
> > > > > > > >> > > > >> > >> not
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // reset the
> > > > sequence
> > > > > > > >> numbers
> > > > > > > >> > to
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > >> > beginning.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>       } else if
> > > > > > > >> > > > >> > >> > > > >
> > > > (lastAckedOffset(batch.topicPartition).orElse(
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > > > > > > >> > > > >> > >> > > > response.logStartOffset) {
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // The head of
> > the
> > > > log
> > > > > > has
> > > > > > > >> been
> > > > > > > >> > > > >> removed,
> > > > > > > >> > > > >> > >> > > probably
> > > > > > > >> > > > >> > >> > > > > due
> > > > > > > >> > > > >> > >> > > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> retention time elapsing.
> > In
> > > > this
> > > > > > case,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // we expect
> to
> > > > lose the
> > > > > > > >> > producer
> > > > > > > >> > > > >> state.
> > > > > > > >> > > > >> > >> For
> > > > > > > >> > > > >> > >> > > the
> > > > > > > >> > > > >> > >> > > > > > > > > transactional
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> producer, reset the
> > > sequences
> > > > of
> > > > > > all
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // inflight
> > > batches
> > > > to
> > > > > > be
> > > > > > > >> from
> > > > > > > >> > > the
> > > > > > > >> > > > >> > >> beginning
> > > > > > > >> > > > >> > >> > > and
> > > > > > > >> > > > >> > >> > > > > > retry
> > > > > > > >> > > > >> > >> > > > > > > > > them,
> > > > > > > >> > > > >> > >> > > > > > > > > >>> so
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> that the transaction
> does
> > > not
> > > > > > need to
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // be aborted.
> > For
> > > > the
> > > > > > > >> > idempotent
> > > > > > > >> > > > >> > >> producer,
> > > > > > > >> > > > >> > >> > > bump
> > > > > > > >> > > > >> > >> > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > epoch
> > > > > > > >> > > > >> > >> > > > > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > >>> avoid
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> reusing (sequence,
> epoch)
> > > > pairs
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           if
> > > > (isTransactional()) {
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > >
> > > > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> this.producerIdAndEpoch);
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           } else {
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> requestEpochBumpForPartition(batch.topicPartition);
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           }
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>       if
> > > (!isTransactional())
> > > > {
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // For the
> > > > idempotent
> > > > > > > >> producer,
> > > > > > > >> > > > >> always
> > > > > > > >> > > > >> > >> retry
> > > > > > > >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> errors. If the batch has
> > the
> > > > > > current
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // producer ID
> > and
> > > > > > epoch,
> > > > > > > >> > > request a
> > > > > > > >> > > > >> bump
> > > > > > > >> > > > >> > >> of
> > > > > > > >> > > > >> > >> > the
> > > > > > > >> > > > >> > >> > > > > > epoch.
> > > > > > > >> > > > >> > >> > > > > > > > > >>> Otherwise
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> just retry the produce.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > >
> > > > > > requestEpochBumpForPartition(batch.topicPartition);
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> }
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> I was considering
> keeping
> > > this
> > > > > > > >> behavior —
> > > > > > > >> > > but
> > > > > > > >> > > > >> am
> > > > > > > >> > > > >> > >> open
> > > > > > > >> > > > >> > >> > to
> > > > > > > >> > > > >> > >> > > > > > > > simplifying
> > > > > > > >> > > > >> > >> > > > > > > > > >>> it.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> We are leaving changes
> to
> > > > older
> > > > > > > >> clients
> > > > > > > >> > off
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > >> table
> > > > > > > >> > > > >> > >> > > here
> > > > > > > >> > > > >> > >> > > > > > since
> > > > > > > >> > > > >> > >> > > > > > > it
> > > > > > > >> > > > >> > >> > > > > > > > > >>> caused
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> many issues for clients
> in
> > > the
> > > > > > past.
> > > > > > > >> > > > Previously
> > > > > > > >> > > > >> > this
> > > > > > > >> > > > >> > >> > was
> > > > > > > >> > > > >> > >> > > a
> > > > > > > >> > > > >> > >> > > > > > fatal
> > > > > > > >> > > > >> > >> > > > > > > > > error
> > > > > > > >> > > > >> > >> > > > > > > > > >>> and
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> we didn't have the
> > > mechanisms
> > > > in
> > > > > > > >> place to
> > > > > > > >> > > > >> detect
> > > > > > > >> > > > >> > >> when
> > > > > > > >> > > > >> > >> > > this
> > > > > > > >> > > > >> > >> > > > > was
> > > > > > > >> > > > >> > >> > > > > > a
> > > > > > > >> > > > >> > >> > > > > > > > > >>> legitimate
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> case vs some bug or gap
> in
> > > the
> > > > > > > >> protocol.
> > > > > > > >> > > > >> Ensuring
> > > > > > > >> > > > >> > >> each
> > > > > > > >> > > > >> > >> > > > > > > transaction
> > > > > > > >> > > > >> > >> > > > > > > > > has
> > > > > > > >> > > > >> > >> > > > > > > > > >>> its
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> own epoch should close
> > this
> > > > gap.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> And to address Jeff's
> > second
> > > > > > point:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> *does the typical
> produce
> > > > request
> > > > > > path
> > > > > > > >> > > append
> > > > > > > >> > > > >> > >> records
> > > > > > > >> > > > >> > >> > to
> > > > > > > >> > > > >> > >> > > > > local
> > > > > > > >> > > > >> > >> > > > > > > log
> > > > > > > >> > > > >> > >> > > > > > > > > >>> along*
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> *with the
> > > > currentTxnFirstOffset
> > > > > > > >> > > information?
> > > > > > > >> > > > I
> > > > > > > >> > > > >> > would
> > > > > > > >> > > > >> > >> > like
> > > > > > > >> > > > >> > >> > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > >>> understand*
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> *when the field is
> written
> > > to
> > > > > > disk.*
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> Yes, the first produce
> > > request
> > > > > > > >> populates
> > > > > > > >> > > this
> > > > > > > >> > > > >> > field
> > > > > > > >> > > > >> > >> and
> > > > > > > >> > > > >> > >> > > > > writes
> > > > > > > >> > > > >> > >> > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> offset
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> as part of the record
> > batch
> > > > and
> > > > > > also
> > > > > > > >> to
> > > > > > > >> > the
> > > > > > > >> > > > >> > producer
> > > > > > > >> > > > >> > >> > > state
> > > > > > > >> > > > >> > >> > > > > > > > snapshot.
> > > > > > > >> > > > >> > >> > > > > > > > > >>> When
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> we reload the records on
> > > > restart
> > > > > > > >> and/or
> > > > > > > >> > > > >> > >> reassignment,
> > > > > > > >> > > > >> > >> > we
> > > > > > > >> > > > >> > >> > > > > > > repopulate
> > > > > > > >> > > > >> > >> > > > > > > > > >>> this
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> field with the snapshot
> > from
> > > > disk
> > > > > > > >> along
> > > > > > > >> > > with
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> rest
> > > > > > > >> > > > >> > >> > of
> > > > > > > >> > > > >> > >> > > > the
> > > > > > > >> > > > >> > >> > > > > > > > producer
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> state.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> Let me know if there are
> > > > further
> > > > > > > >> comments
> > > > > > > >> > > > >> and/or
> > > > > > > >> > > > >> > >> > > questions.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> Thanks,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> Justine
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> On Tue, Nov 22, 2022 at
> > 9:00
> > > > PM
> > > > > > Jeff
> > > > > > > >> Kim
> > > > > > > >> > > > >> > >> > > > > > > > > <jeff.kim@confluent.io.invalid
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> wrote:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Hi Justine,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Thanks for the KIP! I
> > have
> > > > two
> > > > > > > >> > questions:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> 1) For new clients, we
> > can
> > > > once
> > > > > > again
> > > > > > > >> > > return
> > > > > > > >> > > > >> an
> > > > > > > >> > > > >> > >> error
> > > > > > > >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> for sequences
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> that are non-zero when
> > > there
> > > > is
> > > > > > no
> > > > > > > >> > > producer
> > > > > > > >> > > > >> state
> > > > > > > >> > > > >> > >> > > present
> > > > > > > >> > > > >> > >> > > > on
> > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> server.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> This will indicate we
> > > missed
> > > > the
> > > > > > 0
> > > > > > > >> > > sequence
> > > > > > > >> > > > >> and
> > > > > > > >> > > > >> > we
> > > > > > > >> > > > >> > >> > don't
> > > > > > > >> > > > >> > >> > > > yet
> > > > > > > >> > > > >> > >> > > > > > > want
> > > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> write
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> to the log.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> I would like to
> > understand
> > > > the
> > > > > > > >> current
> > > > > > > >> > > > >> behavior
> > > > > > > >> > > > >> > to
> > > > > > > >> > > > >> > >> > > handle
> > > > > > > >> > > > >> > >> > > > > > older
> > > > > > > >> > > > >> > >> > > > > > > > > >>> clients,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> and if there are any
> > > changes
> > > > we
> > > > > > are
> > > > > > > >> > > making.
> > > > > > > >> > > > >> Maybe
> > > > > > > >> > > > >> > >> I'm
> > > > > > > >> > > > >> > >> > > > > missing
> > > > > > > >> > > > >> > >> > > > > > > > > >>> something,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> but we would want to
> > > identify
> > > > > > > >> whether we
> > > > > > > >> > > > >> missed
> > > > > > > >> > > > >> > >> the 0
> > > > > > > >> > > > >> > >> > > > > sequence
> > > > > > > >> > > > >> > >> > > > > > > for
> > > > > > > >> > > > >> > >> > > > > > > > > >>> older
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> clients, no?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> 2) Upon returning from
> > the
> > > > > > > >> transaction
> > > > > > > >> > > > >> > >> coordinator, we
> > > > > > > >> > > > >> > >> > > can
> > > > > > > >> > > > >> > >> > > > > set
> > > > > > > >> > > > >> > >> > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> transaction
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> as ongoing on the
> leader
> > by
> > > > > > > >> populating
> > > > > > > >> > > > >> > >> > > > currentTxnFirstOffset
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> through the typical
> > produce
> > > > > > request
> > > > > > > >> > > > handling.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> does the typical
> produce
> > > > request
> > > > > > path
> > > > > > > >> > > append
> > > > > > > >> > > > >> > >> records
> > > > > > > >> > > > >> > >> > to
> > > > > > > >> > > > >> > >> > > > > local
> > > > > > > >> > > > >> > >> > > > > > > log
> > > > > > > >> > > > >> > >> > > > > > > > > >>> along
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> with the
> > > > currentTxnFirstOffset
> > > > > > > >> > > information?
> > > > > > > >> > > > I
> > > > > > > >> > > > >> > would
> > > > > > > >> > > > >> > >> > like
> > > > > > > >> > > > >> > >> > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > understand
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> when the field is
> written
> > > to
> > > > > > disk.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Thanks,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Jeff
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> On Tue, Nov 22, 2022 at
> > > 4:44
> > > > PM
> > > > > > Artem
> > > > > > > >> > > > Livshits
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> <
> alivshits@confluent.io
> > > > .invalid>
> > > > > > > >> wrote:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> Hi Justine,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> Thank you for the KIP.
> > I
> > > > have
> > > > > > one
> > > > > > > >> > > > question.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> 5) For new clients, we
> > can
> > > > once
> > > > > > > >> again
> > > > > > > >> > > > return
> > > > > > > >> > > > >> an
> > > > > > > >> > > > >> > >> error
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> I believe we had
> > problems
> > > > in the
> > > > > > > >> past
> > > > > > > >> > > with
> > > > > > > >> > > > >> > >> returning
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> because it was
> > considered
> > > > fatal
> > > > > > and
> > > > > > > >> > > > required
> > > > > > > >> > > > >> > >> client
> > > > > > > >> > > > >> > >> > > > > restart.
> > > > > > > >> > > > >> > >> > > > > > > It
> > > > > > > >> > > > >> > >> > > > > > > > > >>> would
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> be
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> good to spell out the
> > new
> > > > client
> > > > > > > >> > behavior
> > > > > > > >> > > > >> when
> > > > > > > >> > > > >> > it
> > > > > > > >> > > > >> > >> > > > receives
> > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > error.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> -Artem
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> On Tue, Nov 22, 2022
> at
> > > > 10:00 AM
> > > > > > > >> > Justine
> > > > > > > >> > > > >> Olshan
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > <jo...@confluent.io.invalid>
> > > > > > > >> wrote:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks for taking a
> > look
> > > > > > Matthias.
> > > > > > > >> > I've
> > > > > > > >> > > > >> tried
> > > > > > > >> > > > >> > to
> > > > > > > >> > > > >> > >> > > answer
> > > > > > > >> > > > >> > >> > > > > your
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> questions
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> below:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 10)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Right — so the
> hanging
> > > > > > transaction
> > > > > > > >> > only
> > > > > > > >> > > > >> occurs
> > > > > > > >> > > > >> > >> when
> > > > > > > >> > > > >> > >> > we
> > > > > > > >> > > > >> > >> > > > > have
> > > > > > > >> > > > >> > >> > > > > > a
> > > > > > > >> > > > >> > >> > > > > > > > late
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> message
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> come in and the
> > partition
> > > > is
> > > > > > never
> > > > > > > >> > added
> > > > > > > >> > > > to
> > > > > > > >> > > > >> a
> > > > > > > >> > > > >> > >> > > > transaction
> > > > > > > >> > > > >> > >> > > > > > > again.
> > > > > > > >> > > > >> > >> > > > > > > > > If
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> we
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> never add the
> partition
> > > to
> > > > a
> > > > > > > >> > > transaction,
> > > > > > > >> > > > we
> > > > > > > >> > > > >> > will
> > > > > > > >> > > > >> > >> > > never
> > > > > > > >> > > > >> > >> > > > > > write
> > > > > > > >> > > > >> > >> > > > > > > a
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> marker
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> and
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> never advance the
> LSO.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> If we do end up
> adding
> > > the
> > > > > > > >> partition
> > > > > > > >> > to
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > >> > > transaction
> > > > > > > >> > > > >> > >> > > > (I
> > > > > > > >> > > > >> > >> > > > > > > > suppose
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> this
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> can
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> happen before or
> after
> > > the
> > > > late
> > > > > > > >> > message
> > > > > > > >> > > > >> comes
> > > > > > > >> > > > >> > in)
> > > > > > > >> > > > >> > >> > then
> > > > > > > >> > > > >> > >> > > > we
> > > > > > > >> > > > >> > >> > > > > > will
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> include
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> late message in the
> > next
> > > > > > > >> (incorrect)
> > > > > > > >> > > > >> > transaction.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> So perhaps it is
> > clearer
> > > to
> > > > > > make
> > > > > > > >> the
> > > > > > > >> > > > >> > distinction
> > > > > > > >> > > > >> > >> > > between
> > > > > > > >> > > > >> > >> > > > > > > > messages
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> that
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> eventually get added
> to
> > > the
> > > > > > > >> > transaction
> > > > > > > >> > > > (but
> > > > > > > >> > > > >> > the
> > > > > > > >> > > > >> > >> > wrong
> > > > > > > >> > > > >> > >> > > > > one)
> > > > > > > >> > > > >> > >> > > > > > or
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> messages
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> that never get added
> > and
> > > > become
> > > > > > > >> > hanging.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 20)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> The client side
> change
> > > for
> > > > 2 is
> > > > > > > >> > removing
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > >> > > > addPartitions
> > > > > > > >> > > > >> > >> > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> transaction
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> call. We don't need
> to
> > > make
> > > > > > this
> > > > > > > >> from
> > > > > > > >> > > the
> > > > > > > >> > > > >> > >> producer
> > > > > > > >> > > > >> > >> > to
> > > > > > > >> > > > >> > >> > > > the
> > > > > > > >> > > > >> > >> > > > > > txn
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> coordinator,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> only server side.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> In my opinion, the
> > issue
> > > > with
> > > > > > the
> > > > > > > >> > > > >> > >> addPartitionsToTxn
> > > > > > > >> > > > >> > >> > > > call
> > > > > > > >> > > > >> > >> > > > > > for
> > > > > > > >> > > > >> > >> > > > > > > > > older
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> clients
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> is that we don't have
> > the
> > > > epoch
> > > > > > > >> bump,
> > > > > > > >> > so
> > > > > > > >> > > > we
> > > > > > > >> > > > >> > don't
> > > > > > > >> > > > >> > >> > know
> > > > > > > >> > > > >> > >> > > > if
> > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> message
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> belongs to the
> previous
> > > > > > > >> transaction or
> > > > > > > >> > > > this
> > > > > > > >> > > > >> > one.
> > > > > > > >> > > > >> > >> We
> > > > > > > >> > > > >> > >> > > need
> > > > > > > >> > > > >> > >> > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > check
> > > > > > > >> > > > >> > >> > > > > > > > > if
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> partition has been
> > added
> > > to
> > > > > > this
> > > > > > > >> > > > >> transaction.
> > > > > > > >> > > > >> > Of
> > > > > > > >> > > > >> > >> > > course,
> > > > > > > >> > > > >> > >> > > > > > this
> > > > > > > >> > > > >> > >> > > > > > > > > means
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> we
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> won't completely
> cover
> > > the
> > > > case
> > > > > > > >> where
> > > > > > > >> > we
> > > > > > > >> > > > >> have a
> > > > > > > >> > > > >> > >> > really
> > > > > > > >> > > > >> > >> > > > > late
> > > > > > > >> > > > >> > >> > > > > > > > > message
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> and
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> we
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> have added the
> > partition
> > > to
> > > > > > the new
> > > > > > > >> > > > >> > transaction,
> > > > > > > >> > > > >> > >> but
> > > > > > > >> > > > >> > >> > > > > that's
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> unfortunately
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> something we will
> need
> > > the
> > > > new
> > > > > > > >> clients
> > > > > > > >> > > to
> > > > > > > >> > > > >> > cover.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 30)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Transaction is
> ongoing
> > =
> > > > > > partition
> > > > > > > >> was
> > > > > > > >> > > > >> added to
> > > > > > > >> > > > >> > >> > > > > transaction
> > > > > > > >> > > > >> > >> > > > > > > via
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> addPartitionsToTxn.
> We
> > > > check
> > > > > > this
> > > > > > > >> with
> > > > > > > >> > > the
> > > > > > > >> > > > >> > >> > > > > > > DescribeTransactions
> > > > > > > >> > > > >> > >> > > > > > > > > >>> call.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Let
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> me know if this
> wasn't
> > > > > > sufficiently
> > > > > > > >> > > > >> explained
> > > > > > > >> > > > >> > >> here:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> > > > >> > >> > > > >
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > > >> > >> > >
> > > > > > > >> > > > >> > >> >
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >>
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >>
> > > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 40)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> The idea here is that
> > if
> > > > any
> > > > > > > >> messages
> > > > > > > >> > > > >> somehow
> > > > > > > >> > > > >> > >> come
> > > > > > > >> > > > >> > >> > in
> > > > > > > >> > > > >> > >> > > > > before
> > > > > > > >> > > > >> > >> > > > > > > we
> > > > > > > >> > > > >> > >> > > > > > > > > get
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> new
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> epoch to the
> producer,
> > > they
> > > > > > will be
> > > > > > > >> > > > fenced.
> > > > > > > >> > > > >> > >> However,
> > > > > > > >> > > > >> > >> > > if
> > > > > > > >> > > > >> > >> > > > we
> > > > > > > >> > > > >> > >> > > > > > > don't
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> think
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> this
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> is necessary, it can
> be
> > > > > > discussed
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 50)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> It should be
> > synchronous
> > > > > > because
> > > > > > > >> if we
> > > > > > > >> > > > have
> > > > > > > >> > > > >> an
> > > > > > > >> > > > >> > >> event
> > > > > > > >> > > > >> > >> > > > (ie,
> > > > > > > >> > > > >> > >> > > > > an
> > > > > > > >> > > > >> > >> > > > > > > > > error)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> that
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> causes us to need to
> > > abort
> > > > the
> > > > > > > >> > > > transaction,
> > > > > > > >> > > > >> we
> > > > > > > >> > > > >> > >> need
> > > > > > > >> > > > >> > >> > to
> > > > > > > >> > > > >> > >> > > > > know
> > > > > > > >> > > > >> > >> > > > > > > > which
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> partitions to send
> > > > transaction
> > > > > > > >> markers
> > > > > > > >> > > to.
> > > > > > > >> > > > >> We
> > > > > > > >> > > > >> > >> know
> > > > > > > >> > > > >> > >> > the
> > > > > > > >> > > > >> > >> > > > > > > > partitions
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> because
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> we added them to the
> > > > > > coordinator
> > > > > > > >> via
> > > > > > > >> > the
> > > > > > > >> > > > >> > >> > > > > addPartitionsToTxn
> > > > > > > >> > > > >> > >> > > > > > > > call.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Previously we have
> had
> > > > > > asynchronous
> > > > > > > >> > > calls
> > > > > > > >> > > > in
> > > > > > > >> > > > >> > the
> > > > > > > >> > > > >> > >> > past
> > > > > > > >> > > > >> > >> > > > (ie,
> > > > > > > >> > > > >> > >> > > > > > > > writing
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> commit markers when
> the
> > > > > > > >> transaction is
> > > > > > > >> > > > >> > completed)
> > > > > > > >> > > > >> > >> > but
> > > > > > > >> > > > >> > >> > > > > often
> > > > > > > >> > > > >> > >> > > > > > > this
> > > > > > > >> > > > >> > >> > > > > > > > > >>> just
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> causes confusion as
> we
> > > > need to
> > > > > > wait
> > > > > > > >> > for
> > > > > > > >> > > > some
> > > > > > > >> > > > >> > >> > > operations
> > > > > > > >> > > > >> > >> > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > complete.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> In
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> writing commit
> markers
> > > > case,
> > > > > > > >> clients
> > > > > > > >> > > often
> > > > > > > >> > > > >> see
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> CONCURRENT_TRANSACTIONs
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> error messages and
> that
> > > > can be
> > > > > > > >> > > confusing.
> > > > > > > >> > > > >> For
> > > > > > > >> > > > >> > >> that
> > > > > > > >> > > > >> > >> > > > reason,
> > > > > > > >> > > > >> > >> > > > > > it
> > > > > > > >> > > > >> > >> > > > > > > > may
> > > > > > > >> > > > >> > >> > > > > > > > > be
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> simpler to just have
> > > > > > synchronous
> > > > > > > >> > calls —
> > > > > > > >> > > > >> > >> especially
> > > > > > > >> > > > >> > >> > if
> > > > > > > >> > > > >> > >> > > > we
> > > > > > > >> > > > >> > >> > > > > > need
> > > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> block
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> on
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> some operation's
> > > completion
> > > > > > anyway
> > > > > > > >> > > before
> > > > > > > >> > > > we
> > > > > > > >> > > > >> > can
> > > > > > > >> > > > >> > >> > start
> > > > > > > >> > > > >> > >> > > > the
> > > > > > > >> > > > >> > >> > > > > > > next
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> transaction. And
> yes, I
> > > > meant
> > > > > > > >> > > > coordinator. I
> > > > > > > >> > > > >> > will
> > > > > > > >> > > > >> > >> > fix
> > > > > > > >> > > > >> > >> > > > > that.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 60)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> When we are checking
> if
> > > the
> > > > > > > >> > transaction
> > > > > > > >> > > is
> > > > > > > >> > > > >> > >> ongoing,
> > > > > > > >> > > > >> > >> > we
> > > > > > > >> > > > >> > >> > > > > need
> > > > > > > >> > > > >> > >> > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > make
> > > > > > > >> > > > >> > >> > > > > > > > > >>> a
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> round
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> trip from the leader
> > > > partition
> > > > > > to
> > > > > > > >> the
> > > > > > > >> > > > >> > transaction
> > > > > > > >> > > > >> > >> > > > > > coordinator.
> > > > > > > >> > > > >> > >> > > > > > > > In
> > > > > > > >> > > > >> > >> > > > > > > > > >>> the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> time
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> we are waiting for
> this
> > > > > > message to
> > > > > > > >> > come
> > > > > > > >> > > > >> back,
> > > > > > > >> > > > >> > in
> > > > > > > >> > > > >> > >> > > theory
> > > > > > > >> > > > >> > >> > > > we
> > > > > > > >> > > > >> > >> > > > > > > could
> > > > > > > >> > > > >> > >> > > > > > > > > >>> have
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> sent
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> a commit/abort call
> > that
> > > > would
> > > > > > make
> > > > > > > >> > the
> > > > > > > >> > > > >> > original
> > > > > > > >> > > > >> > >> > > result
> > > > > > > >> > > > >> > >> > > > of
> > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > check
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> out
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> of
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> date. That is why we
> > can
> > > > check
> > > > > > the
> > > > > > > >> > > leader
> > > > > > > >> > > > >> state
> > > > > > > >> > > > >> > >> > before
> > > > > > > >> > > > >> > >> > > > we
> > > > > > > >> > > > >> > >> > > > > > > write
> > > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> log.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> I'm happy to update
> the
> > > > KIP if
> > > > > > > >> some of
> > > > > > > >> > > > these
> > > > > > > >> > > > >> > >> things
> > > > > > > >> > > > >> > >> > > were
> > > > > > > >> > > > >> > >> > > > > not
> > > > > > > >> > > > >> > >> > > > > > > > > clear.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Justine
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> On Mon, Nov 21, 2022
> at
> > > > 7:11 PM
> > > > > > > >> > Matthias
> > > > > > > >> > > > J.
> > > > > > > >> > > > >> > Sax <
> > > > > > > >> > > > >> > >> > > > > > > > mjsax@apache.org
> > > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> wrote:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Thanks for the KIP.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Couple of
> > clarification
> > > > > > questions
> > > > > > > >> (I
> > > > > > > >> > am
> > > > > > > >> > > > >> not a
> > > > > > > >> > > > >> > >> > broker
> > > > > > > >> > > > >> > >> > > > > expert
> > > > > > > >> > > > >> > >> > > > > > > do
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> maybe
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> some question are
> > > obvious
> > > > for
> > > > > > > >> others,
> > > > > > > >> > > but
> > > > > > > >> > > > >> not
> > > > > > > >> > > > >> > >> for
> > > > > > > >> > > > >> > >> > me
> > > > > > > >> > > > >> > >> > > > with
> > > > > > > >> > > > >> > >> > > > > > my
> > > > > > > >> > > > >> > >> > > > > > > > lack
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> of
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> broker knowledge).
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (10)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> 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.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> What happens if the
> > > > message
> > > > > > come
> > > > > > > >> in
> > > > > > > >> > > > before
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> next
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> addPartitionsToTxn
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> request? It seems
> the
> > > > broker
> > > > > > > >> hosting
> > > > > > > >> > > the
> > > > > > > >> > > > >> data
> > > > > > > >> > > > >> > >> > > > partitions
> > > > > > > >> > > > >> > >> > > > > > > won't
> > > > > > > >> > > > >> > >> > > > > > > > > know
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> anything about it
> and
> > > > append
> > > > > > it to
> > > > > > > >> > the
> > > > > > > >> > > > >> > >> partition,
> > > > > > > >> > > > >> > >> > > too?
> > > > > > > >> > > > >> > >> > > > > What
> > > > > > > >> > > > >> > >> > > > > > > is
> > > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> difference between
> > both
> > > > cases?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Also, it seems a TX
> > > would
> > > > only
> > > > > > > >> hang,
> > > > > > > >> > if
> > > > > > > >> > > > >> there
> > > > > > > >> > > > >> > >> is no
> > > > > > > >> > > > >> > >> > > > > > following
> > > > > > > >> > > > >> > >> > > > > > > > TX
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> that
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> is
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> either committer or
> > > > aborted?
> > > > > > Thus,
> > > > > > > >> > for
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > case
> > > > > > > >> > > > >> > >> > > above,
> > > > > > > >> > > > >> > >> > > > > the
> > > > > > > >> > > > >> > >> > > > > > TX
> > > > > > > >> > > > >> > >> > > > > > > > > might
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> actually not hang
> (of
> > > > course,
> > > > > > we
> > > > > > > >> > might
> > > > > > > >> > > > get
> > > > > > > >> > > > >> an
> > > > > > > >> > > > >> > >> EOS
> > > > > > > >> > > > >> > >> > > > > violation
> > > > > > > >> > > > >> > >> > > > > > > if
> > > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> first
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> TX was aborted and
> the
> > > > second
> > > > > > > >> > > committed,
> > > > > > > >> > > > or
> > > > > > > >> > > > >> > the
> > > > > > > >> > > > >> > >> > other
> > > > > > > >> > > > >> > >> > > > way
> > > > > > > >> > > > >> > >> > > > > > > > > around).
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (20)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Of course, 1 and 2
> > > > require
> > > > > > > >> > client-side
> > > > > > > >> > > > >> > >> changes, so
> > > > > > > >> > > > >> > >> > > for
> > > > > > > >> > > > >> > >> > > > > > older
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> clients,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> those approaches
> won’t
> > > > apply.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> For (1) I understand
> > > why a
> > > > > > client
> > > > > > > >> > > change
> > > > > > > >> > > > is
> > > > > > > >> > > > >> > >> > > necessary,
> > > > > > > >> > > > >> > >> > > > > but
> > > > > > > >> > > > >> > >> > > > > > > not
> > > > > > > >> > > > >> > >> > > > > > > > > sure
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> why
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> we need a client
> > change
> > > > for
> > > > > > (2).
> > > > > > > >> Can
> > > > > > > >> > > you
> > > > > > > >> > > > >> > >> elaborate?
> > > > > > > >> > > > >> > >> > > --
> > > > > > > >> > > > >> > >> > > > > > Later
> > > > > > > >> > > > >> > >> > > > > > > > you
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> explain
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> that we should send
> a
> > > > > > > >> > > > >> > >> DescribeTransactionRequest,
> > > > > > > >> > > > >> > >> > > but I
> > > > > > > >> > > > >> > >> > > > > am
> > > > > > > >> > > > >> > >> > > > > > > not
> > > > > > > >> > > > >> > >> > > > > > > > > sure
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> why?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Can't we not just do
> > an
> > > > > > implicit
> > > > > > > >> > > > >> > >> AddPartiitonToTx,
> > > > > > > >> > > > >> > >> > > too?
> > > > > > > >> > > > >> > >> > > > > If
> > > > > > > >> > > > >> > >> > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > old
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> producer correctly
> > > > registered
> > > > > > the
> > > > > > > >> > > > partition
> > > > > > > >> > > > >> > >> > already,
> > > > > > > >> > > > >> > >> > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> TX-coordinator
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> can just ignore it
> as
> > > > it's an
> > > > > > > >> > > idempotent
> > > > > > > >> > > > >> > >> operation?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (30)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> To cover older
> > clients,
> > > > we
> > > > > > will
> > > > > > > >> > > ensure a
> > > > > > > >> > > > >> > >> > transaction
> > > > > > > >> > > > >> > >> > > > is
> > > > > > > >> > > > >> > >> > > > > > > > ongoing
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> before
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> we write to a
> > > transaction
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Not sure what you
> mean
> > > by
> > > > > > this?
> > > > > > > >> Can
> > > > > > > >> > you
> > > > > > > >> > > > >> > >> elaborate?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (40)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> [the
> TX-coordinator]
> > > will
> > > > > > write
> > > > > > > >> the
> > > > > > > >> > > > >> prepare
> > > > > > > >> > > > >> > >> commit
> > > > > > > >> > > > >> > >> > > > > message
> > > > > > > >> > > > >> > >> > > > > > > > with
> > > > > > > >> > > > >> > >> > > > > > > > > a
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> bumped
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> epoch and send
> > > > > > > >> WriteTxnMarkerRequests
> > > > > > > >> > > > with
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> > bumped
> > > > > > > >> > > > >> > >> > > > > > epoch.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Why do we use the
> > bumped
> > > > > > epoch for
> > > > > > > >> > > both?
> > > > > > > >> > > > It
> > > > > > > >> > > > >> > >> seems
> > > > > > > >> > > > >> > >> > > more
> > > > > > > >> > > > >> > >> > > > > > > > intuitive
> > > > > > > >> > > > >> > >> > > > > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> use
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> the current epoch,
> and
> > > > only
> > > > > > return
> > > > > > > >> > the
> > > > > > > >> > > > >> bumped
> > > > > > > >> > > > >> > >> epoch
> > > > > > > >> > > > >> > >> > > to
> > > > > > > >> > > > >> > >> > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> producer?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (50) "Implicit
> > > > > > > >> > > AddPartitionToTransaction"
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Why does the
> > implicitly
> > > > sent
> > > > > > > >> request
> > > > > > > >> > > need
> > > > > > > >> > > > >> to
> > > > > > > >> > > > >> > be
> > > > > > > >> > > > >> > >> > > > > > synchronous?
> > > > > > > >> > > > >> > >> > > > > > > > The
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> KIP
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> also says
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> in case we need to
> > > abort
> > > > and
> > > > > > > >> need to
> > > > > > > >> > > > know
> > > > > > > >> > > > >> > which
> > > > > > > >> > > > >> > >> > > > > partitions
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> What do you mean by
> > > this?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> we don’t want to
> > write
> > > > to it
> > > > > > > >> before
> > > > > > > >> > we
> > > > > > > >> > > > >> store
> > > > > > > >> > > > >> > in
> > > > > > > >> > > > >> > >> > the
> > > > > > > >> > > > >> > >> > > > > > > > transaction
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> manager
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Do you mean
> > > TX-coordinator
> > > > > > > >> instead of
> > > > > > > >> > > > >> > "manager"?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (60)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> For older clients
> and
> > > > ensuring
> > > > > > > >> that
> > > > > > > >> > the
> > > > > > > >> > > > TX
> > > > > > > >> > > > >> is
> > > > > > > >> > > > >> > >> > > ongoing,
> > > > > > > >> > > > >> > >> > > > > you
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> describe a
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> race condition. I am
> > not
> > > > sure
> > > > > > if I
> > > > > > > >> > can
> > > > > > > >> > > > >> follow
> > > > > > > >> > > > >> > >> here.
> > > > > > > >> > > > >> > >> > > Can
> > > > > > > >> > > > >> > >> > > > > you
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> elaborate?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> -Matthias
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> On 11/18/22 1:21 PM,
> > > > Justine
> > > > > > > >> Olshan
> > > > > > > >> > > > wrote:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Hey all!
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> I'd like to start a
> > > > > > discussion
> > > > > > > >> on my
> > > > > > > >> > > > >> proposal
> > > > > > > >> > > > >> > >> to
> > > > > > > >> > > > >> > >> > add
> > > > > > > >> > > > >> > >> > > > > some
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> server-side
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> checks on
> > transactions
> > > to
> > > > > > avoid
> > > > > > > >> > > hanging
> > > > > > > >> > > > >> > >> > > transactions.
> > > > > > > >> > > > >> > >> > > > I
> > > > > > > >> > > > >> > >> > > > > > know
> > > > > > > >> > > > >> > >> > > > > > > > > this
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> has
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> been
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> an issue for some
> > time,
> > > > so I
> > > > > > > >> really
> > > > > > > >> > > hope
> > > > > > > >> > > > >> this
> > > > > > > >> > > > >> > >> KIP
> > > > > > > >> > > > >> > >> > > will
> > > > > > > >> > > > >> > >> > > > > be
> > > > > > > >> > > > >> > >> > > > > > > > > helpful
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> for
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> many
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> users of EOS.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> The KIP includes
> > > changes
> > > > that
> > > > > > > >> will
> > > > > > > >> > be
> > > > > > > >> > > > >> > >> compatible
> > > > > > > >> > > > >> > >> > > with
> > > > > > > >> > > > >> > >> > > > > old
> > > > > > > >> > > > >> > >> > > > > > > > > clients
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> and
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> changes to improve
> > > > > > performance
> > > > > > > >> and
> > > > > > > >> > > > >> > correctness
> > > > > > > >> > > > >> > >> on
> > > > > > > >> > > > >> > >> > > new
> > > > > > > >> > > > >> > >> > > > > > > clients.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Please take a look
> > and
> > > > leave
> > > > > > any
> > > > > > > >> > > > comments
> > > > > > > >> > > > >> you
> > > > > > > >> > > > >> > >> may
> > > > > > > >> > > > >> > >> > > > have!
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> KIP:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> > > > >> > >> > > > >
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > > >> > >> > >
> > > > > > > >> > > > >> > >> >
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >>
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >>
> > > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> JIRA:
> > > > > > > >> > > > >> > >> > > >
> > > > https://issues.apache.org/jira/browse/KAFKA-14402
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Thanks!
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Justine
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> > > > >> > >> > > > >
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > > >> > >> > >
> > > > > > > >> > > > >> > >> >
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> > >
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >>
> > > > > > > >> > > > >
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >>
> > > > > > > >
> > > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
Artem --
I guess the discussion path we were going down is when we expect to see
this error. I mentioned that it was hard to come up with cases for when the
producer would still be around to receive the error besides the poorly
written client case.
If we don't expect to have a producer to receive the response, it sort of
makes sense for it to be fatal.

I had some discussion with Jason offline about the epoch being off cases
and I'm not sure we could find a ton (outside of produce requests) where we
could/should recover. I'd be happy to hear some examples though, maybe I'm
missing something.

Thanks,
Justine

On Fri, Jan 20, 2023 at 3:19 PM Artem Livshits
<al...@confluent.io.invalid> wrote:

> In general, I'd like to avoid fatal errors as much as possible, in some
> sense fatal errors just push out recovery logic to the application which
> either complicates the application or leads to disruption (we've seen cases
> when a transient broker error could lead to work stoppage when applications
> need to be manually restarted).  I think we should strive to define
> recovery logic for most errors (and/or encapsulate it in the Kafka client
> as much as possible).
>
> One benefit of transactions is that they simplify recovery from errors,
> pretty much any error (that's not handled transparently by retries in Kafka
> client) can be handled by the application via aborting the transaction and
> repeating the transactional logic again.  One tricky error is an error
> during commit, because we don't know the outcome.  For commit errors, the
> recommendation should be to retry the commit until it returns the specific
> result (committed or aborted).
>
> -Artem
>
> On Fri, Jan 20, 2023 at 2:52 PM Justine Olshan
> <jo...@confluent.io.invalid>
> wrote:
>
> > That's a fair point about other clients.
> >
> > I think the abortable error case is interesting because I'm curious how
> > other clients would handle this. I assume they would need to implement
> > handling for the error code unless they did something like "any unknown
> > error codes/any codes that aren't x,y,z are retriable." I would hope that
> > unknown error codes were fatal, and if the code was implemented it would
> > abort the transaction. But I will think on this too.
> >
> > As for InvalidRecord -- you mentioned it was not fatal, but I'm taking a
> > look through the code. We would see this on handling the produce
> response.
> > If I recall correctly, we check if errors are retriable. I think this
> error
> > would not be retriable. But I guess the concern here is that it is not
> > enough for just that batch to fail. I guess I hadn't considered fully
> > fencing the old producer but there are valid arguments here why we would
> > want to.
> >
> > Thanks,
> > Justine
> >
> > On Fri, Jan 20, 2023 at 2:35 PM Guozhang Wang <
> guozhang.wang.us@gmail.com>
> > wrote:
> >
> > > Thanks Justine for the replies! I agree with most of your thoughts.
> > >
> > > Just for 3/7), though I agree for our own AK producer, since we do
> > > "nextRequest(boolean hasIncompleteBatches)", we guarantee the end-txn
> > > would not be sent until we've effectively flushed, but I was referring
> > > to any future bugs or other buggy clients that the same client may get
> > > into this situation, in which case we should give the client a clear
> > > msg that "you did something wrong, and hence now you should fatally
> > > close yourself". What I'm concerned about is that, by seeing an
> > > "abortable error" or in some rare cases an "invalid record", the
> > > client could not realize "something that's really bad happened". So
> > > it's not about adding a new error, it's mainly about those real buggy
> > > situations causing such "should never happen" cases, the errors return
> > > would not be informative enough.
> > >
> > > Thinking in other ways, if we believe that for most cases such error
> > > codes would not reach the original clients since they would be
> > > disconnected or even gone by that time, and only in some rare cases
> > > they would still be seen by the sending clients, then why not make
> > > them more fatal and more specific than generic.
> > >
> > > Guozhang
> > >
> > > On Fri, Jan 20, 2023 at 1:59 PM Justine Olshan
> > > <jo...@confluent.io.invalid> wrote:
> > > >
> > > > Hey Guozhang. Thanks for taking a look and for the detailed comments!
> > > I'll
> > > > do my best to address below.
> > > >
> > > > 1. I see what you are saying here, but I think I need to look through
> > the
> > > > sequence of events you mention. Typically we've seen this issue in a
> > few
> > > > cases.
> > > >
> > > >  One is when we have a producer disconnect when trying to produce.
> > > > Typically in these cases, we abort the transaction. We've seen that
> > after
> > > > the markers are written, the disconnection can sometimes cause the
> > > request
> > > > to get flushed to the broker. In this case, we don't need client
> > handling
> > > > because the producer we are responding to is gone. We just needed to
> > make
> > > > sure we didn't write to the log on the broker side. I'm trying to
> think
> > > of
> > > > a case where we do have the client to return to. I'd think the same
> > > client
> > > > couldn't progress to committing the transaction unless the produce
> > > request
> > > > returned right? Of course, there is the incorrectly written clients
> > case.
> > > > I'll think on this a bit more and let you know if I come up with
> > another
> > > > scenario when we would return to an active client when the
> transaction
> > is
> > > > no longer ongoing.
> > > >
> > > > I was not aware that we checked the result of a send after we commit
> > > > though. I'll need to look into that a bit more.
> > > >
> > > > 2. There were some questions about this in the discussion. The plan
> is
> > to
> > > > handle overflow with the mechanism we currently have in the producer.
> > If
> > > we
> > > > try to bump and the epoch will overflow, we actually allocate a new
> > > > producer ID. I need to confirm the fencing logic on the last epoch
> (ie,
> > > we
> > > > probably shouldn't allow any records to be produced with the final
> > epoch
> > > > since we can never properly fence that one).
> > > >
> > > > 3. I can agree with you that the current error handling is messy. I
> > > recall
> > > > taking a look at your KIP a while back, but I think I mostly saw the
> > > > section about how the errors were wrapped. Maybe I need to take
> another
> > > > look. As for abortable error, the idea was that the handling would be
> > > > simple -- if this error is seen, the transaction should be aborted --
> > no
> > > > other logic about previous state or requests necessary. Is your
> concern
> > > > simply about adding new errors? We were hoping to have an error that
> > > would
> > > > have one meaning and many of the current errors have a history of
> > meaning
> > > > different things on different client versions. That was the main
> > > motivation
> > > > for adding a new error.
> > > >
> > > > 4. This is a good point about record timestamp reordering. Timestamps
> > > don't
> > > > affect compaction, but they do affect retention deletion. For that,
> > kafka
> > > > considers the largest timestamp in the segment, so I think a small
> > amount
> > > > of reordering (hopefully on the order of milliseconds or even
> seconds)
> > > will
> > > > be ok. We take timestamps from clients so there is already a
> > possibility
> > > > for some drift and non-monotonically increasing timestamps.
> > > >
> > > > 5. Thanks for catching. The error is there, but it's actually that
> > those
> > > > fields should be 4+! Due to how the message generator works, I
> actually
> > > > have to redefine those fields inside the
> > `"AddPartitionsToTxnTransaction`
> > > > block for it to build correctly. I'll fix it to be correct.
> > > >
> > > > 6. Correct -- we will only add the request to purgatory if the cache
> > has
> > > no
> > > > ongoing transaction. I can change the wording to make that clearer
> that
> > > we
> > > > only place the request in purgatory if we need to contact the
> > transaction
> > > > coordinator.
> > > >
> > > > 7. We did take a look at some of the errors and it was hard to come
> up
> > > with
> > > > a good one. I agree that InvalidTxnStateException is ideal except for
> > the
> > > > fact that it hasn't been returned on Produce requests before. The
> error
> > > > handling for clients is a bit vague (which is why I opened
> KAFKA-14439
> > > > <https://issues.apache.org/jira/browse/KAFKA-14439>), but the
> decision
> > > we
> > > > made here was to only return errors that have been previously
> returned
> > to
> > > > producers. As for not being fatal, I think part of the theory was
> that
> > in
> > > > many cases, the producer would be disconnected. (See point 1) and
> this
> > > > would just be an error to return from the server. I did plan to think
> > > about
> > > > other cases, so let me know if you think of any as well!
> > > >
> > > > Lots to say! Let me know if you have further thoughts!
> > > > Justine
> > > >
> > > > On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang <
> > > guozhang.wang.us@gmail.com>
> > > > wrote:
> > > >
> > > > > Hello Justine,
> > > > >
> > > > > Thanks for the great write-up! I made a quick pass through it and
> > here
> > > > > are some thoughts (I have not been able to read through this thread
> > so
> > > > > pardon me if they have overlapped or subsumed by previous
> comments):
> > > > >
> > > > > First are some meta ones:
> > > > >
> > > > > 1. I think we need to also improve the client's experience once we
> > > > > have this defence in place. More concretely, say a user's producer
> > > > > code is like following:
> > > > >
> > > > > future = producer.send();
> > > > > // producer.flush();
> > > > > producer.commitTransaction();
> > > > > future.get();
> > > > >
> > > > > Which resulted in the order of a) produce-request sent by producer,
> > b)
> > > > > end-txn-request sent by producer, c) end-txn-response sent back, d)
> > > > > txn-marker-request sent from coordinator to partition leader, e)
> > > > > produce-request finally received by the partition leader, before
> this
> > > > > KIP e) step would be accepted causing a dangling txn; now it would
> be
> > > > > rejected in step e) which is good. But from the client's point of
> > view
> > > > > now it becomes confusing since the `commitTransaction()` returns
> > > > > successfully, but the "future" throws an invalid-epoch error, and
> > they
> > > > > are not sure if the transaction did succeed or not. In fact, it
> > > > > "partially succeeded" with some msgs being rejected but others
> > > > > committed successfully.
> > > > >
> > > > > Of course the easy way to avoid this is, always call
> > > > > "producer.flush()" before commitTxn and that's what we do
> ourselves,
> > > > > and what we recommend users do. But I suspect not everyone does it.
> > In
> > > > > fact I just checked the javadoc in KafkaProducer and our code
> snippet
> > > > > does not include a `flush()` call. So I'm thinking maybe we can in
> > > > > side the `commitTxn` code to enforce flushing before sending the
> > > > > end-txn request.
> > > > >
> > > > > 2. I'd like to clarify a bit details on "just add partitions to the
> > > > > transaction on the first produce request during a transaction". My
> > > > > understanding is that the partition leader's cache has the producer
> > id
> > > > > / sequence / epoch for the latest txn, either on-going or is
> > completed
> > > > > (upon receiving the marker request from coordinator). When a
> produce
> > > > > request is received, if
> > > > >
> > > > > * producer's epoch < cached epoch, or producer's epoch == cached
> > epoch
> > > > > but the latest txn is completed, leader directly reject with
> > > > > invalid-epoch.
> > > > > * producer's epoch > cached epoch, park the the request and send
> > > > > add-partitions request to coordinator.
> > > > >
> > > > > In order to do it, does the coordinator need to bump the sequence
> and
> > > > > reset epoch to 0 when the next epoch is going to overflow? If no
> need
> > > > > to do so, then how we handle the (admittedly rare, but still may
> > > > > happen) epoch overflow situation?
> > > > >
> > > > > 3. I'm a bit concerned about adding a generic "ABORTABLE_ERROR"
> given
> > > > > we already have a pretty messy error classification and error
> > handling
> > > > > on the producer clients side --- I have a summary about the issues
> > and
> > > > > a proposal to address this in
> > > > >
> > > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
> > > > > -- I understand we do not want to use "UNKNOWN_PRODUCER_ID" anymore
> > > > > and in fact we intend to deprecate it in KIP-360 and eventually
> > remove
> > > > > it; but I'm wondering can we still use specific error codes. E.g.
> > what
> > > > > about "InvalidProducerEpochException" since for new clients, the
> > > > > actual reason this would actually be rejected is indeed because the
> > > > > epoch on the coordinator caused the add-partitions-request from the
> > > > > brokers to be rejected anyways?
> > > > >
> > > > > 4. It seems we put the producer request into purgatory before we
> ever
> > > > > append the records, while other producer's records may still be
> > > > > appended during the time; and that potentially may result in some
> > > > > re-ordering compared with reception order. I'm not super concerned
> > > > > about it since Kafka does not guarantee reception ordering across
> > > > > producers anyways, but it may make the timestamps of records
> inside a
> > > > > partition to be more out-of-ordered. Are we aware of any scenarios
> > > > > such as future enhancements on log compactions that may be affected
> > by
> > > > > this effect?
> > > > >
> > > > > Below are just minor comments:
> > > > >
> > > > > 5. In "AddPartitionsToTxnTransaction" field of
> > > > > "AddPartitionsToTxnRequest" RPC, the versions of those inner fields
> > > > > are "0-3" while I thought they should be "0+" still?
> > > > >
> > > > > 6. Regarding "we can place the request in a purgatory of sorts and
> > > > > check if there is any state for the transaction on the broker": i
> > > > > think at this time when we just do the checks against the cached
> > > > > state, we do not need to put the request to purgatory yet?
> > > > >
> > > > > 7. This is related to 3) above. I feel using
> "InvalidRecordException"
> > > > > for older clients may also be a bit confusing, and also it is not
> > > > > fatal -- for old clients, it better to be fatal since this
> indicates
> > > > > the clients is doing something wrong and hence it should be closed.
> > > > > And in general I'd prefer to use slightly more specific meaning
> error
> > > > > codes for clients. That being said, I also feel
> > > > > "InvalidProducerEpochException" is not suitable for old versioned
> > > > > clients, and we'd have to pick one that old clients recognize. I'd
> > > > > prefer "InvalidTxnStateException" but that one is supposed to be
> > > > > returned from txn coordinators only today. I'd suggest we do a
> quick
> > > > > check in the current client's code path and see if that one would
> be
> > > > > handled if it's from a produce-response, and if yes, use this one;
> > > > > otherwise, use "ProducerFencedException" which is much less
> > meaningful
> > > > > but it's still a fatal error.
> > > > >
> > > > >
> > > > > Thanks,
> > > > > Guozhang
> > > > >
> > > > >
> > > > >
> > > > > On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
> > > > > <jo...@confluent.io.invalid> wrote:
> > > > > >
> > > > > > Yeah -- looks like we already have code to handle bumping the
> epoch
> > > and
> > > > > > when the epoch is Short.MAX_VALUE, we get a new producer ID.
> Since
> > > this
> > > > > is
> > > > > > already the behavior, do we want to change it further?
> > > > > >
> > > > > > Justine
> > > > > >
> > > > > > On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <
> > jolshan@confluent.io
> > > >
> > > > > wrote:
> > > > > >
> > > > > > > Hey all, just wanted to quickly update and say I've modified
> the
> > > KIP to
> > > > > > > explicitly mention that AddOffsetCommitsToTxnRequest will be
> > > replaced
> > > > > by
> > > > > > > a coordinator-side (inter-broker) AddPartitionsToTxn implicit
> > > request.
> > > > > This
> > > > > > > mirrors the user partitions and will implicitly add offset
> > > partitions
> > > > > to
> > > > > > > transactions when we commit offsets on them. We will deprecate
> > > > > AddOffsetCommitsToTxnRequest
> > > > > > > for new clients.
> > > > > > >
> > > > > > > Also to address Artem's comments --
> > > > > > > I'm a bit unsure if the changes here will change the previous
> > > behavior
> > > > > for
> > > > > > > fencing producers. In the case you mention in the first
> > paragraph,
> > > are
> > > > > you
> > > > > > > saying we bump the epoch before we try to abort the
> transaction?
> > I
> > > > > think I
> > > > > > > need to understand the scenarios you mention a bit better.
> > > > > > >
> > > > > > > As for the second part -- I think it makes sense to have some
> > sort
> > > of
> > > > > > > "sentinel" epoch to signal epoch is about to overflow (I think
> we
> > > sort
> > > > > of
> > > > > > > have this value in place in some ways) so we can codify it in
> the
> > > KIP.
> > > > > I'll
> > > > > > > look into that and try to update soon.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Justine.
> > > > > > >
> > > > > > > On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
> > > > > > > <al...@confluent.io.invalid> wrote:
> > > > > > >
> > > > > > >> It's good to know that KIP-588 addressed some of the issues.
> > > Looking
> > > > > at
> > > > > > >> the code, it still looks like there are some cases that would
> > > result
> > > > > in
> > > > > > >> fatal error, e.g. PRODUCER_FENCED is issued by the transaction
> > > > > coordinator
> > > > > > >> if epoch doesn't match, and the client treats it as a fatal
> > error
> > > > > (code in
> > > > > > >> TransactionManager request handling).  If we consider, for
> > > example,
> > > > > > >> committing a transaction that returns a timeout, but actually
> > > > > succeeds,
> > > > > > >> trying to abort it or re-commit may result in PRODUCER_FENCED
> > > error
> > > > > > >> (because of epoch bump).
> > > > > > >>
> > > > > > >> For failed commits, specifically, we need to know the actual
> > > outcome,
> > > > > > >> because if we return an error the application may think that
> the
> > > > > > >> transaction is aborted and redo the work, leading to
> duplicates.
> > > > > > >>
> > > > > > >> Re: overflowing epoch.  We could either do it on the TC and
> > return
> > > > > both
> > > > > > >> producer id and epoch (e.g. change the protocol), or signal
> the
> > > client
> > > > > > >> that
> > > > > > >> it needs to get a new producer id.  Checking for max epoch
> could
> > > be a
> > > > > > >> reasonable signal, the value to check should probably be
> present
> > > in
> > > > > the
> > > > > > >> KIP
> > > > > > >> as this is effectively a part of the contract.  Also, the TC
> > > should
> > > > > > >> probably return an error if the client didn't change producer
> id
> > > after
> > > > > > >> hitting max epoch.
> > > > > > >>
> > > > > > >> -Artem
> > > > > > >>
> > > > > > >>
> > > > > > >> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
> > > > > > >> <jo...@confluent.io.invalid> wrote:
> > > > > > >>
> > > > > > >> > Thanks for the discussion Artem.
> > > > > > >> >
> > > > > > >> > With respect to the handling of fenced producers, we have
> some
> > > > > behavior
> > > > > > >> > already in place. As of KIP-588:
> > > > > > >> >
> > > > > > >> >
> > > > > > >>
> > > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> > > > > > >> > ,
> > > > > > >> > we handle timeouts more gracefully. The producer can
> recover.
> > > > > > >> >
> > > > > > >> > Produce requests can also recover from epoch fencing by
> > > aborting the
> > > > > > >> > transaction and starting over.
> > > > > > >> >
> > > > > > >> > What other cases were you considering that would cause us to
> > > have a
> > > > > > >> fenced
> > > > > > >> > epoch but we'd want to recover?
> > > > > > >> >
> > > > > > >> > The first point about handling epoch overflows is fair. I
> > think
> > > > > there is
> > > > > > >> > some logic we'd need to consider. (ie, if we are one away
> from
> > > the
> > > > > max
> > > > > > >> > epoch, we need to reset the producer ID.) I'm still
> wondering
> > if
> > > > > there
> > > > > > >> is a
> > > > > > >> > way to direct this from the response, or if everything
> should
> > be
> > > > > done on
> > > > > > >> > the client side. Let me know if you have any thoughts here.
> > > > > > >> >
> > > > > > >> > Thanks,
> > > > > > >> > Justine
> > > > > > >> >
> > > > > > >> > On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
> > > > > > >> > <al...@confluent.io.invalid> wrote:
> > > > > > >> >
> > > > > > >> > > There are some workflows in the client that are implied by
> > > > > protocol
> > > > > > >> > > changes, e.g.:
> > > > > > >> > >
> > > > > > >> > > - for new clients, epoch changes with every transaction
> and
> > > can
> > > > > > >> overflow,
> > > > > > >> > > in old clients this condition was handled transparently,
> > > because
> > > > > epoch
> > > > > > >> > was
> > > > > > >> > > bumped in InitProducerId and it would return a new
> producer
> > > id if
> > > > > > >> epoch
> > > > > > >> > > overflows, the new clients would need to implement some
> > > workflow
> > > > > to
> > > > > > >> > refresh
> > > > > > >> > > producer id
> > > > > > >> > > - how to handle fenced producers, for new clients epoch
> > > changes
> > > > > with
> > > > > > >> > every
> > > > > > >> > > transaction, so in presence of failures during commits /
> > > aborts,
> > > > > the
> > > > > > >> > > producer could get easily fenced, old clients would pretty
> > > much
> > > > > would
> > > > > > >> get
> > > > > > >> > > fenced when a new incarnation of the producer was
> > initialized
> > > with
> > > > > > >> > > InitProducerId so it's ok to treat as a fatal error, the
> new
> > > > > clients
> > > > > > >> > would
> > > > > > >> > > need to implement some workflow to handle that error,
> > > otherwise
> > > > > they
> > > > > > >> > could
> > > > > > >> > > get fenced by themselves
> > > > > > >> > > - in particular (as a subset of the previous issue), what
> > > would
> > > > > the
> > > > > > >> > client
> > > > > > >> > > do if it got a timeout during commit?  commit could've
> > > succeeded
> > > > > or
> > > > > > >> > failed
> > > > > > >> > >
> > > > > > >> > > Not sure if this has to be defined in the KIP as
> > implementing
> > > > > those
> > > > > > >> > > probably wouldn't require protocol changes, but we have
> > > multiple
> > > > > > >> > > implementations of Kafka clients, so probably would be
> good
> > to
> > > > > have
> > > > > > >> some
> > > > > > >> > > client implementation guidance.  Could also be done as a
> > > separate
> > > > > doc.
> > > > > > >> > >
> > > > > > >> > > -Artem
> > > > > > >> > >
> > > > > > >> > > On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
> > > > > > >> > <jolshan@confluent.io.invalid
> > > > > > >> > > >
> > > > > > >> > > wrote:
> > > > > > >> > >
> > > > > > >> > > > Hey all, I've updated the KIP to incorporate Jason's
> > > > > suggestions.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > > 1. Use AddPartitionsToTxn + verify flag to check on old
> > > clients
> > > > > > >> > > > 2. Updated AddPartitionsToTxn API to support transaction
> > > > > batching
> > > > > > >> > > > 3. Mention IBP bump
> > > > > > >> > > > 4. Mention auth change on new AddPartitionsToTxn
> version.
> > > > > > >> > > >
> > > > > > >> > > > I'm planning on opening a vote soon.
> > > > > > >> > > > Thanks,
> > > > > > >> > > > Justine
> > > > > > >> > > >
> > > > > > >> > > > On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <
> > > > > jolshan@confluent.io
> > > > > > >> >
> > > > > > >> > > > wrote:
> > > > > > >> > > >
> > > > > > >> > > > > Thanks Jason. Those changes make sense to me. I will
> > > update
> > > > > the
> > > > > > >> KIP.
> > > > > > >> > > > >
> > > > > > >> > > > >
> > > > > > >> > > > >
> > > > > > >> > > > > On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
> > > > > > >> > > > <ja...@confluent.io.invalid>
> > > > > > >> > > > > wrote:
> > > > > > >> > > > >
> > > > > > >> > > > >> Hey Justine,
> > > > > > >> > > > >>
> > > > > > >> > > > >> > I was wondering about compatibility here. When we
> > send
> > > > > requests
> > > > > > >> > > > >> between brokers, we want to ensure that the receiving
> > > broker
> > > > > > >> > > understands
> > > > > > >> > > > >> the request (specifically the new fields). Typically
> > > this is
> > > > > done
> > > > > > >> > via
> > > > > > >> > > > >> IBP/metadata version.
> > > > > > >> > > > >> I'm trying to think if there is a way around it but
> I'm
> > > not
> > > > > sure
> > > > > > >> > there
> > > > > > >> > > > is.
> > > > > > >> > > > >>
> > > > > > >> > > > >> Yes. I think we would gate usage of this behind an
> IBP
> > > bump.
> > > > > Does
> > > > > > >> > that
> > > > > > >> > > > >> seem
> > > > > > >> > > > >> reasonable?
> > > > > > >> > > > >>
> > > > > > >> > > > >> > As for the improvements -- can you clarify how the
> > > multiple
> > > > > > >> > > > >> transactional
> > > > > > >> > > > >> IDs would help here? Were you thinking of a case
> where
> > we
> > > > > > >> wait/batch
> > > > > > >> > > > >> multiple produce requests together? My understanding
> > for
> > > now
> > > > > was
> > > > > > >> 1
> > > > > > >> > > > >> transactional ID and one validation per 1 produce
> > > request.
> > > > > > >> > > > >>
> > > > > > >> > > > >> Each call to `AddPartitionsToTxn` is essentially a
> > write
> > > to
> > > > > the
> > > > > > >> > > > >> transaction
> > > > > > >> > > > >> log and must block on replication. The more we can
> fit
> > > into a
> > > > > > >> single
> > > > > > >> > > > >> request, the more writes we can do in parallel. The
> > > > > alternative
> > > > > > >> is
> > > > > > >> > to
> > > > > > >> > > > make
> > > > > > >> > > > >> use of more connections, but usually we prefer
> batching
> > > > > since the
> > > > > > >> > > > network
> > > > > > >> > > > >> stack is not really optimized for high
> > connection/request
> > > > > loads.
> > > > > > >> > > > >>
> > > > > > >> > > > >> > Finally with respect to the authorizations, I think
> > it
> > > > > makes
> > > > > > >> sense
> > > > > > >> > > to
> > > > > > >> > > > >> skip
> > > > > > >> > > > >> topic authorizations, but I'm a bit confused by the
> > > "leader
> > > > > ID"
> > > > > > >> > field.
> > > > > > >> > > > >> Wouldn't we just want to flag the request as from a
> > > broker
> > > > > (does
> > > > > > >> it
> > > > > > >> > > > matter
> > > > > > >> > > > >> which one?).
> > > > > > >> > > > >>
> > > > > > >> > > > >> We could also make it version-based. For the next
> > > version, we
> > > > > > >> could
> > > > > > >> > > > >> require
> > > > > > >> > > > >> CLUSTER auth. So clients would not be able to use the
> > API
> > > > > > >> anymore,
> > > > > > >> > > which
> > > > > > >> > > > >> is
> > > > > > >> > > > >> probably what we want.
> > > > > > >> > > > >>
> > > > > > >> > > > >> -Jason
> > > > > > >> > > > >>
> > > > > > >> > > > >> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> > > > > > >> > > > >> <jo...@confluent.io.invalid>
> > > > > > >> > > > >> wrote:
> > > > > > >> > > > >>
> > > > > > >> > > > >> > As a follow up, I was just thinking about the
> > batching
> > > a
> > > > > bit
> > > > > > >> more.
> > > > > > >> > > > >> > I suppose if we have one request in flight and we
> > > queue up
> > > > > the
> > > > > > >> > other
> > > > > > >> > > > >> > produce requests in some sort of purgatory, we
> could
> > > send
> > > > > > >> > > information
> > > > > > >> > > > >> out
> > > > > > >> > > > >> > for all of them rather than one by one. So that
> would
> > > be a
> > > > > > >> benefit
> > > > > > >> > > of
> > > > > > >> > > > >> > batching partitions to add per transaction.
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > I'll need to think a bit more on the design of this
> > > part
> > > > > of the
> > > > > > >> > KIP,
> > > > > > >> > > > and
> > > > > > >> > > > >> > will update the KIP in the next few days.
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > Thanks,
> > > > > > >> > > > >> > Justine
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan <
> > > > > > >> > > jolshan@confluent.io>
> > > > > > >> > > > >> > wrote:
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > > Hey Jason -- thanks for the input -- I was just
> > > digging
> > > > > a bit
> > > > > > >> > > deeper
> > > > > > >> > > > >> into
> > > > > > >> > > > >> > > the design + implementation of the validation
> calls
> > > here
> > > > > and
> > > > > > >> > what
> > > > > > >> > > > you
> > > > > > >> > > > >> say
> > > > > > >> > > > >> > > makes sense.
> > > > > > >> > > > >> > >
> > > > > > >> > > > >> > > I was wondering about compatibility here. When we
> > > send
> > > > > > >> requests
> > > > > > >> > > > >> > > between brokers, we want to ensure that the
> > receiving
> > > > > broker
> > > > > > >> > > > >> understands
> > > > > > >> > > > >> > > the request (specifically the new fields).
> > Typically
> > > > > this is
> > > > > > >> > done
> > > > > > >> > > > via
> > > > > > >> > > > >> > > IBP/metadata version.
> > > > > > >> > > > >> > > I'm trying to think if there is a way around it
> but
> > > I'm
> > > > > not
> > > > > > >> sure
> > > > > > >> > > > there
> > > > > > >> > > > >> > is.
> > > > > > >> > > > >> > >
> > > > > > >> > > > >> > > As for the improvements -- can you clarify how
> the
> > > > > multiple
> > > > > > >> > > > >> transactional
> > > > > > >> > > > >> > > IDs would help here? Were you thinking of a case
> > > where we
> > > > > > >> > > wait/batch
> > > > > > >> > > > >> > > multiple produce requests together? My
> > understanding
> > > for
> > > > > now
> > > > > > >> > was 1
> > > > > > >> > > > >> > > transactional ID and one validation per 1 produce
> > > > > request.
> > > > > > >> > > > >> > >
> > > > > > >> > > > >> > > Finally with respect to the authorizations, I
> think
> > > it
> > > > > makes
> > > > > > >> > sense
> > > > > > >> > > > to
> > > > > > >> > > > >> > skip
> > > > > > >> > > > >> > > topic authorizations, but I'm a bit confused by
> the
> > > > > "leader
> > > > > > >> ID"
> > > > > > >> > > > field.
> > > > > > >> > > > >> > > Wouldn't we just want to flag the request as
> from a
> > > > > broker
> > > > > > >> (does
> > > > > > >> > > it
> > > > > > >> > > > >> > matter
> > > > > > >> > > > >> > > which one?).
> > > > > > >> > > > >> > >
> > > > > > >> > > > >> > > I think I want to adopt these suggestions, just
> had
> > > a few
> > > > > > >> > > questions
> > > > > > >> > > > on
> > > > > > >> > > > >> > the
> > > > > > >> > > > >> > > details.
> > > > > > >> > > > >> > >
> > > > > > >> > > > >> > > Thanks,
> > > > > > >> > > > >> > > Justine
> > > > > > >> > > > >> > >
> > > > > > >> > > > >> > > On Thu, Jan 5, 2023 at 5:05 PM Jason Gustafson
> > > > > > >> > > > >> > <ja...@confluent.io.invalid>
> > > > > > >> > > > >> > > wrote:
> > > > > > >> > > > >> > >
> > > > > > >> > > > >> > >> Hi Justine,
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> > >> Thanks for the proposal.
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> > >> I was thinking about the implementation a little
> > > bit.
> > > > > In the
> > > > > > >> > > > current
> > > > > > >> > > > >> > >> proposal, the behavior depends on whether we
> have
> > an
> > > > > old or
> > > > > > >> new
> > > > > > >> > > > >> client.
> > > > > > >> > > > >> > >> For
> > > > > > >> > > > >> > >> old clients, we send `DescribeTransactions` and
> > > verify
> > > > > the
> > > > > > >> > result
> > > > > > >> > > > and
> > > > > > >> > > > >> > for
> > > > > > >> > > > >> > >> new clients, we send `AddPartitionsToTxn`. We
> > might
> > > be
> > > > > able
> > > > > > >> to
> > > > > > >> > > > >> simplify
> > > > > > >> > > > >> > >> the
> > > > > > >> > > > >> > >> implementation if we can use the same request
> > type.
> > > For
> > > > > > >> > example,
> > > > > > >> > > > >> what if
> > > > > > >> > > > >> > >> we
> > > > > > >> > > > >> > >> bump the protocol version for
> `AddPartitionsToTxn`
> > > and
> > > > > add a
> > > > > > >> > > > >> > >> `validateOnly`
> > > > > > >> > > > >> > >> flag? For older versions, we can set
> > > > > `validateOnly=true` so
> > > > > > >> > that
> > > > > > >> > > > the
> > > > > > >> > > > >> > >> request only returns successfully if the
> partition
> > > had
> > > > > > >> already
> > > > > > >> > > been
> > > > > > >> > > > >> > added.
> > > > > > >> > > > >> > >> For new versions, we can set
> `validateOnly=false`
> > > and
> > > > > the
> > > > > > >> > > partition
> > > > > > >> > > > >> will
> > > > > > >> > > > >> > >> be
> > > > > > >> > > > >> > >> added to the transaction. The other slightly
> > > annoying
> > > > > thing
> > > > > > >> > that
> > > > > > >> > > > this
> > > > > > >> > > > >> > >> would
> > > > > > >> > > > >> > >> get around is the need to collect the
> transaction
> > > state
> > > > > for
> > > > > > >> all
> > > > > > >> > > > >> > partitions
> > > > > > >> > > > >> > >> even when we only care about a subset.
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> > >> Some additional improvements to consider:
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> > >> - We can give `AddPartitionsToTxn` better batch
> > > support
> > > > > for
> > > > > > >> > > > >> inter-broker
> > > > > > >> > > > >> > >> usage. Currently we only allow one
> > > `TransactionalId` to
> > > > > be
> > > > > > >> > > > specified,
> > > > > > >> > > > >> > but
> > > > > > >> > > > >> > >> the broker may get some benefit being able to
> > batch
> > > > > across
> > > > > > >> > > multiple
> > > > > > >> > > > >> > >> transactions.
> > > > > > >> > > > >> > >> - Another small improvement is skipping topic
> > > > > authorization
> > > > > > >> > > checks
> > > > > > >> > > > >> for
> > > > > > >> > > > >> > >> `AddPartitionsToTxn` when the request is from a
> > > broker.
> > > > > > >> Perhaps
> > > > > > >> > > we
> > > > > > >> > > > >> can
> > > > > > >> > > > >> > add
> > > > > > >> > > > >> > >> a field for the `LeaderId` or something like
> that
> > > and
> > > > > > >> require
> > > > > > >> > > > CLUSTER
> > > > > > >> > > > >> > >> permission when set.
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> > >> Best,
> > > > > > >> > > > >> > >> Jason
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> > >> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
> > > > > > >> > <jun@confluent.io.invalid
> > > > > > >> > > >
> > > > > > >> > > > >> > wrote:
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> > >> > Hi, Justine,
> > > > > > >> > > > >> > >> >
> > > > > > >> > > > >> > >> > Thanks for the explanation. It makes sense to
> me
> > > now.
> > > > > > >> > > > >> > >> >
> > > > > > >> > > > >> > >> > Jun
> > > > > > >> > > > >> > >> >
> > > > > > >> > > > >> > >> > On Mon, Dec 19, 2022 at 1:42 PM Justine Olshan
> > > > > > >> > > > >> > >> > <jo...@confluent.io.invalid>
> > > > > > >> > > > >> > >> > wrote:
> > > > > > >> > > > >> > >> >
> > > > > > >> > > > >> > >> > > Hi Jun,
> > > > > > >> > > > >> > >> > >
> > > > > > >> > > > >> > >> > > My understanding of the mechanism is that
> when
> > > we
> > > > > get to
> > > > > > >> > the
> > > > > > >> > > > last
> > > > > > >> > > > >> > >> epoch,
> > > > > > >> > > > >> > >> > we
> > > > > > >> > > > >> > >> > > increment to the fencing/last epoch and if
> any
> > > > > further
> > > > > > >> > > requests
> > > > > > >> > > > >> come
> > > > > > >> > > > >> > >> in
> > > > > > >> > > > >> > >> > for
> > > > > > >> > > > >> > >> > > this producer ID they are fenced. Then the
> > > producer
> > > > > > >> gets a
> > > > > > >> > > new
> > > > > > >> > > > ID
> > > > > > >> > > > >> > and
> > > > > > >> > > > >> > >> > > restarts with epoch/sequence 0. The fenced
> > epoch
> > > > > sticks
> > > > > > >> > > around
> > > > > > >> > > > >> for
> > > > > > >> > > > >> > the
> > > > > > >> > > > >> > >> > > duration of producer.id.expiration.ms and
> > > blocks
> > > > > any
> > > > > > >> late
> > > > > > >> > > > >> messages
> > > > > > >> > > > >> > >> > there.
> > > > > > >> > > > >> > >> > > The new ID will get to take advantage of the
> > > > > improved
> > > > > > >> > > semantics
> > > > > > >> > > > >> > around
> > > > > > >> > > > >> > >> > > non-zero start sequences. So I think we are
> > > covered.
> > > > > > >> > > > >> > >> > >
> > > > > > >> > > > >> > >> > > The only potential issue is overloading the
> > > cache,
> > > > > but
> > > > > > >> > > > hopefully
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> > > improvements (lowered
> > producer.id.expiration.ms
> > > )
> > > > > will
> > > > > > >> help
> > > > > > >> > > > with
> > > > > > >> > > > >> > that.
> > > > > > >> > > > >> > >> > Let
> > > > > > >> > > > >> > >> > > me know if you still have concerns.
> > > > > > >> > > > >> > >> > >
> > > > > > >> > > > >> > >> > > Thanks,
> > > > > > >> > > > >> > >> > > Justine
> > > > > > >> > > > >> > >> > >
> > > > > > >> > > > >> > >> > > On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
> > > > > > >> > > > >> <ju...@confluent.io.invalid>
> > > > > > >> > > > >> > >> > wrote:
> > > > > > >> > > > >> > >> > >
> > > > > > >> > > > >> > >> > > > Hi, Justine,
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > > >> > >> > > > Thanks for the explanation.
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > > >> > >> > > > 70. The proposed fencing logic doesn't
> apply
> > > when
> > > > > pid
> > > > > > >> > > > changes,
> > > > > > >> > > > >> is
> > > > > > >> > > > >> > >> that
> > > > > > >> > > > >> > >> > > > right? If so, I am not sure how complete
> we
> > > are
> > > > > > >> > addressing
> > > > > > >> > > > this
> > > > > > >> > > > >> > >> issue
> > > > > > >> > > > >> > >> > if
> > > > > > >> > > > >> > >> > > > the pid changes more frequently.
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > > >> > >> > > > Thanks,
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > > >> > >> > > > Jun
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > > >> > >> > > > On Fri, Dec 16, 2022 at 9:16 AM Justine
> > Olshan
> > > > > > >> > > > >> > >> > > > <jo...@confluent.io.invalid>
> > > > > > >> > > > >> > >> > > > wrote:
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > > >> > >> > > > > Hi Jun,
> > > > > > >> > > > >> > >> > > > >
> > > > > > >> > > > >> > >> > > > > Thanks for replying!
> > > > > > >> > > > >> > >> > > > >
> > > > > > >> > > > >> > >> > > > > 70.We already do the overflow mechanism,
> > so
> > > my
> > > > > > >> change
> > > > > > >> > > would
> > > > > > >> > > > >> just
> > > > > > >> > > > >> > >> make
> > > > > > >> > > > >> > >> > > it
> > > > > > >> > > > >> > >> > > > > happen more often.
> > > > > > >> > > > >> > >> > > > > I was also not suggesting a new field in
> > the
> > > > > log,
> > > > > > >> but
> > > > > > >> > in
> > > > > > >> > > > the
> > > > > > >> > > > >> > >> > response,
> > > > > > >> > > > >> > >> > > > > which would be gated by the client
> > version.
> > > > > Sorry if
> > > > > > >> > > > >> something
> > > > > > >> > > > >> > >> there
> > > > > > >> > > > >> > >> > is
> > > > > > >> > > > >> > >> > > > > unclear. I think we are starting to
> > diverge.
> > > > > > >> > > > >> > >> > > > > The goal of this KIP is to not change to
> > the
> > > > > marker
> > > > > > >> > > format
> > > > > > >> > > > at
> > > > > > >> > > > >> > all.
> > > > > > >> > > > >> > >> > > > >
> > > > > > >> > > > >> > >> > > > > 71. Yes, I guess I was going under the
> > > > > assumption
> > > > > > >> that
> > > > > > >> > > the
> > > > > > >> > > > >> log
> > > > > > >> > > > >> > >> would
> > > > > > >> > > > >> > >> > > just
> > > > > > >> > > > >> > >> > > > > look at its last epoch and treat it as
> the
> > > > > current
> > > > > > >> > > epoch. I
> > > > > > >> > > > >> > >> suppose
> > > > > > >> > > > >> > >> > we
> > > > > > >> > > > >> > >> > > > can
> > > > > > >> > > > >> > >> > > > > have some special logic that if the last
> > > epoch
> > > > > was
> > > > > > >> on a
> > > > > > >> > > > >> marker
> > > > > > >> > > > >> > we
> > > > > > >> > > > >> > >> > > > actually
> > > > > > >> > > > >> > >> > > > > expect the next epoch or something like
> > > that. We
> > > > > > >> just
> > > > > > >> > > need
> > > > > > >> > > > to
> > > > > > >> > > > >> > >> > > distinguish
> > > > > > >> > > > >> > >> > > > > based on whether we had a commit/abort
> > > marker.
> > > > > > >> > > > >> > >> > > > >
> > > > > > >> > > > >> > >> > > > > 72.
> > > > > > >> > > > >> > >> > > > > > if the producer epoch hasn't been
> bumped
> > > on
> > > > > the
> > > > > > >> > > > >> > >> > > > > broker, it seems that the stucked
> message
> > > will
> > > > > fail
> > > > > > >> the
> > > > > > >> > > > >> sequence
> > > > > > >> > > > >> > >> > > > validation
> > > > > > >> > > > >> > >> > > > > and will be ignored. If the producer
> epoch
> > > has
> > > > > been
> > > > > > >> > > bumped,
> > > > > > >> > > > >> we
> > > > > > >> > > > >> > >> ignore
> > > > > > >> > > > >> > >> > > the
> > > > > > >> > > > >> > >> > > > > sequence check and the stuck message
> could
> > > be
> > > > > > >> appended
> > > > > > >> > to
> > > > > > >> > > > the
> > > > > > >> > > > >> > log.
> > > > > > >> > > > >> > >> > So,
> > > > > > >> > > > >> > >> > > is
> > > > > > >> > > > >> > >> > > > > the latter case that we want to guard?
> > > > > > >> > > > >> > >> > > > >
> > > > > > >> > > > >> > >> > > > > I'm not sure I follow that "the message
> > will
> > > > > fail
> > > > > > >> the
> > > > > > >> > > > >> sequence
> > > > > > >> > > > >> > >> > > > validation".
> > > > > > >> > > > >> > >> > > > > In some of these cases, we had an abort
> > > marker
> > > > > (due
> > > > > > >> to
> > > > > > >> > an
> > > > > > >> > > > >> error)
> > > > > > >> > > > >> > >> and
> > > > > > >> > > > >> > >> > > then
> > > > > > >> > > > >> > >> > > > > the late message comes in with the
> correct
> > > > > sequence
> > > > > > >> > > number.
> > > > > > >> > > > >> This
> > > > > > >> > > > >> > >> is a
> > > > > > >> > > > >> > >> > > > case
> > > > > > >> > > > >> > >> > > > > covered by the KIP.
> > > > > > >> > > > >> > >> > > > > The latter case is actually not
> something
> > > we've
> > > > > > >> > > considered
> > > > > > >> > > > >> > here. I
> > > > > > >> > > > >> > >> > > think
> > > > > > >> > > > >> > >> > > > > generally when we bump the epoch, we are
> > > > > accepting
> > > > > > >> that
> > > > > > >> > > the
> > > > > > >> > > > >> > >> sequence
> > > > > > >> > > > >> > >> > > does
> > > > > > >> > > > >> > >> > > > > not need to be checked anymore. My
> > > > > understanding is
> > > > > > >> > also
> > > > > > >> > > > >> that we
> > > > > > >> > > > >> > >> > don't
> > > > > > >> > > > >> > >> > > > > typically bump epoch mid transaction
> > (based
> > > on a
> > > > > > >> quick
> > > > > > >> > > look
> > > > > > >> > > > >> at
> > > > > > >> > > > >> > the
> > > > > > >> > > > >> > >> > > code)
> > > > > > >> > > > >> > >> > > > > but let me know if that is the case.
> > > > > > >> > > > >> > >> > > > >
> > > > > > >> > > > >> > >> > > > > Thanks,
> > > > > > >> > > > >> > >> > > > > Justine
> > > > > > >> > > > >> > >> > > > >
> > > > > > >> > > > >> > >> > > > > On Thu, Dec 15, 2022 at 12:23 PM Jun Rao
> > > > > > >> > > > >> > <jun@confluent.io.invalid
> > > > > > >> > > > >> > >> >
> > > > > > >> > > > >> > >> > > > wrote:
> > > > > > >> > > > >> > >> > > > >
> > > > > > >> > > > >> > >> > > > > > Hi, Justine,
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> > > > >> > >> > > > > > Thanks for the reply.
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> > > > >> > >> > > > > > 70. Assigning a new pid on int
> overflow
> > > seems
> > > > > a
> > > > > > >> bit
> > > > > > >> > > > hacky.
> > > > > > >> > > > >> If
> > > > > > >> > > > >> > we
> > > > > > >> > > > >> > >> > > need a
> > > > > > >> > > > >> > >> > > > > txn
> > > > > > >> > > > >> > >> > > > > > level id, it will be better to model
> > this
> > > > > > >> explicitly.
> > > > > > >> > > > >> Adding a
> > > > > > >> > > > >> > >> new
> > > > > > >> > > > >> > >> > > > field
> > > > > > >> > > > >> > >> > > > > > would require a bit more work since it
> > > > > requires a
> > > > > > >> new
> > > > > > >> > > txn
> > > > > > >> > > > >> > marker
> > > > > > >> > > > >> > >> > > format
> > > > > > >> > > > >> > >> > > > > in
> > > > > > >> > > > >> > >> > > > > > the log. So, we probably need to guard
> > it
> > > > > with an
> > > > > > >> IBP
> > > > > > >> > > or
> > > > > > >> > > > >> > >> metadata
> > > > > > >> > > > >> > >> > > > version
> > > > > > >> > > > >> > >> > > > > > and document the impact on downgrade
> > once
> > > the
> > > > > new
> > > > > > >> > > format
> > > > > > >> > > > is
> > > > > > >> > > > >> > >> written
> > > > > > >> > > > >> > >> > > to
> > > > > > >> > > > >> > >> > > > > the
> > > > > > >> > > > >> > >> > > > > > log.
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> > > > >> > >> > > > > > 71. Hmm, once the marker is written,
> the
> > > > > partition
> > > > > > >> > will
> > > > > > >> > > > >> expect
> > > > > > >> > > > >> > >> the
> > > > > > >> > > > >> > >> > > next
> > > > > > >> > > > >> > >> > > > > > append to be on the next epoch. Does
> > that
> > > > > cover
> > > > > > >> the
> > > > > > >> > > case
> > > > > > >> > > > >> you
> > > > > > >> > > > >> > >> > > mentioned?
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> > > > >> > >> > > > > > 72. Also, just to be clear on the
> > stucked
> > > > > message
> > > > > > >> > issue
> > > > > > >> > > > >> > >> described
> > > > > > >> > > > >> > >> > in
> > > > > > >> > > > >> > >> > > > the
> > > > > > >> > > > >> > >> > > > > > motivation. With EoS, we also validate
> > the
> > > > > > >> sequence
> > > > > > >> > id
> > > > > > >> > > > for
> > > > > > >> > > > >> > >> > > idempotency.
> > > > > > >> > > > >> > >> > > > > So,
> > > > > > >> > > > >> > >> > > > > > with the current logic, if the
> producer
> > > epoch
> > > > > > >> hasn't
> > > > > > >> > > been
> > > > > > >> > > > >> > >> bumped on
> > > > > > >> > > > >> > >> > > the
> > > > > > >> > > > >> > >> > > > > > broker, it seems that the stucked
> > message
> > > will
> > > > > > >> fail
> > > > > > >> > the
> > > > > > >> > > > >> > sequence
> > > > > > >> > > > >> > >> > > > > validation
> > > > > > >> > > > >> > >> > > > > > and will be ignored. If the producer
> > > epoch has
> > > > > > >> been
> > > > > > >> > > > >> bumped, we
> > > > > > >> > > > >> > >> > ignore
> > > > > > >> > > > >> > >> > > > the
> > > > > > >> > > > >> > >> > > > > > sequence check and the stuck message
> > > could be
> > > > > > >> > appended
> > > > > > >> > > to
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> log.
> > > > > > >> > > > >> > >> > > So,
> > > > > > >> > > > >> > >> > > > is
> > > > > > >> > > > >> > >> > > > > > the latter case that we want to guard?
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> > > > >> > >> > > > > > Thanks,
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> > > > >> > >> > > > > > Jun
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> > > > >> > >> > > > > > On Wed, Dec 14, 2022 at 10:44 AM
> Justine
> > > > > Olshan
> > > > > > >> > > > >> > >> > > > > > <jo...@confluent.io.invalid> wrote:
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> > > > >> > >> > > > > > > Matthias — thanks again for taking
> > time
> > > to
> > > > > look
> > > > > > >> a
> > > > > > >> > > this.
> > > > > > >> > > > >> You
> > > > > > >> > > > >> > >> said:
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > My proposal was only focusing to
> > avoid
> > > > > > >> dangling
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > transactions if records are added
> > > without
> > > > > > >> > registered
> > > > > > >> > > > >> > >> partition.
> > > > > > >> > > > >> > >> > --
> > > > > > >> > > > >> > >> > > > > Maybe
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > you can add a few more details to
> the
> > > KIP
> > > > > about
> > > > > > >> > this
> > > > > > >> > > > >> > scenario
> > > > > > >> > > > >> > >> for
> > > > > > >> > > > >> > >> > > > > better
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > documentation purpose?
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > I'm not sure I understand what you
> > mean
> > > > > here.
> > > > > > >> The
> > > > > > >> > > > >> motivation
> > > > > > >> > > > >> > >> > > section
> > > > > > >> > > > >> > >> > > > > > > describes two scenarios about how
> the
> > > record
> > > > > > >> can be
> > > > > > >> > > > added
> > > > > > >> > > > >> > >> > without a
> > > > > > >> > > > >> > >> > > > > > > registered partition:
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > 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.
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > Another way hanging transactions
> can
> > > > > occur is
> > > > > > >> > that
> > > > > > >> > > a
> > > > > > >> > > > >> > client
> > > > > > >> > > > >> > >> is
> > > > > > >> > > > >> > >> > > > buggy
> > > > > > >> > > > >> > >> > > > > > and
> > > > > > >> > > > >> > >> > > > > > > may somehow try to write to a
> > partition
> > > > > before
> > > > > > >> it
> > > > > > >> > > adds
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> > > partition
> > > > > > >> > > > >> > >> > > > to
> > > > > > >> > > > >> > >> > > > > > the
> > > > > > >> > > > >> > >> > > > > > > transaction.
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > For the first example of this would
> it
> > > be
> > > > > > >> helpful
> > > > > > >> > to
> > > > > > >> > > > say
> > > > > > >> > > > >> > that
> > > > > > >> > > > >> > >> > this
> > > > > > >> > > > >> > >> > > > > > message
> > > > > > >> > > > >> > >> > > > > > > comes in after the abort, but before
> > the
> > > > > > >> partition
> > > > > > >> > is
> > > > > > >> > > > >> added
> > > > > > >> > > > >> > to
> > > > > > >> > > > >> > >> > the
> > > > > > >> > > > >> > >> > > > next
> > > > > > >> > > > >> > >> > > > > > > transaction so it becomes "hanging."
> > > > > Perhaps the
> > > > > > >> > next
> > > > > > >> > > > >> > sentence
> > > > > > >> > > > >> > >> > > > > describing
> > > > > > >> > > > >> > >> > > > > > > the message becoming part of the
> next
> > > > > > >> transaction
> > > > > > >> > (a
> > > > > > >> > > > >> > different
> > > > > > >> > > > >> > >> > > case)
> > > > > > >> > > > >> > >> > > > > was
> > > > > > >> > > > >> > >> > > > > > > not properly differentiated.
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > Jun — thanks for reading the KIP.
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > 70. The int typing was a concern.
> > > Currently
> > > > > we
> > > > > > >> > have a
> > > > > > >> > > > >> > >> mechanism
> > > > > > >> > > > >> > >> > in
> > > > > > >> > > > >> > >> > > > > place
> > > > > > >> > > > >> > >> > > > > > to
> > > > > > >> > > > >> > >> > > > > > > fence the final epoch when the epoch
> > is
> > > > > about to
> > > > > > >> > > > overflow
> > > > > > >> > > > >> > and
> > > > > > >> > > > >> > >> > > assign
> > > > > > >> > > > >> > >> > > > a
> > > > > > >> > > > >> > >> > > > > > new
> > > > > > >> > > > >> > >> > > > > > > producer ID with epoch 0. Of course,
> > > this
> > > > > is a
> > > > > > >> bit
> > > > > > >> > > > tricky
> > > > > > >> > > > >> > >> when it
> > > > > > >> > > > >> > >> > > > comes
> > > > > > >> > > > >> > >> > > > > > to
> > > > > > >> > > > >> > >> > > > > > > the response back to the client.
> > > > > > >> > > > >> > >> > > > > > > Making this a long could be another
> > > option,
> > > > > but
> > > > > > >> I
> > > > > > >> > > > wonder
> > > > > > >> > > > >> are
> > > > > > >> > > > >> > >> > there
> > > > > > >> > > > >> > >> > > > any
> > > > > > >> > > > >> > >> > > > > > > implications on changing this field
> if
> > > the
> > > > > > >> epoch is
> > > > > > >> > > > >> > persisted
> > > > > > >> > > > >> > >> to
> > > > > > >> > > > >> > >> > > > disk?
> > > > > > >> > > > >> > >> > > > > > I'd
> > > > > > >> > > > >> > >> > > > > > > need to check the usages.
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > 71.This was something Matthias asked
> > > about
> > > > > as
> > > > > > >> > well. I
> > > > > > >> > > > was
> > > > > > >> > > > >> > >> > > > considering a
> > > > > > >> > > > >> > >> > > > > > > possible edge case where a produce
> > > request
> > > > > from
> > > > > > >> a
> > > > > > >> > new
> > > > > > >> > > > >> > >> transaction
> > > > > > >> > > > >> > >> > > > > somehow
> > > > > > >> > > > >> > >> > > > > > > gets sent right after the marker is
> > > > > written, but
> > > > > > >> > > before
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> > > producer
> > > > > > >> > > > >> > >> > > > is
> > > > > > >> > > > >> > >> > > > > > > alerted of the newly bumped epoch.
> In
> > > this
> > > > > > >> case, we
> > > > > > >> > > may
> > > > > > >> > > > >> > >> include
> > > > > > >> > > > >> > >> > > this
> > > > > > >> > > > >> > >> > > > > > record
> > > > > > >> > > > >> > >> > > > > > > when we don't want to. I suppose we
> > > could
> > > > > try
> > > > > > >> to do
> > > > > > >> > > > >> > something
> > > > > > >> > > > >> > >> > > client
> > > > > > >> > > > >> > >> > > > > side
> > > > > > >> > > > >> > >> > > > > > > to bump the epoch after sending an
> > > endTxn as
> > > > > > >> well
> > > > > > >> > in
> > > > > > >> > > > this
> > > > > > >> > > > >> > >> > scenario
> > > > > > >> > > > >> > >> > > —
> > > > > > >> > > > >> > >> > > > > but
> > > > > > >> > > > >> > >> > > > > > I
> > > > > > >> > > > >> > >> > > > > > > wonder how it would work when the
> > > server is
> > > > > > >> > aborting
> > > > > > >> > > > >> based
> > > > > > >> > > > >> > on
> > > > > > >> > > > >> > >> a
> > > > > > >> > > > >> > >> > > > > > server-side
> > > > > > >> > > > >> > >> > > > > > > error. I could also be missing
> > > something and
> > > > > > >> this
> > > > > > >> > > > >> scenario
> > > > > > >> > > > >> > is
> > > > > > >> > > > >> > >> > > > actually
> > > > > > >> > > > >> > >> > > > > > not
> > > > > > >> > > > >> > >> > > > > > > possible.
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > Thanks again to everyone reading and
> > > > > commenting.
> > > > > > >> > Let
> > > > > > >> > > me
> > > > > > >> > > > >> know
> > > > > > >> > > > >> > >> > about
> > > > > > >> > > > >> > >> > > > any
> > > > > > >> > > > >> > >> > > > > > > further questions or comments.
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > Justine
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > On Wed, Dec 14, 2022 at 9:41 AM Jun
> > Rao
> > > > > > >> > > > >> > >> <jun@confluent.io.invalid
> > > > > > >> > > > >> > >> > >
> > > > > > >> > > > >> > >> > > > > > wrote:
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > Hi, Justine,
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > Thanks for the KIP. A couple of
> > > comments.
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > 70. Currently, the producer epoch
> is
> > > an
> > > > > int.
> > > > > > >> I am
> > > > > > >> > > not
> > > > > > >> > > > >> sure
> > > > > > >> > > > >> > >> if
> > > > > > >> > > > >> > >> > > it's
> > > > > > >> > > > >> > >> > > > > > enough
> > > > > > >> > > > >> > >> > > > > > > > to accommodate all transactions in
> > the
> > > > > > >> lifetime
> > > > > > >> > of
> > > > > > >> > > a
> > > > > > >> > > > >> > >> producer.
> > > > > > >> > > > >> > >> > > > Should
> > > > > > >> > > > >> > >> > > > > > we
> > > > > > >> > > > >> > >> > > > > > > > change that to a long or add a new
> > > long
> > > > > field
> > > > > > >> > like
> > > > > > >> > > > >> txnId?
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > 71. "it will write the prepare
> > commit
> > > > > message
> > > > > > >> > with
> > > > > > >> > > a
> > > > > > >> > > > >> > bumped
> > > > > > >> > > > >> > >> > epoch
> > > > > > >> > > > >> > >> > > > and
> > > > > > >> > > > >> > >> > > > > > > send
> > > > > > >> > > > >> > >> > > > > > > > WriteTxnMarkerRequests with the
> > bumped
> > > > > epoch."
> > > > > > >> > Hmm,
> > > > > > >> > > > the
> > > > > > >> > > > >> > >> epoch
> > > > > > >> > > > >> > >> > is
> > > > > > >> > > > >> > >> > > > > > > associated
> > > > > > >> > > > >> > >> > > > > > > > with the current txn right? So, it
> > > seems
> > > > > > >> weird to
> > > > > > >> > > > >> write a
> > > > > > >> > > > >> > >> > commit
> > > > > > >> > > > >> > >> > > > > > message
> > > > > > >> > > > >> > >> > > > > > > > with a bumped epoch. Should we
> only
> > > bump
> > > > > up
> > > > > > >> the
> > > > > > >> > > epoch
> > > > > > >> > > > >> in
> > > > > > >> > > > >> > >> > > > > EndTxnResponse
> > > > > > >> > > > >> > >> > > > > > > and
> > > > > > >> > > > >> > >> > > > > > > > rename the field to sth like
> > > > > > >> nextProducerEpoch?
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > Thanks,
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > Jun
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > On Mon, Dec 12, 2022 at 8:54 PM
> > > Matthias
> > > > > J.
> > > > > > >> Sax <
> > > > > > >> > > > >> > >> > > mjsax@apache.org>
> > > > > > >> > > > >> > >> > > > > > > wrote:
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > Thanks for the background.
> > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > 20/30: SGTM. My proposal was
> only
> > > > > focusing
> > > > > > >> to
> > > > > > >> > > avoid
> > > > > > >> > > > >> > >> dangling
> > > > > > >> > > > >> > >> > > > > > > > > transactions if records are
> added
> > > > > without
> > > > > > >> > > > registered
> > > > > > >> > > > >> > >> > partition.
> > > > > > >> > > > >> > >> > > > --
> > > > > > >> > > > >> > >> > > > > > > Maybe
> > > > > > >> > > > >> > >> > > > > > > > > you can add a few more details
> to
> > > the
> > > > > KIP
> > > > > > >> about
> > > > > > >> > > > this
> > > > > > >> > > > >> > >> scenario
> > > > > > >> > > > >> > >> > > for
> > > > > > >> > > > >> > >> > > > > > > better
> > > > > > >> > > > >> > >> > > > > > > > > documentation purpose?
> > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > 40: I think you hit a fair point
> > > about
> > > > > race
> > > > > > >> > > > >> conditions
> > > > > > >> > > > >> > or
> > > > > > >> > > > >> > >> > > client
> > > > > > >> > > > >> > >> > > > > bugs
> > > > > > >> > > > >> > >> > > > > > > > > (incorrectly not bumping the
> > > epoch). The
> > > > > > >> > > > >> > >> complexity/confusion
> > > > > > >> > > > >> > >> > > for
> > > > > > >> > > > >> > >> > > > > > using
> > > > > > >> > > > >> > >> > > > > > > > > the bumped epoch I see, is
> mainly
> > > for
> > > > > > >> internal
> > > > > > >> > > > >> > debugging,
> > > > > > >> > > > >> > >> ie,
> > > > > > >> > > > >> > >> > > > > > > inspecting
> > > > > > >> > > > >> > >> > > > > > > > > log segment dumps -- it seems
> > > harder to
> > > > > > >> reason
> > > > > > >> > > > about
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> > system
> > > > > > >> > > > >> > >> > > > for
> > > > > > >> > > > >> > >> > > > > > us
> > > > > > >> > > > >> > >> > > > > > > > > humans. But if we get better
> > > > > guarantees, it
> > > > > > >> > would
> > > > > > >> > > > be
> > > > > > >> > > > >> > >> worth to
> > > > > > >> > > > >> > >> > > use
> > > > > > >> > > > >> > >> > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > bumped epoch.
> > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > 60: as I mentioned already, I
> > don't
> > > > > know the
> > > > > > >> > > broker
> > > > > > >> > > > >> > >> internals
> > > > > > >> > > > >> > >> > > to
> > > > > > >> > > > >> > >> > > > > > > provide
> > > > > > >> > > > >> > >> > > > > > > > > more input. So if nobody else
> > chimes
> > > > > in, we
> > > > > > >> > > should
> > > > > > >> > > > >> just
> > > > > > >> > > > >> > >> move
> > > > > > >> > > > >> > >> > > > > forward
> > > > > > >> > > > >> > >> > > > > > > > > with your proposal.
> > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > -Matthias
> > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > On 12/6/22 4:22 PM, Justine
> Olshan
> > > > > wrote:
> > > > > > >> > > > >> > >> > > > > > > > > > Hi all,
> > > > > > >> > > > >> > >> > > > > > > > > > After Artem's questions about
> > > error
> > > > > > >> behavior,
> > > > > > >> > > > I've
> > > > > > >> > > > >> > >> > > re-evaluated
> > > > > > >> > > > >> > >> > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > > unknown producer ID exception
> > and
> > > had
> > > > > some
> > > > > > >> > > > >> discussions
> > > > > > >> > > > >> > >> > > offline.
> > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > > I think generally it makes
> sense
> > > to
> > > > > > >> simplify
> > > > > > >> > > > error
> > > > > > >> > > > >> > >> handling
> > > > > > >> > > > >> > >> > > in
> > > > > > >> > > > >> > >> > > > > > cases
> > > > > > >> > > > >> > >> > > > > > > > like
> > > > > > >> > > > >> > >> > > > > > > > > > this and the
> UNKNOWN_PRODUCER_ID
> > > error
> > > > > > >> has a
> > > > > > >> > > > pretty
> > > > > > >> > > > >> > long
> > > > > > >> > > > >> > >> > and
> > > > > > >> > > > >> > >> > > > > > > > complicated
> > > > > > >> > > > >> > >> > > > > > > > > > history. Because of this, I
> > > propose
> > > > > > >> adding a
> > > > > > >> > > new
> > > > > > >> > > > >> error
> > > > > > >> > > > >> > >> code
> > > > > > >> > > > >> > >> > > > > > > > > ABORTABLE_ERROR
> > > > > > >> > > > >> > >> > > > > > > > > > that when encountered by new
> > > clients
> > > > > > >> (gated
> > > > > > >> > by
> > > > > > >> > > > the
> > > > > > >> > > > >> > >> produce
> > > > > > >> > > > >> > >> > > > > request
> > > > > > >> > > > >> > >> > > > > > > > > version)
> > > > > > >> > > > >> > >> > > > > > > > > > will simply abort the
> > transaction.
> > > > > This
> > > > > > >> > allows
> > > > > > >> > > > the
> > > > > > >> > > > >> > >> server
> > > > > > >> > > > >> > >> > to
> > > > > > >> > > > >> > >> > > > have
> > > > > > >> > > > >> > >> > > > > > > some
> > > > > > >> > > > >> > >> > > > > > > > > say
> > > > > > >> > > > >> > >> > > > > > > > > > in whether the client aborts
> and
> > > makes
> > > > > > >> > handling
> > > > > > >> > > > >> much
> > > > > > >> > > > >> > >> > simpler.
> > > > > > >> > > > >> > >> > > > In
> > > > > > >> > > > >> > >> > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > > future, we can also use this
> > > error in
> > > > > > >> other
> > > > > > >> > > > >> situations
> > > > > > >> > > > >> > >> > where
> > > > > > >> > > > >> > >> > > we
> > > > > > >> > > > >> > >> > > > > > want
> > > > > > >> > > > >> > >> > > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > > > abort the transactions. We can
> > > even
> > > > > use on
> > > > > > >> > > other
> > > > > > >> > > > >> apis.
> > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > > I've added this to the KIP.
> Let
> > me
> > > > > know if
> > > > > > >> > > there
> > > > > > >> > > > >> are
> > > > > > >> > > > >> > any
> > > > > > >> > > > >> > >> > > > > questions
> > > > > > >> > > > >> > >> > > > > > or
> > > > > > >> > > > >> > >> > > > > > > > > > issues.
> > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > > Justine
> > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > > On Fri, Dec 2, 2022 at 10:22
> AM
> > > > > Justine
> > > > > > >> > Olshan
> > > > > > >> > > <
> > > > > > >> > > > >> > >> > > > > > jolshan@confluent.io
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > wrote:
> > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > >> Hey Matthias,
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >> 20/30 — Maybe I also didn't
> > > express
> > > > > > >> myself
> > > > > > >> > > > >> clearly.
> > > > > > >> > > > >> > For
> > > > > > >> > > > >> > >> > > older
> > > > > > >> > > > >> > >> > > > > > > clients
> > > > > > >> > > > >> > >> > > > > > > > we
> > > > > > >> > > > >> > >> > > > > > > > > >> don't have a way to
> distinguish
> > > > > between a
> > > > > > >> > > > previous
> > > > > > >> > > > >> > and
> > > > > > >> > > > >> > >> the
> > > > > > >> > > > >> > >> > > > > current
> > > > > > >> > > > >> > >> > > > > > > > > >> transaction since we don't
> have
> > > the
> > > > > epoch
> > > > > > >> > > bump.
> > > > > > >> > > > >> This
> > > > > > >> > > > >> > >> means
> > > > > > >> > > > >> > >> > > > that
> > > > > > >> > > > >> > >> > > > > a
> > > > > > >> > > > >> > >> > > > > > > late
> > > > > > >> > > > >> > >> > > > > > > > > >> message from the previous
> > > transaction
> > > > > > >> may be
> > > > > > >> > > > >> added to
> > > > > > >> > > > >> > >> the
> > > > > > >> > > > >> > >> > > new
> > > > > > >> > > > >> > >> > > > > one.
> > > > > > >> > > > >> > >> > > > > > > > With
> > > > > > >> > > > >> > >> > > > > > > > > >> older clients — we can't
> > > guarantee
> > > > > this
> > > > > > >> > won't
> > > > > > >> > > > >> happen
> > > > > > >> > > > >> > >> if we
> > > > > > >> > > > >> > >> > > > > already
> > > > > > >> > > > >> > >> > > > > > > > sent
> > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call (why
> we
> > > make
> > > > > > >> changes
> > > > > > >> > > for
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> newer
> > > > > > >> > > > >> > >> > > > > client)
> > > > > > >> > > > >> > >> > > > > > > but
> > > > > > >> > > > >> > >> > > > > > > > > we
> > > > > > >> > > > >> > >> > > > > > > > > >> can at least gate some by
> > > ensuring
> > > > > that
> > > > > > >> the
> > > > > > >> > > > >> partition
> > > > > > >> > > > >> > >> has
> > > > > > >> > > > >> > >> > > been
> > > > > > >> > > > >> > >> > > > > > added
> > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >> transaction. The rationale
> here
> > > is
> > > > > that
> > > > > > >> > there
> > > > > > >> > > > are
> > > > > > >> > > > >> > >> likely
> > > > > > >> > > > >> > >> > > LESS
> > > > > > >> > > > >> > >> > > > > late
> > > > > > >> > > > >> > >> > > > > > > > > arrivals
> > > > > > >> > > > >> > >> > > > > > > > > >> as time goes on, so hopefully
> > > most
> > > > > late
> > > > > > >> > > arrivals
> > > > > > >> > > > >> will
> > > > > > >> > > > >> > >> come
> > > > > > >> > > > >> > >> > > in
> > > > > > >> > > > >> > >> > > > > > BEFORE
> > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call.
> Those
> > > that
> > > > > > >> arrive
> > > > > > >> > > > before
> > > > > > >> > > > >> > will
> > > > > > >> > > > >> > >> be
> > > > > > >> > > > >> > >> > > > > properly
> > > > > > >> > > > >> > >> > > > > > > > gated
> > > > > > >> > > > >> > >> > > > > > > > > >> with the describeTransactions
> > > > > approach.
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >> If we take the approach you
> > > > > suggested,
> > > > > > >> ANY
> > > > > > >> > > late
> > > > > > >> > > > >> > arrival
> > > > > > >> > > > >> > >> > > from a
> > > > > > >> > > > >> > >> > > > > > > > previous
> > > > > > >> > > > >> > >> > > > > > > > > >> transaction will be added.
> And
> > we
> > > > > don't
> > > > > > >> want
> > > > > > >> > > > >> that. I
> > > > > > >> > > > >> > >> also
> > > > > > >> > > > >> > >> > > > don't
> > > > > > >> > > > >> > >> > > > > > see
> > > > > > >> > > > >> > >> > > > > > > > any
> > > > > > >> > > > >> > >> > > > > > > > > >> benefit in sending
> > > addPartitionsToTxn
> > > > > > >> over
> > > > > > >> > the
> > > > > > >> > > > >> > >> > describeTxns
> > > > > > >> > > > >> > >> > > > > call.
> > > > > > >> > > > >> > >> > > > > > > They
> > > > > > >> > > > >> > >> > > > > > > > > will
> > > > > > >> > > > >> > >> > > > > > > > > >> both be one extra RPC to the
> > Txn
> > > > > > >> > coordinator.
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >> To be clear — newer clients
> > will
> > > use
> > > > > > >> > > > >> > addPartitionsToTxn
> > > > > > >> > > > >> > >> > > > instead
> > > > > > >> > > > >> > >> > > > > of
> > > > > > >> > > > >> > >> > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >> DescribeTxns.
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >> 40)
> > > > > > >> > > > >> > >> > > > > > > > > >> My concern is that if we have
> > > some
> > > > > delay
> > > > > > >> in
> > > > > > >> > > the
> > > > > > >> > > > >> > client
> > > > > > >> > > > >> > >> to
> > > > > > >> > > > >> > >> > > bump
> > > > > > >> > > > >> > >> > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > epoch,
> > > > > > >> > > > >> > >> > > > > > > > > >> it could continue to send
> epoch
> > > 73
> > > > > and
> > > > > > >> those
> > > > > > >> > > > >> records
> > > > > > >> > > > >> > >> would
> > > > > > >> > > > >> > >> > > not
> > > > > > >> > > > >> > >> > > > > be
> > > > > > >> > > > >> > >> > > > > > > > > fenced.
> > > > > > >> > > > >> > >> > > > > > > > > >> Perhaps this is not an issue
> if
> > > we
> > > > > don't
> > > > > > >> > allow
> > > > > > >> > > > the
> > > > > > >> > > > >> > next
> > > > > > >> > > > >> > >> > > > produce
> > > > > > >> > > > >> > >> > > > > to
> > > > > > >> > > > >> > >> > > > > > > go
> > > > > > >> > > > >> > >> > > > > > > > > >> through before the EndTxn
> > request
> > > > > > >> returns.
> > > > > > >> > I'm
> > > > > > >> > > > >> also
> > > > > > >> > > > >> > >> > thinking
> > > > > > >> > > > >> > >> > > > > about
> > > > > > >> > > > >> > >> > > > > > > > > cases of
> > > > > > >> > > > >> > >> > > > > > > > > >> failure. I will need to think
> > on
> > > > > this a
> > > > > > >> bit.
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >> I wasn't sure if it was that
> > > > > confusing.
> > > > > > >> But
> > > > > > >> > if
> > > > > > >> > > > we
> > > > > > >> > > > >> > >> think it
> > > > > > >> > > > >> > >> > > is,
> > > > > > >> > > > >> > >> > > > > we
> > > > > > >> > > > >> > >> > > > > > > can
> > > > > > >> > > > >> > >> > > > > > > > > >> investigate other ways.
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >> 60)
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >> I'm not sure these are the
> same
> > > > > > >> purgatories
> > > > > > >> > > > since
> > > > > > >> > > > >> one
> > > > > > >> > > > >> > >> is a
> > > > > > >> > > > >> > >> > > > > produce
> > > > > > >> > > > >> > >> > > > > > > > > >> purgatory (I was planning on
> > > using a
> > > > > > >> > callback
> > > > > > >> > > > >> rather
> > > > > > >> > > > >> > >> than
> > > > > > >> > > > >> > >> > > > > > purgatory)
> > > > > > >> > > > >> > >> > > > > > > > and
> > > > > > >> > > > >> > >> > > > > > > > > >> the other is simply a request
> > to
> > > > > append
> > > > > > >> to
> > > > > > >> > the
> > > > > > >> > > > >> log.
> > > > > > >> > > > >> > Not
> > > > > > >> > > > >> > >> > sure
> > > > > > >> > > > >> > >> > > > we
> > > > > > >> > > > >> > >> > > > > > have
> > > > > > >> > > > >> > >> > > > > > > > any
> > > > > > >> > > > >> > >> > > > > > > > > >> structure here for ordering,
> > but
> > > my
> > > > > > >> > > > understanding
> > > > > > >> > > > >> is
> > > > > > >> > > > >> > >> that
> > > > > > >> > > > >> > >> > > the
> > > > > > >> > > > >> > >> > > > > > broker
> > > > > > >> > > > >> > >> > > > > > > > > could
> > > > > > >> > > > >> > >> > > > > > > > > >> handle the write request
> before
> > > it
> > > > > hears
> > > > > > >> > back
> > > > > > >> > > > from
> > > > > > >> > > > >> > the
> > > > > > >> > > > >> > >> Txn
> > > > > > >> > > > >> > >> > > > > > > > Coordinator.
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >> Let me know if I
> misunderstood
> > > > > something
> > > > > > >> or
> > > > > > >> > > > >> something
> > > > > > >> > > > >> > >> was
> > > > > > >> > > > >> > >> > > > > unclear.
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >> Justine
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >> On Thu, Dec 1, 2022 at 12:15
> PM
> > > > > Matthias
> > > > > > >> J.
> > > > > > >> > > Sax
> > > > > > >> > > > <
> > > > > > >> > > > >> > >> > > > > mjsax@apache.org
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > wrote:
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >>> Thanks for the details
> > Justine!
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>> 20)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>> The client side change for
> 2
> > is
> > > > > > >> removing
> > > > > > >> > the
> > > > > > >> > > > >> > >> > addPartitions
> > > > > > >> > > > >> > >> > > > to
> > > > > > >> > > > >> > >> > > > > > > > > >>> transaction
> > > > > > >> > > > >> > >> > > > > > > > > >>>> call. We don't need to make
> > > this
> > > > > from
> > > > > > >> the
> > > > > > >> > > > >> producer
> > > > > > >> > > > >> > to
> > > > > > >> > > > >> > >> > the
> > > > > > >> > > > >> > >> > > > txn
> > > > > > >> > > > >> > >> > > > > > > > > >>> coordinator,
> > > > > > >> > > > >> > >> > > > > > > > > >>>> only server side.
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>> I think I did not express
> > myself
> > > > > > >> clearly. I
> > > > > > >> > > > >> > understand
> > > > > > >> > > > >> > >> > that
> > > > > > >> > > > >> > >> > > > we
> > > > > > >> > > > >> > >> > > > > > can
> > > > > > >> > > > >> > >> > > > > > > > (and
> > > > > > >> > > > >> > >> > > > > > > > > >>> should) change the producer
> to
> > > not
> > > > > send
> > > > > > >> the
> > > > > > >> > > > >> > >> > `addPartitions`
> > > > > > >> > > > >> > >> > > > > > request
> > > > > > >> > > > >> > >> > > > > > > > any
> > > > > > >> > > > >> > >> > > > > > > > > >>> longer. But I don't thinks
> > it's
> > > > > > >> requirement
> > > > > > >> > > to
> > > > > > >> > > > >> > change
> > > > > > >> > > > >> > >> the
> > > > > > >> > > > >> > >> > > > > broker?
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>> What I am trying to say is:
> > as a
> > > > > > >> safe-guard
> > > > > > >> > > and
> > > > > > >> > > > >> > >> > improvement
> > > > > > >> > > > >> > >> > > > for
> > > > > > >> > > > >> > >> > > > > > > older
> > > > > > >> > > > >> > >> > > > > > > > > >>> producers, the partition
> > leader
> > > can
> > > > > just
> > > > > > >> > send
> > > > > > >> > > > the
> > > > > > >> > > > >> > >> > > > > `addPartitions`
> > > > > > >> > > > >> > >> > > > > > > > > >>> request to the
> TX-coordinator
> > > in any
> > > > > > >> case
> > > > > > >> > --
> > > > > > >> > > if
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> old
> > > > > > >> > > > >> > >> > > > > producer
> > > > > > >> > > > >> > >> > > > > > > > > >>> correctly did send the
> > > > > `addPartition`
> > > > > > >> > request
> > > > > > >> > > > to
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> > > > > > TX-coordinator
> > > > > > >> > > > >> > >> > > > > > > > > >>> already, the TX-coordinator
> > can
> > > just
> > > > > > >> > "ignore"
> > > > > > >> > > > is
> > > > > > >> > > > >> as
> > > > > > >> > > > >> > >> > > > idempotent.
> > > > > > >> > > > >> > >> > > > > > > > > However,
> > > > > > >> > > > >> > >> > > > > > > > > >>> if the old producer has a
> bug
> > > and
> > > > > did
> > > > > > >> > forget
> > > > > > >> > > to
> > > > > > >> > > > >> sent
> > > > > > >> > > > >> > >> the
> > > > > > >> > > > >> > >> > > > > > > > `addPartition`
> > > > > > >> > > > >> > >> > > > > > > > > >>> request, we would now ensure
> > > that
> > > > > the
> > > > > > >> > > partition
> > > > > > >> > > > >> is
> > > > > > >> > > > >> > >> indeed
> > > > > > >> > > > >> > >> > > > added
> > > > > > >> > > > >> > >> > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> TX and thus fix a potential
> > > > > producer bug
> > > > > > >> > > (even
> > > > > > >> > > > >> if we
> > > > > > >> > > > >> > >> > don't
> > > > > > >> > > > >> > >> > > > get
> > > > > > >> > > > >> > >> > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> fencing via the bump epoch).
> > --
> > > It
> > > > > > >> seems to
> > > > > > >> > > be
> > > > > > >> > > > a
> > > > > > >> > > > >> > good
> > > > > > >> > > > >> > >> > > > > > improvement?
> > > > > > >> > > > >> > >> > > > > > > Or
> > > > > > >> > > > >> > >> > > > > > > > > is
> > > > > > >> > > > >> > >> > > > > > > > > >>> there a reason to not do
> this?
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>> 30)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>> Transaction is ongoing =
> > > partition
> > > > > was
> > > > > > >> > added
> > > > > > >> > > > to
> > > > > > >> > > > >> > >> > > transaction
> > > > > > >> > > > >> > >> > > > > via
> > > > > > >> > > > >> > >> > > > > > > > > >>>> addPartitionsToTxn. We
> check
> > > this
> > > > > with
> > > > > > >> the
> > > > > > >> > > > >> > >> > > > > DescribeTransactions
> > > > > > >> > > > >> > >> > > > > > > > call.
> > > > > > >> > > > >> > >> > > > > > > > > >>> Let
> > > > > > >> > > > >> > >> > > > > > > > > >>>> me know if this wasn't
> > > sufficiently
> > > > > > >> > > explained
> > > > > > >> > > > >> here:
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>> If we do what I propose in
> > > (20), we
> > > > > > >> don't
> > > > > > >> > > > really
> > > > > > >> > > > >> > need
> > > > > > >> > > > >> > >> to
> > > > > > >> > > > >> > >> > > make
> > > > > > >> > > > >> > >> > > > > > this
> > > > > > >> > > > >> > >> > > > > > > > > >>> `DescribeTransaction` call,
> as
> > > the
> > > > > > >> > partition
> > > > > > >> > > > >> leader
> > > > > > >> > > > >> > >> adds
> > > > > > >> > > > >> > >> > > the
> > > > > > >> > > > >> > >> > > > > > > > partition
> > > > > > >> > > > >> > >> > > > > > > > > >>> for older clients and we get
> > > this
> > > > > check
> > > > > > >> for
> > > > > > >> > > > free.
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>> 40)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>> The idea here is that if
> any
> > > > > messages
> > > > > > >> > > somehow
> > > > > > >> > > > >> come
> > > > > > >> > > > >> > in
> > > > > > >> > > > >> > >> > > before
> > > > > > >> > > > >> > >> > > > > we
> > > > > > >> > > > >> > >> > > > > > > get
> > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> new
> > > > > > >> > > > >> > >> > > > > > > > > >>>> epoch to the producer, they
> > > will be
> > > > > > >> > fenced.
> > > > > > >> > > > >> > However,
> > > > > > >> > > > >> > >> if
> > > > > > >> > > > >> > >> > we
> > > > > > >> > > > >> > >> > > > > don't
> > > > > > >> > > > >> > >> > > > > > > > think
> > > > > > >> > > > >> > >> > > > > > > > > >>> this
> > > > > > >> > > > >> > >> > > > > > > > > >>>> is necessary, it can be
> > > discussed
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>> I agree that we should have
> > > epoch
> > > > > > >> fencing.
> > > > > > >> > My
> > > > > > >> > > > >> > >> question is
> > > > > > >> > > > >> > >> > > > > > > different:
> > > > > > >> > > > >> > >> > > > > > > > > >>> Assume we are at epoch 73,
> and
> > > we
> > > > > have
> > > > > > >> an
> > > > > > >> > > > ongoing
> > > > > > >> > > > >> > >> > > > transaction,
> > > > > > >> > > > >> > >> > > > > > that
> > > > > > >> > > > >> > >> > > > > > > > is
> > > > > > >> > > > >> > >> > > > > > > > > >>> committed. It seems natural
> to
> > > > > write the
> > > > > > >> > > > "prepare
> > > > > > >> > > > >> > >> commit"
> > > > > > >> > > > >> > >> > > > > marker
> > > > > > >> > > > >> > >> > > > > > > and
> > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` both
> > with
> > > > > epoch
> > > > > > >> 73,
> > > > > > >> > > too,
> > > > > > >> > > > >> as
> > > > > > >> > > > >> > it
> > > > > > >> > > > >> > >> > > belongs
> > > > > > >> > > > >> > >> > > > > to
> > > > > > >> > > > >> > >> > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> current transaction. Of
> > course,
> > > we
> > > > > now
> > > > > > >> also
> > > > > > >> > > > bump
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> > epoch
> > > > > > >> > > > >> > >> > > > and
> > > > > > >> > > > >> > >> > > > > > > expect
> > > > > > >> > > > >> > >> > > > > > > > > >>> the next requests to have
> > epoch
> > > 74,
> > > > > and
> > > > > > >> > would
> > > > > > >> > > > >> reject
> > > > > > >> > > > >> > >> an
> > > > > > >> > > > >> > >> > > > request
> > > > > > >> > > > >> > >> > > > > > > with
> > > > > > >> > > > >> > >> > > > > > > > > >>> epoch 73, as the
> corresponding
> > > TX
> > > > > for
> > > > > > >> epoch
> > > > > > >> > > 73
> > > > > > >> > > > >> was
> > > > > > >> > > > >> > >> > already
> > > > > > >> > > > >> > >> > > > > > > committed.
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>> It seems you propose to
> write
> > > the
> > > > > > >> "prepare
> > > > > > >> > > > commit
> > > > > > >> > > > >> > >> marker"
> > > > > > >> > > > >> > >> > > and
> > > > > > >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` with
> > > epoch 74
> > > > > > >> > though,
> > > > > > >> > > > what
> > > > > > >> > > > >> > >> would
> > > > > > >> > > > >> > >> > > work,
> > > > > > >> > > > >> > >> > > > > but
> > > > > > >> > > > >> > >> > > > > > > it
> > > > > > >> > > > >> > >> > > > > > > > > >>> seems confusing. Is there a
> > > reason
> > > > > why
> > > > > > >> we
> > > > > > >> > > would
> > > > > > >> > > > >> use
> > > > > > >> > > > >> > >> the
> > > > > > >> > > > >> > >> > > > bumped
> > > > > > >> > > > >> > >> > > > > > > epoch
> > > > > > >> > > > >> > >> > > > > > > > 74
> > > > > > >> > > > >> > >> > > > > > > > > >>> instead of the current epoch
> > 73?
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>> 60)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>> When we are checking if the
> > > > > > >> transaction is
> > > > > > >> > > > >> ongoing,
> > > > > > >> > > > >> > >> we
> > > > > > >> > > > >> > >> > > need
> > > > > > >> > > > >> > >> > > > to
> > > > > > >> > > > >> > >> > > > > > > make
> > > > > > >> > > > >> > >> > > > > > > > a
> > > > > > >> > > > >> > >> > > > > > > > > >>> round
> > > > > > >> > > > >> > >> > > > > > > > > >>>> trip from the leader
> > partition
> > > to
> > > > > the
> > > > > > >> > > > >> transaction
> > > > > > >> > > > >> > >> > > > coordinator.
> > > > > > >> > > > >> > >> > > > > > In
> > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> time
> > > > > > >> > > > >> > >> > > > > > > > > >>>> we are waiting for this
> > > message to
> > > > > come
> > > > > > >> > > back,
> > > > > > >> > > > in
> > > > > > >> > > > >> > >> theory
> > > > > > >> > > > >> > >> > we
> > > > > > >> > > > >> > >> > > > > could
> > > > > > >> > > > >> > >> > > > > > > > have
> > > > > > >> > > > >> > >> > > > > > > > > >>> sent
> > > > > > >> > > > >> > >> > > > > > > > > >>>> a commit/abort call that
> > would
> > > > > make the
> > > > > > >> > > > original
> > > > > > >> > > > >> > >> result
> > > > > > >> > > > >> > >> > of
> > > > > > >> > > > >> > >> > > > the
> > > > > > >> > > > >> > >> > > > > > > check
> > > > > > >> > > > >> > >> > > > > > > > > >>> out of
> > > > > > >> > > > >> > >> > > > > > > > > >>>> date. That is why we can
> > check
> > > the
> > > > > > >> leader
> > > > > > >> > > > state
> > > > > > >> > > > >> > >> before
> > > > > > >> > > > >> > >> > we
> > > > > > >> > > > >> > >> > > > > write
> > > > > > >> > > > >> > >> > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> log.
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>> Thanks. Got it.
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>> However, is this really an
> > > issue?
> > > > > We put
> > > > > > >> > the
> > > > > > >> > > > >> produce
> > > > > > >> > > > >> > >> > > request
> > > > > > >> > > > >> > >> > > > in
> > > > > > >> > > > >> > >> > > > > > > > > >>> purgatory, so how could we
> > > process
> > > > > the
> > > > > > >> > > > >> > >> > > > `WriteTxnMarkerRequest`
> > > > > > >> > > > >> > >> > > > > > > first?
> > > > > > >> > > > >> > >> > > > > > > > > >>> Don't we need to put the
> > > > > > >> > > > `WriteTxnMarkerRequest`
> > > > > > >> > > > >> > into
> > > > > > >> > > > >> > >> > > > > purgatory,
> > > > > > >> > > > >> > >> > > > > > > too,
> > > > > > >> > > > >> > >> > > > > > > > > >>> for this case, and process
> > both
> > > > > request
> > > > > > >> > > > in-order?
> > > > > > >> > > > >> > >> (Again,
> > > > > > >> > > > >> > >> > > my
> > > > > > >> > > > >> > >> > > > > > broker
> > > > > > >> > > > >> > >> > > > > > > > > >>> knowledge is limited and
> maybe
> > > we
> > > > > don't
> > > > > > >> > > > maintain
> > > > > > >> > > > >> > >> request
> > > > > > >> > > > >> > >> > > > order
> > > > > > >> > > > >> > >> > > > > > for
> > > > > > >> > > > >> > >> > > > > > > > this
> > > > > > >> > > > >> > >> > > > > > > > > >>> case, what seems to be an
> > issue
> > > > > IMHO,
> > > > > > >> and I
> > > > > > >> > > am
> > > > > > >> > > > >> > >> wondering
> > > > > > >> > > > >> > >> > if
> > > > > > >> > > > >> > >> > > > > > > changing
> > > > > > >> > > > >> > >> > > > > > > > > >>> request handling to preserve
> > > order
> > > > > for
> > > > > > >> this
> > > > > > >> > > > case
> > > > > > >> > > > >> > >> might be
> > > > > > >> > > > >> > >> > > the
> > > > > > >> > > > >> > >> > > > > > > cleaner
> > > > > > >> > > > >> > >> > > > > > > > > >>> solution?)
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>> -Matthias
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>> On 11/30/22 3:28 PM, Artem
> > > Livshits
> > > > > > >> wrote:
> > > > > > >> > > > >> > >> > > > > > > > > >>>> Hi Justine,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>> I think the interesting
> part
> > is
> > > > > not in
> > > > > > >> > this
> > > > > > >> > > > >> logic
> > > > > > >> > > > >> > >> > (because
> > > > > > >> > > > >> > >> > > > it
> > > > > > >> > > > >> > >> > > > > > > tries
> > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > > >>>> figure out when
> > > > > UNKNOWN_PRODUCER_ID is
> > > > > > >> > > > retriable
> > > > > > >> > > > >> > and
> > > > > > >> > > > >> > >> if
> > > > > > >> > > > >> > >> > > it's
> > > > > > >> > > > >> > >> > > > > > > > > retryable,
> > > > > > >> > > > >> > >> > > > > > > > > >>>> it's definitely not fatal),
> > but
> > > > > what
> > > > > > >> > happens
> > > > > > >> > > > >> when
> > > > > > >> > > > >> > >> this
> > > > > > >> > > > >> > >> > > logic
> > > > > > >> > > > >> > >> > > > > > > doesn't
> > > > > > >> > > > >> > >> > > > > > > > > >>> return
> > > > > > >> > > > >> > >> > > > > > > > > >>>> 'true' and falls through.
> In
> > > the
> > > > > old
> > > > > > >> > > clients
> > > > > > >> > > > it
> > > > > > >> > > > >> > >> seems
> > > > > > >> > > > >> > >> > to
> > > > > > >> > > > >> > >> > > be
> > > > > > >> > > > >> > >> > > > > > > fatal,
> > > > > > >> > > > >> > >> > > > > > > > if
> > > > > > >> > > > >> > >> > > > > > > > > >>> we
> > > > > > >> > > > >> > >> > > > > > > > > >>>> keep the behavior in the
> new
> > > > > clients,
> > > > > > >> I'd
> > > > > > >> > > > >> expect it
> > > > > > >> > > > >> > >> > would
> > > > > > >> > > > >> > >> > > be
> > > > > > >> > > > >> > >> > > > > > fatal
> > > > > > >> > > > >> > >> > > > > > > > as
> > > > > > >> > > > >> > >> > > > > > > > > >>> well.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>> -Artem
> > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>> On Tue, Nov 29, 2022 at
> 11:57
> > > AM
> > > > > > >> Justine
> > > > > > >> > > > Olshan
> > > > > > >> > > > >> > >> > > > > > > > > >>>>
> <jolshan@confluent.io.invalid
> > >
> > > > > wrote:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> Hi Artem and Jeff,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> Thanks for taking a look
> and
> > > > > sorry for
> > > > > > >> > the
> > > > > > >> > > > slow
> > > > > > >> > > > >> > >> > response.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> You both mentioned the
> > change
> > > to
> > > > > > >> handle
> > > > > > >> > > > >> > >> > > UNKNOWN_PRODUCER_ID
> > > > > > >> > > > >> > >> > > > > > > errors.
> > > > > > >> > > > >> > >> > > > > > > > > To
> > > > > > >> > > > >> > >> > > > > > > > > >>> be
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> clear — this error code
> will
> > > only
> > > > > be
> > > > > > >> sent
> > > > > > >> > > > again
> > > > > > >> > > > >> > when
> > > > > > >> > > > >> > >> > the
> > > > > > >> > > > >> > >> > > > > > client's
> > > > > > >> > > > >> > >> > > > > > > > > >>> request
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> version is high enough to
> > > ensure
> > > > > we
> > > > > > >> > handle
> > > > > > >> > > it
> > > > > > >> > > > >> > >> > correctly.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> The current (Java) client
> > > handles
> > > > > > >> this by
> > > > > > >> > > the
> > > > > > >> > > > >> > >> following
> > > > > > >> > > > >> > >> > > > > > (somewhat
> > > > > > >> > > > >> > >> > > > > > > > > long)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> code snippet:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> // An UNKNOWN_PRODUCER_ID
> > > means
> > > > > that
> > > > > > >> we
> > > > > > >> > > have
> > > > > > >> > > > >> lost
> > > > > > >> > > > >> > >> the
> > > > > > >> > > > >> > >> > > > > producer
> > > > > > >> > > > >> > >> > > > > > > > state
> > > > > > >> > > > >> > >> > > > > > > > > >>> on the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> broker. Depending on the
> log
> > > start
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> // offset, we may want to
> > > retry
> > > > > > >> these, as
> > > > > > >> > > > >> > described
> > > > > > >> > > > >> > >> for
> > > > > > >> > > > >> > >> > > > each
> > > > > > >> > > > >> > >> > > > > > case
> > > > > > >> > > > >> > >> > > > > > > > > >>> below. If
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> none of those apply, then
> > for
> > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> // idempotent producer, we
> > > will
> > > > > > >> locally
> > > > > > >> > > bump
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> epoch
> > > > > > >> > > > >> > >> > > and
> > > > > > >> > > > >> > >> > > > > > reset
> > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> sequence numbers of
> > in-flight
> > > > > batches
> > > > > > >> > from
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> // sequence 0, then retry
> > the
> > > > > failed
> > > > > > >> > batch,
> > > > > > >> > > > >> which
> > > > > > >> > > > >> > >> > should
> > > > > > >> > > > >> > >> > > > now
> > > > > > >> > > > >> > >> > > > > > > > succeed.
> > > > > > >> > > > >> > >> > > > > > > > > >>> For
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> the transactional
> producer,
> > > allow
> > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> // batch to fail. When
> > > processing
> > > > > the
> > > > > > >> > > failed
> > > > > > >> > > > >> > batch,
> > > > > > >> > > > >> > >> we
> > > > > > >> > > > >> > >> > > will
> > > > > > >> > > > >> > >> > > > > > > > > transition
> > > > > > >> > > > >> > >> > > > > > > > > >>> to
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> an abortable error and
> set a
> > > flag
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> // indicating that we need
> > to
> > > > > bump the
> > > > > > >> > > epoch
> > > > > > >> > > > >> (if
> > > > > > >> > > > >> > >> > > supported
> > > > > > >> > > > >> > >> > > > by
> > > > > > >> > > > >> > >> > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> broker).
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> if (error ==
> > > > > > >> > Errors.*UNKNOWN_PRODUCER_ID*)
> > > > > > >> > > {
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>       if
> > > (response.logStartOffset
> > > > > ==
> > > > > > >> -1)
> > > > > > >> > {
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // We don't know
> > > the log
> > > > > > >> start
> > > > > > >> > > > offset
> > > > > > >> > > > >> > with
> > > > > > >> > > > >> > >> > this
> > > > > > >> > > > >> > >> > > > > > > response.
> > > > > > >> > > > >> > >> > > > > > > > > We
> > > > > > >> > > > >> > >> > > > > > > > > >>> should
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> just retry the request
> until
> > > we
> > > > > get
> > > > > > >> it.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // The
> > > > > UNKNOWN_PRODUCER_ID
> > > > > > >> > error
> > > > > > >> > > > code
> > > > > > >> > > > >> > was
> > > > > > >> > > > >> > >> > added
> > > > > > >> > > > >> > >> > > > > along
> > > > > > >> > > > >> > >> > > > > > > > with
> > > > > > >> > > > >> > >> > > > > > > > > >>> the new
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> ProduceResponse which
> > > includes the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           //
> logStartOffset.
> > > So
> > > > > the
> > > > > > >> '-1'
> > > > > > >> > > > >> sentinel
> > > > > > >> > > > >> > is
> > > > > > >> > > > >> > >> > not
> > > > > > >> > > > >> > >> > > > for
> > > > > > >> > > > >> > >> > > > > > > > backward
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> compatibility. Instead, it
> > is
> > > > > possible
> > > > > > >> > for
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // a broker to
> not
> > > know
> > > > > the
> > > > > > >> > > > >> > >> logStartOffset at
> > > > > > >> > > > >> > >> > > > when
> > > > > > >> > > > >> > >> > > > > it
> > > > > > >> > > > >> > >> > > > > > > is
> > > > > > >> > > > >> > >> > > > > > > > > >>> returning
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> the response because the
> > > partition
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // may have
> moved
> > > away
> > > > > from
> > > > > > >> the
> > > > > > >> > > > >> broker
> > > > > > >> > > > >> > >> from
> > > > > > >> > > > >> > >> > the
> > > > > > >> > > > >> > >> > > > > time
> > > > > > >> > > > >> > >> > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> error was
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> initially raised to the
> time
> > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // response was
> > > being
> > > > > > >> > > constructed.
> > > > > > >> > > > In
> > > > > > >> > > > >> > >> these
> > > > > > >> > > > >> > >> > > > cases,
> > > > > > >> > > > >> > >> > > > > we
> > > > > > >> > > > >> > >> > > > > > > > > should
> > > > > > >> > > > >> > >> > > > > > > > > >>> just
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> retry the request: we are
> > > > > guaranteed
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // to eventually
> > > get a
> > > > > > >> > > > logStartOffset
> > > > > > >> > > > >> > once
> > > > > > >> > > > >> > >> > > things
> > > > > > >> > > > >> > >> > > > > > > settle
> > > > > > >> > > > >> > >> > > > > > > > > down.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>       if
> > > > > > >> (batch.sequenceHasBeenReset()) {
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // When the
> first
> > > > > inflight
> > > > > > >> > batch
> > > > > > >> > > > >> fails
> > > > > > >> > > > >> > >> due to
> > > > > > >> > > > >> > >> > > the
> > > > > > >> > > > >> > >> > > > > > > > > truncation
> > > > > > >> > > > >> > >> > > > > > > > > >>> case,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> then the sequences of all
> > the
> > > > > other
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // in flight
> > batches
> > > > > would
> > > > > > >> have
> > > > > > >> > > > been
> > > > > > >> > > > >> > >> > restarted
> > > > > > >> > > > >> > >> > > > from
> > > > > > >> > > > >> > >> > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> beginning.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> However, when those
> > responses
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // come back
> from
> > > the
> > > > > > >> broker,
> > > > > > >> > > they
> > > > > > >> > > > >> would
> > > > > > >> > > > >> > >> also
> > > > > > >> > > > >> > >> > > > come
> > > > > > >> > > > >> > >> > > > > > with
> > > > > > >> > > > >> > >> > > > > > > > an
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID error.
> > In
> > > this
> > > > > > >> case,
> > > > > > >> > we
> > > > > > >> > > > >> should
> > > > > > >> > > > >> > >> not
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // reset the
> > > sequence
> > > > > > >> numbers
> > > > > > >> > to
> > > > > > >> > > > the
> > > > > > >> > > > >> > >> > beginning.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>       } else if
> > > > > > >> > > > >> > >> > > > >
> > > (lastAckedOffset(batch.topicPartition).orElse(
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > > > > > >> > > > >> > >> > > > response.logStartOffset) {
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // The head of
> the
> > > log
> > > > > has
> > > > > > >> been
> > > > > > >> > > > >> removed,
> > > > > > >> > > > >> > >> > > probably
> > > > > > >> > > > >> > >> > > > > due
> > > > > > >> > > > >> > >> > > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> retention time elapsing.
> In
> > > this
> > > > > case,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // we expect to
> > > lose the
> > > > > > >> > producer
> > > > > > >> > > > >> state.
> > > > > > >> > > > >> > >> For
> > > > > > >> > > > >> > >> > > the
> > > > > > >> > > > >> > >> > > > > > > > > transactional
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> producer, reset the
> > sequences
> > > of
> > > > > all
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // inflight
> > batches
> > > to
> > > > > be
> > > > > > >> from
> > > > > > >> > > the
> > > > > > >> > > > >> > >> beginning
> > > > > > >> > > > >> > >> > > and
> > > > > > >> > > > >> > >> > > > > > retry
> > > > > > >> > > > >> > >> > > > > > > > > them,
> > > > > > >> > > > >> > >> > > > > > > > > >>> so
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> that the transaction does
> > not
> > > > > need to
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // be aborted.
> For
> > > the
> > > > > > >> > idempotent
> > > > > > >> > > > >> > >> producer,
> > > > > > >> > > > >> > >> > > bump
> > > > > > >> > > > >> > >> > > > > the
> > > > > > >> > > > >> > >> > > > > > > > epoch
> > > > > > >> > > > >> > >> > > > > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > > >>> avoid
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> reusing (sequence, epoch)
> > > pairs
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           if
> > > (isTransactional()) {
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > >
> > > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> this.producerIdAndEpoch);
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           } else {
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> requestEpochBumpForPartition(batch.topicPartition);
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           }
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>       if
> > (!isTransactional())
> > > {
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // For the
> > > idempotent
> > > > > > >> producer,
> > > > > > >> > > > >> always
> > > > > > >> > > > >> > >> retry
> > > > > > >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> errors. If the batch has
> the
> > > > > current
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // producer ID
> and
> > > > > epoch,
> > > > > > >> > > request a
> > > > > > >> > > > >> bump
> > > > > > >> > > > >> > >> of
> > > > > > >> > > > >> > >> > the
> > > > > > >> > > > >> > >> > > > > > epoch.
> > > > > > >> > > > >> > >> > > > > > > > > >>> Otherwise
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> just retry the produce.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > >
> > > > > requestEpochBumpForPartition(batch.topicPartition);
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> }
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> I was considering keeping
> > this
> > > > > > >> behavior —
> > > > > > >> > > but
> > > > > > >> > > > >> am
> > > > > > >> > > > >> > >> open
> > > > > > >> > > > >> > >> > to
> > > > > > >> > > > >> > >> > > > > > > > simplifying
> > > > > > >> > > > >> > >> > > > > > > > > >>> it.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> We are leaving changes to
> > > older
> > > > > > >> clients
> > > > > > >> > off
> > > > > > >> > > > the
> > > > > > >> > > > >> > >> table
> > > > > > >> > > > >> > >> > > here
> > > > > > >> > > > >> > >> > > > > > since
> > > > > > >> > > > >> > >> > > > > > > it
> > > > > > >> > > > >> > >> > > > > > > > > >>> caused
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> many issues for clients in
> > the
> > > > > past.
> > > > > > >> > > > Previously
> > > > > > >> > > > >> > this
> > > > > > >> > > > >> > >> > was
> > > > > > >> > > > >> > >> > > a
> > > > > > >> > > > >> > >> > > > > > fatal
> > > > > > >> > > > >> > >> > > > > > > > > error
> > > > > > >> > > > >> > >> > > > > > > > > >>> and
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> we didn't have the
> > mechanisms
> > > in
> > > > > > >> place to
> > > > > > >> > > > >> detect
> > > > > > >> > > > >> > >> when
> > > > > > >> > > > >> > >> > > this
> > > > > > >> > > > >> > >> > > > > was
> > > > > > >> > > > >> > >> > > > > > a
> > > > > > >> > > > >> > >> > > > > > > > > >>> legitimate
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> case vs some bug or gap in
> > the
> > > > > > >> protocol.
> > > > > > >> > > > >> Ensuring
> > > > > > >> > > > >> > >> each
> > > > > > >> > > > >> > >> > > > > > > transaction
> > > > > > >> > > > >> > >> > > > > > > > > has
> > > > > > >> > > > >> > >> > > > > > > > > >>> its
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> own epoch should close
> this
> > > gap.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> And to address Jeff's
> second
> > > > > point:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> *does the typical produce
> > > request
> > > > > path
> > > > > > >> > > append
> > > > > > >> > > > >> > >> records
> > > > > > >> > > > >> > >> > to
> > > > > > >> > > > >> > >> > > > > local
> > > > > > >> > > > >> > >> > > > > > > log
> > > > > > >> > > > >> > >> > > > > > > > > >>> along*
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> *with the
> > > currentTxnFirstOffset
> > > > > > >> > > information?
> > > > > > >> > > > I
> > > > > > >> > > > >> > would
> > > > > > >> > > > >> > >> > like
> > > > > > >> > > > >> > >> > > > to
> > > > > > >> > > > >> > >> > > > > > > > > >>> understand*
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> *when the field is written
> > to
> > > > > disk.*
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> Yes, the first produce
> > request
> > > > > > >> populates
> > > > > > >> > > this
> > > > > > >> > > > >> > field
> > > > > > >> > > > >> > >> and
> > > > > > >> > > > >> > >> > > > > writes
> > > > > > >> > > > >> > >> > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> offset
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> as part of the record
> batch
> > > and
> > > > > also
> > > > > > >> to
> > > > > > >> > the
> > > > > > >> > > > >> > producer
> > > > > > >> > > > >> > >> > > state
> > > > > > >> > > > >> > >> > > > > > > > snapshot.
> > > > > > >> > > > >> > >> > > > > > > > > >>> When
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> we reload the records on
> > > restart
> > > > > > >> and/or
> > > > > > >> > > > >> > >> reassignment,
> > > > > > >> > > > >> > >> > we
> > > > > > >> > > > >> > >> > > > > > > repopulate
> > > > > > >> > > > >> > >> > > > > > > > > >>> this
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> field with the snapshot
> from
> > > disk
> > > > > > >> along
> > > > > > >> > > with
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> rest
> > > > > > >> > > > >> > >> > of
> > > > > > >> > > > >> > >> > > > the
> > > > > > >> > > > >> > >> > > > > > > > producer
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> state.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> Let me know if there are
> > > further
> > > > > > >> comments
> > > > > > >> > > > >> and/or
> > > > > > >> > > > >> > >> > > questions.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> Thanks,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> Justine
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> On Tue, Nov 22, 2022 at
> 9:00
> > > PM
> > > > > Jeff
> > > > > > >> Kim
> > > > > > >> > > > >> > >> > > > > > > > > <jeff.kim@confluent.io.invalid
> > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> wrote:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Hi Justine,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Thanks for the KIP! I
> have
> > > two
> > > > > > >> > questions:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> 1) For new clients, we
> can
> > > once
> > > > > again
> > > > > > >> > > return
> > > > > > >> > > > >> an
> > > > > > >> > > > >> > >> error
> > > > > > >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> for sequences
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> that are non-zero when
> > there
> > > is
> > > > > no
> > > > > > >> > > producer
> > > > > > >> > > > >> state
> > > > > > >> > > > >> > >> > > present
> > > > > > >> > > > >> > >> > > > on
> > > > > > >> > > > >> > >> > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> server.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> This will indicate we
> > missed
> > > the
> > > > > 0
> > > > > > >> > > sequence
> > > > > > >> > > > >> and
> > > > > > >> > > > >> > we
> > > > > > >> > > > >> > >> > don't
> > > > > > >> > > > >> > >> > > > yet
> > > > > > >> > > > >> > >> > > > > > > want
> > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> write
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> to the log.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> I would like to
> understand
> > > the
> > > > > > >> current
> > > > > > >> > > > >> behavior
> > > > > > >> > > > >> > to
> > > > > > >> > > > >> > >> > > handle
> > > > > > >> > > > >> > >> > > > > > older
> > > > > > >> > > > >> > >> > > > > > > > > >>> clients,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> and if there are any
> > changes
> > > we
> > > > > are
> > > > > > >> > > making.
> > > > > > >> > > > >> Maybe
> > > > > > >> > > > >> > >> I'm
> > > > > > >> > > > >> > >> > > > > missing
> > > > > > >> > > > >> > >> > > > > > > > > >>> something,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> but we would want to
> > identify
> > > > > > >> whether we
> > > > > > >> > > > >> missed
> > > > > > >> > > > >> > >> the 0
> > > > > > >> > > > >> > >> > > > > sequence
> > > > > > >> > > > >> > >> > > > > > > for
> > > > > > >> > > > >> > >> > > > > > > > > >>> older
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> clients, no?
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> 2) Upon returning from
> the
> > > > > > >> transaction
> > > > > > >> > > > >> > >> coordinator, we
> > > > > > >> > > > >> > >> > > can
> > > > > > >> > > > >> > >> > > > > set
> > > > > > >> > > > >> > >> > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> transaction
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> as ongoing on the leader
> by
> > > > > > >> populating
> > > > > > >> > > > >> > >> > > > currentTxnFirstOffset
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> through the typical
> produce
> > > > > request
> > > > > > >> > > > handling.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> does the typical produce
> > > request
> > > > > path
> > > > > > >> > > append
> > > > > > >> > > > >> > >> records
> > > > > > >> > > > >> > >> > to
> > > > > > >> > > > >> > >> > > > > local
> > > > > > >> > > > >> > >> > > > > > > log
> > > > > > >> > > > >> > >> > > > > > > > > >>> along
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> with the
> > > currentTxnFirstOffset
> > > > > > >> > > information?
> > > > > > >> > > > I
> > > > > > >> > > > >> > would
> > > > > > >> > > > >> > >> > like
> > > > > > >> > > > >> > >> > > > to
> > > > > > >> > > > >> > >> > > > > > > > > understand
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> when the field is written
> > to
> > > > > disk.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Thanks,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Jeff
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> On Tue, Nov 22, 2022 at
> > 4:44
> > > PM
> > > > > Artem
> > > > > > >> > > > Livshits
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> <alivshits@confluent.io
> > > .invalid>
> > > > > > >> wrote:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> Hi Justine,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> Thank you for the KIP.
> I
> > > have
> > > > > one
> > > > > > >> > > > question.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> 5) For new clients, we
> can
> > > once
> > > > > > >> again
> > > > > > >> > > > return
> > > > > > >> > > > >> an
> > > > > > >> > > > >> > >> error
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> I believe we had
> problems
> > > in the
> > > > > > >> past
> > > > > > >> > > with
> > > > > > >> > > > >> > >> returning
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> because it was
> considered
> > > fatal
> > > > > and
> > > > > > >> > > > required
> > > > > > >> > > > >> > >> client
> > > > > > >> > > > >> > >> > > > > restart.
> > > > > > >> > > > >> > >> > > > > > > It
> > > > > > >> > > > >> > >> > > > > > > > > >>> would
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> be
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> good to spell out the
> new
> > > client
> > > > > > >> > behavior
> > > > > > >> > > > >> when
> > > > > > >> > > > >> > it
> > > > > > >> > > > >> > >> > > > receives
> > > > > > >> > > > >> > >> > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > error.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> -Artem
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> On Tue, Nov 22, 2022 at
> > > 10:00 AM
> > > > > > >> > Justine
> > > > > > >> > > > >> Olshan
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > <jo...@confluent.io.invalid>
> > > > > > >> wrote:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks for taking a
> look
> > > > > Matthias.
> > > > > > >> > I've
> > > > > > >> > > > >> tried
> > > > > > >> > > > >> > to
> > > > > > >> > > > >> > >> > > answer
> > > > > > >> > > > >> > >> > > > > your
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> questions
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> below:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 10)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Right — so the hanging
> > > > > transaction
> > > > > > >> > only
> > > > > > >> > > > >> occurs
> > > > > > >> > > > >> > >> when
> > > > > > >> > > > >> > >> > we
> > > > > > >> > > > >> > >> > > > > have
> > > > > > >> > > > >> > >> > > > > > a
> > > > > > >> > > > >> > >> > > > > > > > late
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> message
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> come in and the
> partition
> > > is
> > > > > never
> > > > > > >> > added
> > > > > > >> > > > to
> > > > > > >> > > > >> a
> > > > > > >> > > > >> > >> > > > transaction
> > > > > > >> > > > >> > >> > > > > > > again.
> > > > > > >> > > > >> > >> > > > > > > > > If
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> we
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> never add the partition
> > to
> > > a
> > > > > > >> > > transaction,
> > > > > > >> > > > we
> > > > > > >> > > > >> > will
> > > > > > >> > > > >> > >> > > never
> > > > > > >> > > > >> > >> > > > > > write
> > > > > > >> > > > >> > >> > > > > > > a
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> marker
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> and
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> never advance the LSO.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> If we do end up adding
> > the
> > > > > > >> partition
> > > > > > >> > to
> > > > > > >> > > > the
> > > > > > >> > > > >> > >> > > transaction
> > > > > > >> > > > >> > >> > > > (I
> > > > > > >> > > > >> > >> > > > > > > > suppose
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> this
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> can
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> happen before or after
> > the
> > > late
> > > > > > >> > message
> > > > > > >> > > > >> comes
> > > > > > >> > > > >> > in)
> > > > > > >> > > > >> > >> > then
> > > > > > >> > > > >> > >> > > > we
> > > > > > >> > > > >> > >> > > > > > will
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> include
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> late message in the
> next
> > > > > > >> (incorrect)
> > > > > > >> > > > >> > transaction.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> So perhaps it is
> clearer
> > to
> > > > > make
> > > > > > >> the
> > > > > > >> > > > >> > distinction
> > > > > > >> > > > >> > >> > > between
> > > > > > >> > > > >> > >> > > > > > > > messages
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> that
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> eventually get added to
> > the
> > > > > > >> > transaction
> > > > > > >> > > > (but
> > > > > > >> > > > >> > the
> > > > > > >> > > > >> > >> > wrong
> > > > > > >> > > > >> > >> > > > > one)
> > > > > > >> > > > >> > >> > > > > > or
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> messages
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> that never get added
> and
> > > become
> > > > > > >> > hanging.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 20)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> The client side change
> > for
> > > 2 is
> > > > > > >> > removing
> > > > > > >> > > > the
> > > > > > >> > > > >> > >> > > > addPartitions
> > > > > > >> > > > >> > >> > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> transaction
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> call. We don't need to
> > make
> > > > > this
> > > > > > >> from
> > > > > > >> > > the
> > > > > > >> > > > >> > >> producer
> > > > > > >> > > > >> > >> > to
> > > > > > >> > > > >> > >> > > > the
> > > > > > >> > > > >> > >> > > > > > txn
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> coordinator,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> only server side.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> In my opinion, the
> issue
> > > with
> > > > > the
> > > > > > >> > > > >> > >> addPartitionsToTxn
> > > > > > >> > > > >> > >> > > > call
> > > > > > >> > > > >> > >> > > > > > for
> > > > > > >> > > > >> > >> > > > > > > > > older
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> clients
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> is that we don't have
> the
> > > epoch
> > > > > > >> bump,
> > > > > > >> > so
> > > > > > >> > > > we
> > > > > > >> > > > >> > don't
> > > > > > >> > > > >> > >> > know
> > > > > > >> > > > >> > >> > > > if
> > > > > > >> > > > >> > >> > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> message
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> belongs to the previous
> > > > > > >> transaction or
> > > > > > >> > > > this
> > > > > > >> > > > >> > one.
> > > > > > >> > > > >> > >> We
> > > > > > >> > > > >> > >> > > need
> > > > > > >> > > > >> > >> > > > > to
> > > > > > >> > > > >> > >> > > > > > > > check
> > > > > > >> > > > >> > >> > > > > > > > > if
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> partition has been
> added
> > to
> > > > > this
> > > > > > >> > > > >> transaction.
> > > > > > >> > > > >> > Of
> > > > > > >> > > > >> > >> > > course,
> > > > > > >> > > > >> > >> > > > > > this
> > > > > > >> > > > >> > >> > > > > > > > > means
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> we
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> won't completely cover
> > the
> > > case
> > > > > > >> where
> > > > > > >> > we
> > > > > > >> > > > >> have a
> > > > > > >> > > > >> > >> > really
> > > > > > >> > > > >> > >> > > > > late
> > > > > > >> > > > >> > >> > > > > > > > > message
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> and
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> we
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> have added the
> partition
> > to
> > > > > the new
> > > > > > >> > > > >> > transaction,
> > > > > > >> > > > >> > >> but
> > > > > > >> > > > >> > >> > > > > that's
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> unfortunately
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> something we will need
> > the
> > > new
> > > > > > >> clients
> > > > > > >> > > to
> > > > > > >> > > > >> > cover.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 30)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Transaction is ongoing
> =
> > > > > partition
> > > > > > >> was
> > > > > > >> > > > >> added to
> > > > > > >> > > > >> > >> > > > > transaction
> > > > > > >> > > > >> > >> > > > > > > via
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> addPartitionsToTxn. We
> > > check
> > > > > this
> > > > > > >> with
> > > > > > >> > > the
> > > > > > >> > > > >> > >> > > > > > > DescribeTransactions
> > > > > > >> > > > >> > >> > > > > > > > > >>> call.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Let
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> me know if this wasn't
> > > > > sufficiently
> > > > > > >> > > > >> explained
> > > > > > >> > > > >> > >> here:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> > > > >> > >> > > > >
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > > >> > >> > >
> > > > > > >> > > > >> > >> >
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> >
> > > > > > >> > > > >>
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 40)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> The idea here is that
> if
> > > any
> > > > > > >> messages
> > > > > > >> > > > >> somehow
> > > > > > >> > > > >> > >> come
> > > > > > >> > > > >> > >> > in
> > > > > > >> > > > >> > >> > > > > before
> > > > > > >> > > > >> > >> > > > > > > we
> > > > > > >> > > > >> > >> > > > > > > > > get
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> new
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> epoch to the producer,
> > they
> > > > > will be
> > > > > > >> > > > fenced.
> > > > > > >> > > > >> > >> However,
> > > > > > >> > > > >> > >> > > if
> > > > > > >> > > > >> > >> > > > we
> > > > > > >> > > > >> > >> > > > > > > don't
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> think
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> this
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> is necessary, it can be
> > > > > discussed
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 50)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> It should be
> synchronous
> > > > > because
> > > > > > >> if we
> > > > > > >> > > > have
> > > > > > >> > > > >> an
> > > > > > >> > > > >> > >> event
> > > > > > >> > > > >> > >> > > > (ie,
> > > > > > >> > > > >> > >> > > > > an
> > > > > > >> > > > >> > >> > > > > > > > > error)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> that
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> causes us to need to
> > abort
> > > the
> > > > > > >> > > > transaction,
> > > > > > >> > > > >> we
> > > > > > >> > > > >> > >> need
> > > > > > >> > > > >> > >> > to
> > > > > > >> > > > >> > >> > > > > know
> > > > > > >> > > > >> > >> > > > > > > > which
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> partitions to send
> > > transaction
> > > > > > >> markers
> > > > > > >> > > to.
> > > > > > >> > > > >> We
> > > > > > >> > > > >> > >> know
> > > > > > >> > > > >> > >> > the
> > > > > > >> > > > >> > >> > > > > > > > partitions
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> because
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> we added them to the
> > > > > coordinator
> > > > > > >> via
> > > > > > >> > the
> > > > > > >> > > > >> > >> > > > > addPartitionsToTxn
> > > > > > >> > > > >> > >> > > > > > > > call.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Previously we have had
> > > > > asynchronous
> > > > > > >> > > calls
> > > > > > >> > > > in
> > > > > > >> > > > >> > the
> > > > > > >> > > > >> > >> > past
> > > > > > >> > > > >> > >> > > > (ie,
> > > > > > >> > > > >> > >> > > > > > > > writing
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> commit markers when the
> > > > > > >> transaction is
> > > > > > >> > > > >> > completed)
> > > > > > >> > > > >> > >> > but
> > > > > > >> > > > >> > >> > > > > often
> > > > > > >> > > > >> > >> > > > > > > this
> > > > > > >> > > > >> > >> > > > > > > > > >>> just
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> causes confusion as we
> > > need to
> > > > > wait
> > > > > > >> > for
> > > > > > >> > > > some
> > > > > > >> > > > >> > >> > > operations
> > > > > > >> > > > >> > >> > > > to
> > > > > > >> > > > >> > >> > > > > > > > > complete.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> In
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> writing commit markers
> > > case,
> > > > > > >> clients
> > > > > > >> > > often
> > > > > > >> > > > >> see
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> CONCURRENT_TRANSACTIONs
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> error messages and that
> > > can be
> > > > > > >> > > confusing.
> > > > > > >> > > > >> For
> > > > > > >> > > > >> > >> that
> > > > > > >> > > > >> > >> > > > reason,
> > > > > > >> > > > >> > >> > > > > > it
> > > > > > >> > > > >> > >> > > > > > > > may
> > > > > > >> > > > >> > >> > > > > > > > > be
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> simpler to just have
> > > > > synchronous
> > > > > > >> > calls —
> > > > > > >> > > > >> > >> especially
> > > > > > >> > > > >> > >> > if
> > > > > > >> > > > >> > >> > > > we
> > > > > > >> > > > >> > >> > > > > > need
> > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> block
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> on
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> some operation's
> > completion
> > > > > anyway
> > > > > > >> > > before
> > > > > > >> > > > we
> > > > > > >> > > > >> > can
> > > > > > >> > > > >> > >> > start
> > > > > > >> > > > >> > >> > > > the
> > > > > > >> > > > >> > >> > > > > > > next
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> transaction. And yes, I
> > > meant
> > > > > > >> > > > coordinator. I
> > > > > > >> > > > >> > will
> > > > > > >> > > > >> > >> > fix
> > > > > > >> > > > >> > >> > > > > that.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 60)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> When we are checking if
> > the
> > > > > > >> > transaction
> > > > > > >> > > is
> > > > > > >> > > > >> > >> ongoing,
> > > > > > >> > > > >> > >> > we
> > > > > > >> > > > >> > >> > > > > need
> > > > > > >> > > > >> > >> > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > > make
> > > > > > >> > > > >> > >> > > > > > > > > >>> a
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> round
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> trip from the leader
> > > partition
> > > > > to
> > > > > > >> the
> > > > > > >> > > > >> > transaction
> > > > > > >> > > > >> > >> > > > > > coordinator.
> > > > > > >> > > > >> > >> > > > > > > > In
> > > > > > >> > > > >> > >> > > > > > > > > >>> the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> time
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> we are waiting for this
> > > > > message to
> > > > > > >> > come
> > > > > > >> > > > >> back,
> > > > > > >> > > > >> > in
> > > > > > >> > > > >> > >> > > theory
> > > > > > >> > > > >> > >> > > > we
> > > > > > >> > > > >> > >> > > > > > > could
> > > > > > >> > > > >> > >> > > > > > > > > >>> have
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> sent
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> a commit/abort call
> that
> > > would
> > > > > make
> > > > > > >> > the
> > > > > > >> > > > >> > original
> > > > > > >> > > > >> > >> > > result
> > > > > > >> > > > >> > >> > > > of
> > > > > > >> > > > >> > >> > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > check
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> out
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> of
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> date. That is why we
> can
> > > check
> > > > > the
> > > > > > >> > > leader
> > > > > > >> > > > >> state
> > > > > > >> > > > >> > >> > before
> > > > > > >> > > > >> > >> > > > we
> > > > > > >> > > > >> > >> > > > > > > write
> > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> log.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> I'm happy to update the
> > > KIP if
> > > > > > >> some of
> > > > > > >> > > > these
> > > > > > >> > > > >> > >> things
> > > > > > >> > > > >> > >> > > were
> > > > > > >> > > > >> > >> > > > > not
> > > > > > >> > > > >> > >> > > > > > > > > clear.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Justine
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> On Mon, Nov 21, 2022 at
> > > 7:11 PM
> > > > > > >> > Matthias
> > > > > > >> > > > J.
> > > > > > >> > > > >> > Sax <
> > > > > > >> > > > >> > >> > > > > > > > mjsax@apache.org
> > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> wrote:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Thanks for the KIP.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Couple of
> clarification
> > > > > questions
> > > > > > >> (I
> > > > > > >> > am
> > > > > > >> > > > >> not a
> > > > > > >> > > > >> > >> > broker
> > > > > > >> > > > >> > >> > > > > expert
> > > > > > >> > > > >> > >> > > > > > > do
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> maybe
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> some question are
> > obvious
> > > for
> > > > > > >> others,
> > > > > > >> > > but
> > > > > > >> > > > >> not
> > > > > > >> > > > >> > >> for
> > > > > > >> > > > >> > >> > me
> > > > > > >> > > > >> > >> > > > with
> > > > > > >> > > > >> > >> > > > > > my
> > > > > > >> > > > >> > >> > > > > > > > lack
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> of
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> broker knowledge).
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (10)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> 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.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> What happens if the
> > > message
> > > > > come
> > > > > > >> in
> > > > > > >> > > > before
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> next
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> addPartitionsToTxn
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> request? It seems the
> > > broker
> > > > > > >> hosting
> > > > > > >> > > the
> > > > > > >> > > > >> data
> > > > > > >> > > > >> > >> > > > partitions
> > > > > > >> > > > >> > >> > > > > > > won't
> > > > > > >> > > > >> > >> > > > > > > > > know
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> anything about it and
> > > append
> > > > > it to
> > > > > > >> > the
> > > > > > >> > > > >> > >> partition,
> > > > > > >> > > > >> > >> > > too?
> > > > > > >> > > > >> > >> > > > > What
> > > > > > >> > > > >> > >> > > > > > > is
> > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> difference between
> both
> > > cases?
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Also, it seems a TX
> > would
> > > only
> > > > > > >> hang,
> > > > > > >> > if
> > > > > > >> > > > >> there
> > > > > > >> > > > >> > >> is no
> > > > > > >> > > > >> > >> > > > > > following
> > > > > > >> > > > >> > >> > > > > > > > TX
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> that
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> is
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> either committer or
> > > aborted?
> > > > > Thus,
> > > > > > >> > for
> > > > > > >> > > > the
> > > > > > >> > > > >> > case
> > > > > > >> > > > >> > >> > > above,
> > > > > > >> > > > >> > >> > > > > the
> > > > > > >> > > > >> > >> > > > > > TX
> > > > > > >> > > > >> > >> > > > > > > > > might
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> actually not hang (of
> > > course,
> > > > > we
> > > > > > >> > might
> > > > > > >> > > > get
> > > > > > >> > > > >> an
> > > > > > >> > > > >> > >> EOS
> > > > > > >> > > > >> > >> > > > > violation
> > > > > > >> > > > >> > >> > > > > > > if
> > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> first
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> TX was aborted and the
> > > second
> > > > > > >> > > committed,
> > > > > > >> > > > or
> > > > > > >> > > > >> > the
> > > > > > >> > > > >> > >> > other
> > > > > > >> > > > >> > >> > > > way
> > > > > > >> > > > >> > >> > > > > > > > > around).
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (20)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Of course, 1 and 2
> > > require
> > > > > > >> > client-side
> > > > > > >> > > > >> > >> changes, so
> > > > > > >> > > > >> > >> > > for
> > > > > > >> > > > >> > >> > > > > > older
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> clients,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> those approaches won’t
> > > apply.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> For (1) I understand
> > why a
> > > > > client
> > > > > > >> > > change
> > > > > > >> > > > is
> > > > > > >> > > > >> > >> > > necessary,
> > > > > > >> > > > >> > >> > > > > but
> > > > > > >> > > > >> > >> > > > > > > not
> > > > > > >> > > > >> > >> > > > > > > > > sure
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> why
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> we need a client
> change
> > > for
> > > > > (2).
> > > > > > >> Can
> > > > > > >> > > you
> > > > > > >> > > > >> > >> elaborate?
> > > > > > >> > > > >> > >> > > --
> > > > > > >> > > > >> > >> > > > > > Later
> > > > > > >> > > > >> > >> > > > > > > > you
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> explain
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> that we should send a
> > > > > > >> > > > >> > >> DescribeTransactionRequest,
> > > > > > >> > > > >> > >> > > but I
> > > > > > >> > > > >> > >> > > > > am
> > > > > > >> > > > >> > >> > > > > > > not
> > > > > > >> > > > >> > >> > > > > > > > > sure
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> why?
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Can't we not just do
> an
> > > > > implicit
> > > > > > >> > > > >> > >> AddPartiitonToTx,
> > > > > > >> > > > >> > >> > > too?
> > > > > > >> > > > >> > >> > > > > If
> > > > > > >> > > > >> > >> > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > old
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> producer correctly
> > > registered
> > > > > the
> > > > > > >> > > > partition
> > > > > > >> > > > >> > >> > already,
> > > > > > >> > > > >> > >> > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> TX-coordinator
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> can just ignore it as
> > > it's an
> > > > > > >> > > idempotent
> > > > > > >> > > > >> > >> operation?
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (30)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> To cover older
> clients,
> > > we
> > > > > will
> > > > > > >> > > ensure a
> > > > > > >> > > > >> > >> > transaction
> > > > > > >> > > > >> > >> > > > is
> > > > > > >> > > > >> > >> > > > > > > > ongoing
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> before
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> we write to a
> > transaction
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Not sure what you mean
> > by
> > > > > this?
> > > > > > >> Can
> > > > > > >> > you
> > > > > > >> > > > >> > >> elaborate?
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (40)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> [the TX-coordinator]
> > will
> > > > > write
> > > > > > >> the
> > > > > > >> > > > >> prepare
> > > > > > >> > > > >> > >> commit
> > > > > > >> > > > >> > >> > > > > message
> > > > > > >> > > > >> > >> > > > > > > > with
> > > > > > >> > > > >> > >> > > > > > > > > a
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> bumped
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> epoch and send
> > > > > > >> WriteTxnMarkerRequests
> > > > > > >> > > > with
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> > bumped
> > > > > > >> > > > >> > >> > > > > > epoch.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Why do we use the
> bumped
> > > > > epoch for
> > > > > > >> > > both?
> > > > > > >> > > > It
> > > > > > >> > > > >> > >> seems
> > > > > > >> > > > >> > >> > > more
> > > > > > >> > > > >> > >> > > > > > > > intuitive
> > > > > > >> > > > >> > >> > > > > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> use
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> the current epoch, and
> > > only
> > > > > return
> > > > > > >> > the
> > > > > > >> > > > >> bumped
> > > > > > >> > > > >> > >> epoch
> > > > > > >> > > > >> > >> > > to
> > > > > > >> > > > >> > >> > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> producer?
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (50) "Implicit
> > > > > > >> > > AddPartitionToTransaction"
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Why does the
> implicitly
> > > sent
> > > > > > >> request
> > > > > > >> > > need
> > > > > > >> > > > >> to
> > > > > > >> > > > >> > be
> > > > > > >> > > > >> > >> > > > > > synchronous?
> > > > > > >> > > > >> > >> > > > > > > > The
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> KIP
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> also says
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> in case we need to
> > abort
> > > and
> > > > > > >> need to
> > > > > > >> > > > know
> > > > > > >> > > > >> > which
> > > > > > >> > > > >> > >> > > > > partitions
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> What do you mean by
> > this?
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> we don’t want to
> write
> > > to it
> > > > > > >> before
> > > > > > >> > we
> > > > > > >> > > > >> store
> > > > > > >> > > > >> > in
> > > > > > >> > > > >> > >> > the
> > > > > > >> > > > >> > >> > > > > > > > transaction
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> manager
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Do you mean
> > TX-coordinator
> > > > > > >> instead of
> > > > > > >> > > > >> > "manager"?
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (60)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> For older clients and
> > > ensuring
> > > > > > >> that
> > > > > > >> > the
> > > > > > >> > > > TX
> > > > > > >> > > > >> is
> > > > > > >> > > > >> > >> > > ongoing,
> > > > > > >> > > > >> > >> > > > > you
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> describe a
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> race condition. I am
> not
> > > sure
> > > > > if I
> > > > > > >> > can
> > > > > > >> > > > >> follow
> > > > > > >> > > > >> > >> here.
> > > > > > >> > > > >> > >> > > Can
> > > > > > >> > > > >> > >> > > > > you
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> elaborate?
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> -Matthias
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> On 11/18/22 1:21 PM,
> > > Justine
> > > > > > >> Olshan
> > > > > > >> > > > wrote:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Hey all!
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> I'd like to start a
> > > > > discussion
> > > > > > >> on my
> > > > > > >> > > > >> proposal
> > > > > > >> > > > >> > >> to
> > > > > > >> > > > >> > >> > add
> > > > > > >> > > > >> > >> > > > > some
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> server-side
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> checks on
> transactions
> > to
> > > > > avoid
> > > > > > >> > > hanging
> > > > > > >> > > > >> > >> > > transactions.
> > > > > > >> > > > >> > >> > > > I
> > > > > > >> > > > >> > >> > > > > > know
> > > > > > >> > > > >> > >> > > > > > > > > this
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> has
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> been
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> an issue for some
> time,
> > > so I
> > > > > > >> really
> > > > > > >> > > hope
> > > > > > >> > > > >> this
> > > > > > >> > > > >> > >> KIP
> > > > > > >> > > > >> > >> > > will
> > > > > > >> > > > >> > >> > > > > be
> > > > > > >> > > > >> > >> > > > > > > > > helpful
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> for
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> many
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> users of EOS.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> The KIP includes
> > changes
> > > that
> > > > > > >> will
> > > > > > >> > be
> > > > > > >> > > > >> > >> compatible
> > > > > > >> > > > >> > >> > > with
> > > > > > >> > > > >> > >> > > > > old
> > > > > > >> > > > >> > >> > > > > > > > > clients
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> and
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> changes to improve
> > > > > performance
> > > > > > >> and
> > > > > > >> > > > >> > correctness
> > > > > > >> > > > >> > >> on
> > > > > > >> > > > >> > >> > > new
> > > > > > >> > > > >> > >> > > > > > > clients.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Please take a look
> and
> > > leave
> > > > > any
> > > > > > >> > > > comments
> > > > > > >> > > > >> you
> > > > > > >> > > > >> > >> may
> > > > > > >> > > > >> > >> > > > have!
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> KIP:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> > > > >> > >> > > > >
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > > >> > >> > >
> > > > > > >> > > > >> > >> >
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> >
> > > > > > >> > > > >>
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> JIRA:
> > > > > > >> > > > >> > >> > > >
> > > https://issues.apache.org/jira/browse/KAFKA-14402
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Thanks!
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Justine
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> > > > >> > >> > > > >
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > > >> > >> > >
> > > > > > >> > > > >> > >> >
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> > >
> > > > > > >> > > > >> >
> > > > > > >> > > > >>
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > > >
> > > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Artem Livshits <al...@confluent.io.INVALID>.
In general, I'd like to avoid fatal errors as much as possible, in some
sense fatal errors just push out recovery logic to the application which
either complicates the application or leads to disruption (we've seen cases
when a transient broker error could lead to work stoppage when applications
need to be manually restarted).  I think we should strive to define
recovery logic for most errors (and/or encapsulate it in the Kafka client
as much as possible).

One benefit of transactions is that they simplify recovery from errors,
pretty much any error (that's not handled transparently by retries in Kafka
client) can be handled by the application via aborting the transaction and
repeating the transactional logic again.  One tricky error is an error
during commit, because we don't know the outcome.  For commit errors, the
recommendation should be to retry the commit until it returns the specific
result (committed or aborted).

-Artem

On Fri, Jan 20, 2023 at 2:52 PM Justine Olshan <jo...@confluent.io.invalid>
wrote:

> That's a fair point about other clients.
>
> I think the abortable error case is interesting because I'm curious how
> other clients would handle this. I assume they would need to implement
> handling for the error code unless they did something like "any unknown
> error codes/any codes that aren't x,y,z are retriable." I would hope that
> unknown error codes were fatal, and if the code was implemented it would
> abort the transaction. But I will think on this too.
>
> As for InvalidRecord -- you mentioned it was not fatal, but I'm taking a
> look through the code. We would see this on handling the produce response.
> If I recall correctly, we check if errors are retriable. I think this error
> would not be retriable. But I guess the concern here is that it is not
> enough for just that batch to fail. I guess I hadn't considered fully
> fencing the old producer but there are valid arguments here why we would
> want to.
>
> Thanks,
> Justine
>
> On Fri, Jan 20, 2023 at 2:35 PM Guozhang Wang <gu...@gmail.com>
> wrote:
>
> > Thanks Justine for the replies! I agree with most of your thoughts.
> >
> > Just for 3/7), though I agree for our own AK producer, since we do
> > "nextRequest(boolean hasIncompleteBatches)", we guarantee the end-txn
> > would not be sent until we've effectively flushed, but I was referring
> > to any future bugs or other buggy clients that the same client may get
> > into this situation, in which case we should give the client a clear
> > msg that "you did something wrong, and hence now you should fatally
> > close yourself". What I'm concerned about is that, by seeing an
> > "abortable error" or in some rare cases an "invalid record", the
> > client could not realize "something that's really bad happened". So
> > it's not about adding a new error, it's mainly about those real buggy
> > situations causing such "should never happen" cases, the errors return
> > would not be informative enough.
> >
> > Thinking in other ways, if we believe that for most cases such error
> > codes would not reach the original clients since they would be
> > disconnected or even gone by that time, and only in some rare cases
> > they would still be seen by the sending clients, then why not make
> > them more fatal and more specific than generic.
> >
> > Guozhang
> >
> > On Fri, Jan 20, 2023 at 1:59 PM Justine Olshan
> > <jo...@confluent.io.invalid> wrote:
> > >
> > > Hey Guozhang. Thanks for taking a look and for the detailed comments!
> > I'll
> > > do my best to address below.
> > >
> > > 1. I see what you are saying here, but I think I need to look through
> the
> > > sequence of events you mention. Typically we've seen this issue in a
> few
> > > cases.
> > >
> > >  One is when we have a producer disconnect when trying to produce.
> > > Typically in these cases, we abort the transaction. We've seen that
> after
> > > the markers are written, the disconnection can sometimes cause the
> > request
> > > to get flushed to the broker. In this case, we don't need client
> handling
> > > because the producer we are responding to is gone. We just needed to
> make
> > > sure we didn't write to the log on the broker side. I'm trying to think
> > of
> > > a case where we do have the client to return to. I'd think the same
> > client
> > > couldn't progress to committing the transaction unless the produce
> > request
> > > returned right? Of course, there is the incorrectly written clients
> case.
> > > I'll think on this a bit more and let you know if I come up with
> another
> > > scenario when we would return to an active client when the transaction
> is
> > > no longer ongoing.
> > >
> > > I was not aware that we checked the result of a send after we commit
> > > though. I'll need to look into that a bit more.
> > >
> > > 2. There were some questions about this in the discussion. The plan is
> to
> > > handle overflow with the mechanism we currently have in the producer.
> If
> > we
> > > try to bump and the epoch will overflow, we actually allocate a new
> > > producer ID. I need to confirm the fencing logic on the last epoch (ie,
> > we
> > > probably shouldn't allow any records to be produced with the final
> epoch
> > > since we can never properly fence that one).
> > >
> > > 3. I can agree with you that the current error handling is messy. I
> > recall
> > > taking a look at your KIP a while back, but I think I mostly saw the
> > > section about how the errors were wrapped. Maybe I need to take another
> > > look. As for abortable error, the idea was that the handling would be
> > > simple -- if this error is seen, the transaction should be aborted --
> no
> > > other logic about previous state or requests necessary. Is your concern
> > > simply about adding new errors? We were hoping to have an error that
> > would
> > > have one meaning and many of the current errors have a history of
> meaning
> > > different things on different client versions. That was the main
> > motivation
> > > for adding a new error.
> > >
> > > 4. This is a good point about record timestamp reordering. Timestamps
> > don't
> > > affect compaction, but they do affect retention deletion. For that,
> kafka
> > > considers the largest timestamp in the segment, so I think a small
> amount
> > > of reordering (hopefully on the order of milliseconds or even seconds)
> > will
> > > be ok. We take timestamps from clients so there is already a
> possibility
> > > for some drift and non-monotonically increasing timestamps.
> > >
> > > 5. Thanks for catching. The error is there, but it's actually that
> those
> > > fields should be 4+! Due to how the message generator works, I actually
> > > have to redefine those fields inside the
> `"AddPartitionsToTxnTransaction`
> > > block for it to build correctly. I'll fix it to be correct.
> > >
> > > 6. Correct -- we will only add the request to purgatory if the cache
> has
> > no
> > > ongoing transaction. I can change the wording to make that clearer that
> > we
> > > only place the request in purgatory if we need to contact the
> transaction
> > > coordinator.
> > >
> > > 7. We did take a look at some of the errors and it was hard to come up
> > with
> > > a good one. I agree that InvalidTxnStateException is ideal except for
> the
> > > fact that it hasn't been returned on Produce requests before. The error
> > > handling for clients is a bit vague (which is why I opened KAFKA-14439
> > > <https://issues.apache.org/jira/browse/KAFKA-14439>), but the decision
> > we
> > > made here was to only return errors that have been previously returned
> to
> > > producers. As for not being fatal, I think part of the theory was that
> in
> > > many cases, the producer would be disconnected. (See point 1) and this
> > > would just be an error to return from the server. I did plan to think
> > about
> > > other cases, so let me know if you think of any as well!
> > >
> > > Lots to say! Let me know if you have further thoughts!
> > > Justine
> > >
> > > On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang <
> > guozhang.wang.us@gmail.com>
> > > wrote:
> > >
> > > > Hello Justine,
> > > >
> > > > Thanks for the great write-up! I made a quick pass through it and
> here
> > > > are some thoughts (I have not been able to read through this thread
> so
> > > > pardon me if they have overlapped or subsumed by previous comments):
> > > >
> > > > First are some meta ones:
> > > >
> > > > 1. I think we need to also improve the client's experience once we
> > > > have this defence in place. More concretely, say a user's producer
> > > > code is like following:
> > > >
> > > > future = producer.send();
> > > > // producer.flush();
> > > > producer.commitTransaction();
> > > > future.get();
> > > >
> > > > Which resulted in the order of a) produce-request sent by producer,
> b)
> > > > end-txn-request sent by producer, c) end-txn-response sent back, d)
> > > > txn-marker-request sent from coordinator to partition leader, e)
> > > > produce-request finally received by the partition leader, before this
> > > > KIP e) step would be accepted causing a dangling txn; now it would be
> > > > rejected in step e) which is good. But from the client's point of
> view
> > > > now it becomes confusing since the `commitTransaction()` returns
> > > > successfully, but the "future" throws an invalid-epoch error, and
> they
> > > > are not sure if the transaction did succeed or not. In fact, it
> > > > "partially succeeded" with some msgs being rejected but others
> > > > committed successfully.
> > > >
> > > > Of course the easy way to avoid this is, always call
> > > > "producer.flush()" before commitTxn and that's what we do ourselves,
> > > > and what we recommend users do. But I suspect not everyone does it.
> In
> > > > fact I just checked the javadoc in KafkaProducer and our code snippet
> > > > does not include a `flush()` call. So I'm thinking maybe we can in
> > > > side the `commitTxn` code to enforce flushing before sending the
> > > > end-txn request.
> > > >
> > > > 2. I'd like to clarify a bit details on "just add partitions to the
> > > > transaction on the first produce request during a transaction". My
> > > > understanding is that the partition leader's cache has the producer
> id
> > > > / sequence / epoch for the latest txn, either on-going or is
> completed
> > > > (upon receiving the marker request from coordinator). When a produce
> > > > request is received, if
> > > >
> > > > * producer's epoch < cached epoch, or producer's epoch == cached
> epoch
> > > > but the latest txn is completed, leader directly reject with
> > > > invalid-epoch.
> > > > * producer's epoch > cached epoch, park the the request and send
> > > > add-partitions request to coordinator.
> > > >
> > > > In order to do it, does the coordinator need to bump the sequence and
> > > > reset epoch to 0 when the next epoch is going to overflow? If no need
> > > > to do so, then how we handle the (admittedly rare, but still may
> > > > happen) epoch overflow situation?
> > > >
> > > > 3. I'm a bit concerned about adding a generic "ABORTABLE_ERROR" given
> > > > we already have a pretty messy error classification and error
> handling
> > > > on the producer clients side --- I have a summary about the issues
> and
> > > > a proposal to address this in
> > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
> > > > -- I understand we do not want to use "UNKNOWN_PRODUCER_ID" anymore
> > > > and in fact we intend to deprecate it in KIP-360 and eventually
> remove
> > > > it; but I'm wondering can we still use specific error codes. E.g.
> what
> > > > about "InvalidProducerEpochException" since for new clients, the
> > > > actual reason this would actually be rejected is indeed because the
> > > > epoch on the coordinator caused the add-partitions-request from the
> > > > brokers to be rejected anyways?
> > > >
> > > > 4. It seems we put the producer request into purgatory before we ever
> > > > append the records, while other producer's records may still be
> > > > appended during the time; and that potentially may result in some
> > > > re-ordering compared with reception order. I'm not super concerned
> > > > about it since Kafka does not guarantee reception ordering across
> > > > producers anyways, but it may make the timestamps of records inside a
> > > > partition to be more out-of-ordered. Are we aware of any scenarios
> > > > such as future enhancements on log compactions that may be affected
> by
> > > > this effect?
> > > >
> > > > Below are just minor comments:
> > > >
> > > > 5. In "AddPartitionsToTxnTransaction" field of
> > > > "AddPartitionsToTxnRequest" RPC, the versions of those inner fields
> > > > are "0-3" while I thought they should be "0+" still?
> > > >
> > > > 6. Regarding "we can place the request in a purgatory of sorts and
> > > > check if there is any state for the transaction on the broker": i
> > > > think at this time when we just do the checks against the cached
> > > > state, we do not need to put the request to purgatory yet?
> > > >
> > > > 7. This is related to 3) above. I feel using "InvalidRecordException"
> > > > for older clients may also be a bit confusing, and also it is not
> > > > fatal -- for old clients, it better to be fatal since this indicates
> > > > the clients is doing something wrong and hence it should be closed.
> > > > And in general I'd prefer to use slightly more specific meaning error
> > > > codes for clients. That being said, I also feel
> > > > "InvalidProducerEpochException" is not suitable for old versioned
> > > > clients, and we'd have to pick one that old clients recognize. I'd
> > > > prefer "InvalidTxnStateException" but that one is supposed to be
> > > > returned from txn coordinators only today. I'd suggest we do a quick
> > > > check in the current client's code path and see if that one would be
> > > > handled if it's from a produce-response, and if yes, use this one;
> > > > otherwise, use "ProducerFencedException" which is much less
> meaningful
> > > > but it's still a fatal error.
> > > >
> > > >
> > > > Thanks,
> > > > Guozhang
> > > >
> > > >
> > > >
> > > > On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
> > > > <jo...@confluent.io.invalid> wrote:
> > > > >
> > > > > Yeah -- looks like we already have code to handle bumping the epoch
> > and
> > > > > when the epoch is Short.MAX_VALUE, we get a new producer ID. Since
> > this
> > > > is
> > > > > already the behavior, do we want to change it further?
> > > > >
> > > > > Justine
> > > > >
> > > > > On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <
> jolshan@confluent.io
> > >
> > > > wrote:
> > > > >
> > > > > > Hey all, just wanted to quickly update and say I've modified the
> > KIP to
> > > > > > explicitly mention that AddOffsetCommitsToTxnRequest will be
> > replaced
> > > > by
> > > > > > a coordinator-side (inter-broker) AddPartitionsToTxn implicit
> > request.
> > > > This
> > > > > > mirrors the user partitions and will implicitly add offset
> > partitions
> > > > to
> > > > > > transactions when we commit offsets on them. We will deprecate
> > > > AddOffsetCommitsToTxnRequest
> > > > > > for new clients.
> > > > > >
> > > > > > Also to address Artem's comments --
> > > > > > I'm a bit unsure if the changes here will change the previous
> > behavior
> > > > for
> > > > > > fencing producers. In the case you mention in the first
> paragraph,
> > are
> > > > you
> > > > > > saying we bump the epoch before we try to abort the transaction?
> I
> > > > think I
> > > > > > need to understand the scenarios you mention a bit better.
> > > > > >
> > > > > > As for the second part -- I think it makes sense to have some
> sort
> > of
> > > > > > "sentinel" epoch to signal epoch is about to overflow (I think we
> > sort
> > > > of
> > > > > > have this value in place in some ways) so we can codify it in the
> > KIP.
> > > > I'll
> > > > > > look into that and try to update soon.
> > > > > >
> > > > > > Thanks,
> > > > > > Justine.
> > > > > >
> > > > > > On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
> > > > > > <al...@confluent.io.invalid> wrote:
> > > > > >
> > > > > >> It's good to know that KIP-588 addressed some of the issues.
> > Looking
> > > > at
> > > > > >> the code, it still looks like there are some cases that would
> > result
> > > > in
> > > > > >> fatal error, e.g. PRODUCER_FENCED is issued by the transaction
> > > > coordinator
> > > > > >> if epoch doesn't match, and the client treats it as a fatal
> error
> > > > (code in
> > > > > >> TransactionManager request handling).  If we consider, for
> > example,
> > > > > >> committing a transaction that returns a timeout, but actually
> > > > succeeds,
> > > > > >> trying to abort it or re-commit may result in PRODUCER_FENCED
> > error
> > > > > >> (because of epoch bump).
> > > > > >>
> > > > > >> For failed commits, specifically, we need to know the actual
> > outcome,
> > > > > >> because if we return an error the application may think that the
> > > > > >> transaction is aborted and redo the work, leading to duplicates.
> > > > > >>
> > > > > >> Re: overflowing epoch.  We could either do it on the TC and
> return
> > > > both
> > > > > >> producer id and epoch (e.g. change the protocol), or signal the
> > client
> > > > > >> that
> > > > > >> it needs to get a new producer id.  Checking for max epoch could
> > be a
> > > > > >> reasonable signal, the value to check should probably be present
> > in
> > > > the
> > > > > >> KIP
> > > > > >> as this is effectively a part of the contract.  Also, the TC
> > should
> > > > > >> probably return an error if the client didn't change producer id
> > after
> > > > > >> hitting max epoch.
> > > > > >>
> > > > > >> -Artem
> > > > > >>
> > > > > >>
> > > > > >> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
> > > > > >> <jo...@confluent.io.invalid> wrote:
> > > > > >>
> > > > > >> > Thanks for the discussion Artem.
> > > > > >> >
> > > > > >> > With respect to the handling of fenced producers, we have some
> > > > behavior
> > > > > >> > already in place. As of KIP-588:
> > > > > >> >
> > > > > >> >
> > > > > >>
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> > > > > >> > ,
> > > > > >> > we handle timeouts more gracefully. The producer can recover.
> > > > > >> >
> > > > > >> > Produce requests can also recover from epoch fencing by
> > aborting the
> > > > > >> > transaction and starting over.
> > > > > >> >
> > > > > >> > What other cases were you considering that would cause us to
> > have a
> > > > > >> fenced
> > > > > >> > epoch but we'd want to recover?
> > > > > >> >
> > > > > >> > The first point about handling epoch overflows is fair. I
> think
> > > > there is
> > > > > >> > some logic we'd need to consider. (ie, if we are one away from
> > the
> > > > max
> > > > > >> > epoch, we need to reset the producer ID.) I'm still wondering
> if
> > > > there
> > > > > >> is a
> > > > > >> > way to direct this from the response, or if everything should
> be
> > > > done on
> > > > > >> > the client side. Let me know if you have any thoughts here.
> > > > > >> >
> > > > > >> > Thanks,
> > > > > >> > Justine
> > > > > >> >
> > > > > >> > On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
> > > > > >> > <al...@confluent.io.invalid> wrote:
> > > > > >> >
> > > > > >> > > There are some workflows in the client that are implied by
> > > > protocol
> > > > > >> > > changes, e.g.:
> > > > > >> > >
> > > > > >> > > - for new clients, epoch changes with every transaction and
> > can
> > > > > >> overflow,
> > > > > >> > > in old clients this condition was handled transparently,
> > because
> > > > epoch
> > > > > >> > was
> > > > > >> > > bumped in InitProducerId and it would return a new producer
> > id if
> > > > > >> epoch
> > > > > >> > > overflows, the new clients would need to implement some
> > workflow
> > > > to
> > > > > >> > refresh
> > > > > >> > > producer id
> > > > > >> > > - how to handle fenced producers, for new clients epoch
> > changes
> > > > with
> > > > > >> > every
> > > > > >> > > transaction, so in presence of failures during commits /
> > aborts,
> > > > the
> > > > > >> > > producer could get easily fenced, old clients would pretty
> > much
> > > > would
> > > > > >> get
> > > > > >> > > fenced when a new incarnation of the producer was
> initialized
> > with
> > > > > >> > > InitProducerId so it's ok to treat as a fatal error, the new
> > > > clients
> > > > > >> > would
> > > > > >> > > need to implement some workflow to handle that error,
> > otherwise
> > > > they
> > > > > >> > could
> > > > > >> > > get fenced by themselves
> > > > > >> > > - in particular (as a subset of the previous issue), what
> > would
> > > > the
> > > > > >> > client
> > > > > >> > > do if it got a timeout during commit?  commit could've
> > succeeded
> > > > or
> > > > > >> > failed
> > > > > >> > >
> > > > > >> > > Not sure if this has to be defined in the KIP as
> implementing
> > > > those
> > > > > >> > > probably wouldn't require protocol changes, but we have
> > multiple
> > > > > >> > > implementations of Kafka clients, so probably would be good
> to
> > > > have
> > > > > >> some
> > > > > >> > > client implementation guidance.  Could also be done as a
> > separate
> > > > doc.
> > > > > >> > >
> > > > > >> > > -Artem
> > > > > >> > >
> > > > > >> > > On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
> > > > > >> > <jolshan@confluent.io.invalid
> > > > > >> > > >
> > > > > >> > > wrote:
> > > > > >> > >
> > > > > >> > > > Hey all, I've updated the KIP to incorporate Jason's
> > > > suggestions.
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > > 1. Use AddPartitionsToTxn + verify flag to check on old
> > clients
> > > > > >> > > > 2. Updated AddPartitionsToTxn API to support transaction
> > > > batching
> > > > > >> > > > 3. Mention IBP bump
> > > > > >> > > > 4. Mention auth change on new AddPartitionsToTxn version.
> > > > > >> > > >
> > > > > >> > > > I'm planning on opening a vote soon.
> > > > > >> > > > Thanks,
> > > > > >> > > > Justine
> > > > > >> > > >
> > > > > >> > > > On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <
> > > > jolshan@confluent.io
> > > > > >> >
> > > > > >> > > > wrote:
> > > > > >> > > >
> > > > > >> > > > > Thanks Jason. Those changes make sense to me. I will
> > update
> > > > the
> > > > > >> KIP.
> > > > > >> > > > >
> > > > > >> > > > >
> > > > > >> > > > >
> > > > > >> > > > > On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
> > > > > >> > > > <ja...@confluent.io.invalid>
> > > > > >> > > > > wrote:
> > > > > >> > > > >
> > > > > >> > > > >> Hey Justine,
> > > > > >> > > > >>
> > > > > >> > > > >> > I was wondering about compatibility here. When we
> send
> > > > requests
> > > > > >> > > > >> between brokers, we want to ensure that the receiving
> > broker
> > > > > >> > > understands
> > > > > >> > > > >> the request (specifically the new fields). Typically
> > this is
> > > > done
> > > > > >> > via
> > > > > >> > > > >> IBP/metadata version.
> > > > > >> > > > >> I'm trying to think if there is a way around it but I'm
> > not
> > > > sure
> > > > > >> > there
> > > > > >> > > > is.
> > > > > >> > > > >>
> > > > > >> > > > >> Yes. I think we would gate usage of this behind an IBP
> > bump.
> > > > Does
> > > > > >> > that
> > > > > >> > > > >> seem
> > > > > >> > > > >> reasonable?
> > > > > >> > > > >>
> > > > > >> > > > >> > As for the improvements -- can you clarify how the
> > multiple
> > > > > >> > > > >> transactional
> > > > > >> > > > >> IDs would help here? Were you thinking of a case where
> we
> > > > > >> wait/batch
> > > > > >> > > > >> multiple produce requests together? My understanding
> for
> > now
> > > > was
> > > > > >> 1
> > > > > >> > > > >> transactional ID and one validation per 1 produce
> > request.
> > > > > >> > > > >>
> > > > > >> > > > >> Each call to `AddPartitionsToTxn` is essentially a
> write
> > to
> > > > the
> > > > > >> > > > >> transaction
> > > > > >> > > > >> log and must block on replication. The more we can fit
> > into a
> > > > > >> single
> > > > > >> > > > >> request, the more writes we can do in parallel. The
> > > > alternative
> > > > > >> is
> > > > > >> > to
> > > > > >> > > > make
> > > > > >> > > > >> use of more connections, but usually we prefer batching
> > > > since the
> > > > > >> > > > network
> > > > > >> > > > >> stack is not really optimized for high
> connection/request
> > > > loads.
> > > > > >> > > > >>
> > > > > >> > > > >> > Finally with respect to the authorizations, I think
> it
> > > > makes
> > > > > >> sense
> > > > > >> > > to
> > > > > >> > > > >> skip
> > > > > >> > > > >> topic authorizations, but I'm a bit confused by the
> > "leader
> > > > ID"
> > > > > >> > field.
> > > > > >> > > > >> Wouldn't we just want to flag the request as from a
> > broker
> > > > (does
> > > > > >> it
> > > > > >> > > > matter
> > > > > >> > > > >> which one?).
> > > > > >> > > > >>
> > > > > >> > > > >> We could also make it version-based. For the next
> > version, we
> > > > > >> could
> > > > > >> > > > >> require
> > > > > >> > > > >> CLUSTER auth. So clients would not be able to use the
> API
> > > > > >> anymore,
> > > > > >> > > which
> > > > > >> > > > >> is
> > > > > >> > > > >> probably what we want.
> > > > > >> > > > >>
> > > > > >> > > > >> -Jason
> > > > > >> > > > >>
> > > > > >> > > > >> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> > > > > >> > > > >> <jo...@confluent.io.invalid>
> > > > > >> > > > >> wrote:
> > > > > >> > > > >>
> > > > > >> > > > >> > As a follow up, I was just thinking about the
> batching
> > a
> > > > bit
> > > > > >> more.
> > > > > >> > > > >> > I suppose if we have one request in flight and we
> > queue up
> > > > the
> > > > > >> > other
> > > > > >> > > > >> > produce requests in some sort of purgatory, we could
> > send
> > > > > >> > > information
> > > > > >> > > > >> out
> > > > > >> > > > >> > for all of them rather than one by one. So that would
> > be a
> > > > > >> benefit
> > > > > >> > > of
> > > > > >> > > > >> > batching partitions to add per transaction.
> > > > > >> > > > >> >
> > > > > >> > > > >> > I'll need to think a bit more on the design of this
> > part
> > > > of the
> > > > > >> > KIP,
> > > > > >> > > > and
> > > > > >> > > > >> > will update the KIP in the next few days.
> > > > > >> > > > >> >
> > > > > >> > > > >> > Thanks,
> > > > > >> > > > >> > Justine
> > > > > >> > > > >> >
> > > > > >> > > > >> > On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan <
> > > > > >> > > jolshan@confluent.io>
> > > > > >> > > > >> > wrote:
> > > > > >> > > > >> >
> > > > > >> > > > >> > > Hey Jason -- thanks for the input -- I was just
> > digging
> > > > a bit
> > > > > >> > > deeper
> > > > > >> > > > >> into
> > > > > >> > > > >> > > the design + implementation of the validation calls
> > here
> > > > and
> > > > > >> > what
> > > > > >> > > > you
> > > > > >> > > > >> say
> > > > > >> > > > >> > > makes sense.
> > > > > >> > > > >> > >
> > > > > >> > > > >> > > I was wondering about compatibility here. When we
> > send
> > > > > >> requests
> > > > > >> > > > >> > > between brokers, we want to ensure that the
> receiving
> > > > broker
> > > > > >> > > > >> understands
> > > > > >> > > > >> > > the request (specifically the new fields).
> Typically
> > > > this is
> > > > > >> > done
> > > > > >> > > > via
> > > > > >> > > > >> > > IBP/metadata version.
> > > > > >> > > > >> > > I'm trying to think if there is a way around it but
> > I'm
> > > > not
> > > > > >> sure
> > > > > >> > > > there
> > > > > >> > > > >> > is.
> > > > > >> > > > >> > >
> > > > > >> > > > >> > > As for the improvements -- can you clarify how the
> > > > multiple
> > > > > >> > > > >> transactional
> > > > > >> > > > >> > > IDs would help here? Were you thinking of a case
> > where we
> > > > > >> > > wait/batch
> > > > > >> > > > >> > > multiple produce requests together? My
> understanding
> > for
> > > > now
> > > > > >> > was 1
> > > > > >> > > > >> > > transactional ID and one validation per 1 produce
> > > > request.
> > > > > >> > > > >> > >
> > > > > >> > > > >> > > Finally with respect to the authorizations, I think
> > it
> > > > makes
> > > > > >> > sense
> > > > > >> > > > to
> > > > > >> > > > >> > skip
> > > > > >> > > > >> > > topic authorizations, but I'm a bit confused by the
> > > > "leader
> > > > > >> ID"
> > > > > >> > > > field.
> > > > > >> > > > >> > > Wouldn't we just want to flag the request as from a
> > > > broker
> > > > > >> (does
> > > > > >> > > it
> > > > > >> > > > >> > matter
> > > > > >> > > > >> > > which one?).
> > > > > >> > > > >> > >
> > > > > >> > > > >> > > I think I want to adopt these suggestions, just had
> > a few
> > > > > >> > > questions
> > > > > >> > > > on
> > > > > >> > > > >> > the
> > > > > >> > > > >> > > details.
> > > > > >> > > > >> > >
> > > > > >> > > > >> > > Thanks,
> > > > > >> > > > >> > > Justine
> > > > > >> > > > >> > >
> > > > > >> > > > >> > > On Thu, Jan 5, 2023 at 5:05 PM Jason Gustafson
> > > > > >> > > > >> > <ja...@confluent.io.invalid>
> > > > > >> > > > >> > > wrote:
> > > > > >> > > > >> > >
> > > > > >> > > > >> > >> Hi Justine,
> > > > > >> > > > >> > >>
> > > > > >> > > > >> > >> Thanks for the proposal.
> > > > > >> > > > >> > >>
> > > > > >> > > > >> > >> I was thinking about the implementation a little
> > bit.
> > > > In the
> > > > > >> > > > current
> > > > > >> > > > >> > >> proposal, the behavior depends on whether we have
> an
> > > > old or
> > > > > >> new
> > > > > >> > > > >> client.
> > > > > >> > > > >> > >> For
> > > > > >> > > > >> > >> old clients, we send `DescribeTransactions` and
> > verify
> > > > the
> > > > > >> > result
> > > > > >> > > > and
> > > > > >> > > > >> > for
> > > > > >> > > > >> > >> new clients, we send `AddPartitionsToTxn`. We
> might
> > be
> > > > able
> > > > > >> to
> > > > > >> > > > >> simplify
> > > > > >> > > > >> > >> the
> > > > > >> > > > >> > >> implementation if we can use the same request
> type.
> > For
> > > > > >> > example,
> > > > > >> > > > >> what if
> > > > > >> > > > >> > >> we
> > > > > >> > > > >> > >> bump the protocol version for `AddPartitionsToTxn`
> > and
> > > > add a
> > > > > >> > > > >> > >> `validateOnly`
> > > > > >> > > > >> > >> flag? For older versions, we can set
> > > > `validateOnly=true` so
> > > > > >> > that
> > > > > >> > > > the
> > > > > >> > > > >> > >> request only returns successfully if the partition
> > had
> > > > > >> already
> > > > > >> > > been
> > > > > >> > > > >> > added.
> > > > > >> > > > >> > >> For new versions, we can set `validateOnly=false`
> > and
> > > > the
> > > > > >> > > partition
> > > > > >> > > > >> will
> > > > > >> > > > >> > >> be
> > > > > >> > > > >> > >> added to the transaction. The other slightly
> > annoying
> > > > thing
> > > > > >> > that
> > > > > >> > > > this
> > > > > >> > > > >> > >> would
> > > > > >> > > > >> > >> get around is the need to collect the transaction
> > state
> > > > for
> > > > > >> all
> > > > > >> > > > >> > partitions
> > > > > >> > > > >> > >> even when we only care about a subset.
> > > > > >> > > > >> > >>
> > > > > >> > > > >> > >> Some additional improvements to consider:
> > > > > >> > > > >> > >>
> > > > > >> > > > >> > >> - We can give `AddPartitionsToTxn` better batch
> > support
> > > > for
> > > > > >> > > > >> inter-broker
> > > > > >> > > > >> > >> usage. Currently we only allow one
> > `TransactionalId` to
> > > > be
> > > > > >> > > > specified,
> > > > > >> > > > >> > but
> > > > > >> > > > >> > >> the broker may get some benefit being able to
> batch
> > > > across
> > > > > >> > > multiple
> > > > > >> > > > >> > >> transactions.
> > > > > >> > > > >> > >> - Another small improvement is skipping topic
> > > > authorization
> > > > > >> > > checks
> > > > > >> > > > >> for
> > > > > >> > > > >> > >> `AddPartitionsToTxn` when the request is from a
> > broker.
> > > > > >> Perhaps
> > > > > >> > > we
> > > > > >> > > > >> can
> > > > > >> > > > >> > add
> > > > > >> > > > >> > >> a field for the `LeaderId` or something like that
> > and
> > > > > >> require
> > > > > >> > > > CLUSTER
> > > > > >> > > > >> > >> permission when set.
> > > > > >> > > > >> > >>
> > > > > >> > > > >> > >> Best,
> > > > > >> > > > >> > >> Jason
> > > > > >> > > > >> > >>
> > > > > >> > > > >> > >>
> > > > > >> > > > >> > >>
> > > > > >> > > > >> > >> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
> > > > > >> > <jun@confluent.io.invalid
> > > > > >> > > >
> > > > > >> > > > >> > wrote:
> > > > > >> > > > >> > >>
> > > > > >> > > > >> > >> > Hi, Justine,
> > > > > >> > > > >> > >> >
> > > > > >> > > > >> > >> > Thanks for the explanation. It makes sense to me
> > now.
> > > > > >> > > > >> > >> >
> > > > > >> > > > >> > >> > Jun
> > > > > >> > > > >> > >> >
> > > > > >> > > > >> > >> > On Mon, Dec 19, 2022 at 1:42 PM Justine Olshan
> > > > > >> > > > >> > >> > <jo...@confluent.io.invalid>
> > > > > >> > > > >> > >> > wrote:
> > > > > >> > > > >> > >> >
> > > > > >> > > > >> > >> > > Hi Jun,
> > > > > >> > > > >> > >> > >
> > > > > >> > > > >> > >> > > My understanding of the mechanism is that when
> > we
> > > > get to
> > > > > >> > the
> > > > > >> > > > last
> > > > > >> > > > >> > >> epoch,
> > > > > >> > > > >> > >> > we
> > > > > >> > > > >> > >> > > increment to the fencing/last epoch and if any
> > > > further
> > > > > >> > > requests
> > > > > >> > > > >> come
> > > > > >> > > > >> > >> in
> > > > > >> > > > >> > >> > for
> > > > > >> > > > >> > >> > > this producer ID they are fenced. Then the
> > producer
> > > > > >> gets a
> > > > > >> > > new
> > > > > >> > > > ID
> > > > > >> > > > >> > and
> > > > > >> > > > >> > >> > > restarts with epoch/sequence 0. The fenced
> epoch
> > > > sticks
> > > > > >> > > around
> > > > > >> > > > >> for
> > > > > >> > > > >> > the
> > > > > >> > > > >> > >> > > duration of producer.id.expiration.ms and
> > blocks
> > > > any
> > > > > >> late
> > > > > >> > > > >> messages
> > > > > >> > > > >> > >> > there.
> > > > > >> > > > >> > >> > > The new ID will get to take advantage of the
> > > > improved
> > > > > >> > > semantics
> > > > > >> > > > >> > around
> > > > > >> > > > >> > >> > > non-zero start sequences. So I think we are
> > covered.
> > > > > >> > > > >> > >> > >
> > > > > >> > > > >> > >> > > The only potential issue is overloading the
> > cache,
> > > > but
> > > > > >> > > > hopefully
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> > > improvements (lowered
> producer.id.expiration.ms
> > )
> > > > will
> > > > > >> help
> > > > > >> > > > with
> > > > > >> > > > >> > that.
> > > > > >> > > > >> > >> > Let
> > > > > >> > > > >> > >> > > me know if you still have concerns.
> > > > > >> > > > >> > >> > >
> > > > > >> > > > >> > >> > > Thanks,
> > > > > >> > > > >> > >> > > Justine
> > > > > >> > > > >> > >> > >
> > > > > >> > > > >> > >> > > On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
> > > > > >> > > > >> <ju...@confluent.io.invalid>
> > > > > >> > > > >> > >> > wrote:
> > > > > >> > > > >> > >> > >
> > > > > >> > > > >> > >> > > > Hi, Justine,
> > > > > >> > > > >> > >> > > >
> > > > > >> > > > >> > >> > > > Thanks for the explanation.
> > > > > >> > > > >> > >> > > >
> > > > > >> > > > >> > >> > > > 70. The proposed fencing logic doesn't apply
> > when
> > > > pid
> > > > > >> > > > changes,
> > > > > >> > > > >> is
> > > > > >> > > > >> > >> that
> > > > > >> > > > >> > >> > > > right? If so, I am not sure how complete we
> > are
> > > > > >> > addressing
> > > > > >> > > > this
> > > > > >> > > > >> > >> issue
> > > > > >> > > > >> > >> > if
> > > > > >> > > > >> > >> > > > the pid changes more frequently.
> > > > > >> > > > >> > >> > > >
> > > > > >> > > > >> > >> > > > Thanks,
> > > > > >> > > > >> > >> > > >
> > > > > >> > > > >> > >> > > > Jun
> > > > > >> > > > >> > >> > > >
> > > > > >> > > > >> > >> > > >
> > > > > >> > > > >> > >> > > >
> > > > > >> > > > >> > >> > > > On Fri, Dec 16, 2022 at 9:16 AM Justine
> Olshan
> > > > > >> > > > >> > >> > > > <jo...@confluent.io.invalid>
> > > > > >> > > > >> > >> > > > wrote:
> > > > > >> > > > >> > >> > > >
> > > > > >> > > > >> > >> > > > > Hi Jun,
> > > > > >> > > > >> > >> > > > >
> > > > > >> > > > >> > >> > > > > Thanks for replying!
> > > > > >> > > > >> > >> > > > >
> > > > > >> > > > >> > >> > > > > 70.We already do the overflow mechanism,
> so
> > my
> > > > > >> change
> > > > > >> > > would
> > > > > >> > > > >> just
> > > > > >> > > > >> > >> make
> > > > > >> > > > >> > >> > > it
> > > > > >> > > > >> > >> > > > > happen more often.
> > > > > >> > > > >> > >> > > > > I was also not suggesting a new field in
> the
> > > > log,
> > > > > >> but
> > > > > >> > in
> > > > > >> > > > the
> > > > > >> > > > >> > >> > response,
> > > > > >> > > > >> > >> > > > > which would be gated by the client
> version.
> > > > Sorry if
> > > > > >> > > > >> something
> > > > > >> > > > >> > >> there
> > > > > >> > > > >> > >> > is
> > > > > >> > > > >> > >> > > > > unclear. I think we are starting to
> diverge.
> > > > > >> > > > >> > >> > > > > The goal of this KIP is to not change to
> the
> > > > marker
> > > > > >> > > format
> > > > > >> > > > at
> > > > > >> > > > >> > all.
> > > > > >> > > > >> > >> > > > >
> > > > > >> > > > >> > >> > > > > 71. Yes, I guess I was going under the
> > > > assumption
> > > > > >> that
> > > > > >> > > the
> > > > > >> > > > >> log
> > > > > >> > > > >> > >> would
> > > > > >> > > > >> > >> > > just
> > > > > >> > > > >> > >> > > > > look at its last epoch and treat it as the
> > > > current
> > > > > >> > > epoch. I
> > > > > >> > > > >> > >> suppose
> > > > > >> > > > >> > >> > we
> > > > > >> > > > >> > >> > > > can
> > > > > >> > > > >> > >> > > > > have some special logic that if the last
> > epoch
> > > > was
> > > > > >> on a
> > > > > >> > > > >> marker
> > > > > >> > > > >> > we
> > > > > >> > > > >> > >> > > > actually
> > > > > >> > > > >> > >> > > > > expect the next epoch or something like
> > that. We
> > > > > >> just
> > > > > >> > > need
> > > > > >> > > > to
> > > > > >> > > > >> > >> > > distinguish
> > > > > >> > > > >> > >> > > > > based on whether we had a commit/abort
> > marker.
> > > > > >> > > > >> > >> > > > >
> > > > > >> > > > >> > >> > > > > 72.
> > > > > >> > > > >> > >> > > > > > if the producer epoch hasn't been bumped
> > on
> > > > the
> > > > > >> > > > >> > >> > > > > broker, it seems that the stucked message
> > will
> > > > fail
> > > > > >> the
> > > > > >> > > > >> sequence
> > > > > >> > > > >> > >> > > > validation
> > > > > >> > > > >> > >> > > > > and will be ignored. If the producer epoch
> > has
> > > > been
> > > > > >> > > bumped,
> > > > > >> > > > >> we
> > > > > >> > > > >> > >> ignore
> > > > > >> > > > >> > >> > > the
> > > > > >> > > > >> > >> > > > > sequence check and the stuck message could
> > be
> > > > > >> appended
> > > > > >> > to
> > > > > >> > > > the
> > > > > >> > > > >> > log.
> > > > > >> > > > >> > >> > So,
> > > > > >> > > > >> > >> > > is
> > > > > >> > > > >> > >> > > > > the latter case that we want to guard?
> > > > > >> > > > >> > >> > > > >
> > > > > >> > > > >> > >> > > > > I'm not sure I follow that "the message
> will
> > > > fail
> > > > > >> the
> > > > > >> > > > >> sequence
> > > > > >> > > > >> > >> > > > validation".
> > > > > >> > > > >> > >> > > > > In some of these cases, we had an abort
> > marker
> > > > (due
> > > > > >> to
> > > > > >> > an
> > > > > >> > > > >> error)
> > > > > >> > > > >> > >> and
> > > > > >> > > > >> > >> > > then
> > > > > >> > > > >> > >> > > > > the late message comes in with the correct
> > > > sequence
> > > > > >> > > number.
> > > > > >> > > > >> This
> > > > > >> > > > >> > >> is a
> > > > > >> > > > >> > >> > > > case
> > > > > >> > > > >> > >> > > > > covered by the KIP.
> > > > > >> > > > >> > >> > > > > The latter case is actually not something
> > we've
> > > > > >> > > considered
> > > > > >> > > > >> > here. I
> > > > > >> > > > >> > >> > > think
> > > > > >> > > > >> > >> > > > > generally when we bump the epoch, we are
> > > > accepting
> > > > > >> that
> > > > > >> > > the
> > > > > >> > > > >> > >> sequence
> > > > > >> > > > >> > >> > > does
> > > > > >> > > > >> > >> > > > > not need to be checked anymore. My
> > > > understanding is
> > > > > >> > also
> > > > > >> > > > >> that we
> > > > > >> > > > >> > >> > don't
> > > > > >> > > > >> > >> > > > > typically bump epoch mid transaction
> (based
> > on a
> > > > > >> quick
> > > > > >> > > look
> > > > > >> > > > >> at
> > > > > >> > > > >> > the
> > > > > >> > > > >> > >> > > code)
> > > > > >> > > > >> > >> > > > > but let me know if that is the case.
> > > > > >> > > > >> > >> > > > >
> > > > > >> > > > >> > >> > > > > Thanks,
> > > > > >> > > > >> > >> > > > > Justine
> > > > > >> > > > >> > >> > > > >
> > > > > >> > > > >> > >> > > > > On Thu, Dec 15, 2022 at 12:23 PM Jun Rao
> > > > > >> > > > >> > <jun@confluent.io.invalid
> > > > > >> > > > >> > >> >
> > > > > >> > > > >> > >> > > > wrote:
> > > > > >> > > > >> > >> > > > >
> > > > > >> > > > >> > >> > > > > > Hi, Justine,
> > > > > >> > > > >> > >> > > > > >
> > > > > >> > > > >> > >> > > > > > Thanks for the reply.
> > > > > >> > > > >> > >> > > > > >
> > > > > >> > > > >> > >> > > > > > 70. Assigning a new pid on int overflow
> > seems
> > > > a
> > > > > >> bit
> > > > > >> > > > hacky.
> > > > > >> > > > >> If
> > > > > >> > > > >> > we
> > > > > >> > > > >> > >> > > need a
> > > > > >> > > > >> > >> > > > > txn
> > > > > >> > > > >> > >> > > > > > level id, it will be better to model
> this
> > > > > >> explicitly.
> > > > > >> > > > >> Adding a
> > > > > >> > > > >> > >> new
> > > > > >> > > > >> > >> > > > field
> > > > > >> > > > >> > >> > > > > > would require a bit more work since it
> > > > requires a
> > > > > >> new
> > > > > >> > > txn
> > > > > >> > > > >> > marker
> > > > > >> > > > >> > >> > > format
> > > > > >> > > > >> > >> > > > > in
> > > > > >> > > > >> > >> > > > > > the log. So, we probably need to guard
> it
> > > > with an
> > > > > >> IBP
> > > > > >> > > or
> > > > > >> > > > >> > >> metadata
> > > > > >> > > > >> > >> > > > version
> > > > > >> > > > >> > >> > > > > > and document the impact on downgrade
> once
> > the
> > > > new
> > > > > >> > > format
> > > > > >> > > > is
> > > > > >> > > > >> > >> written
> > > > > >> > > > >> > >> > > to
> > > > > >> > > > >> > >> > > > > the
> > > > > >> > > > >> > >> > > > > > log.
> > > > > >> > > > >> > >> > > > > >
> > > > > >> > > > >> > >> > > > > > 71. Hmm, once the marker is written, the
> > > > partition
> > > > > >> > will
> > > > > >> > > > >> expect
> > > > > >> > > > >> > >> the
> > > > > >> > > > >> > >> > > next
> > > > > >> > > > >> > >> > > > > > append to be on the next epoch. Does
> that
> > > > cover
> > > > > >> the
> > > > > >> > > case
> > > > > >> > > > >> you
> > > > > >> > > > >> > >> > > mentioned?
> > > > > >> > > > >> > >> > > > > >
> > > > > >> > > > >> > >> > > > > > 72. Also, just to be clear on the
> stucked
> > > > message
> > > > > >> > issue
> > > > > >> > > > >> > >> described
> > > > > >> > > > >> > >> > in
> > > > > >> > > > >> > >> > > > the
> > > > > >> > > > >> > >> > > > > > motivation. With EoS, we also validate
> the
> > > > > >> sequence
> > > > > >> > id
> > > > > >> > > > for
> > > > > >> > > > >> > >> > > idempotency.
> > > > > >> > > > >> > >> > > > > So,
> > > > > >> > > > >> > >> > > > > > with the current logic, if the producer
> > epoch
> > > > > >> hasn't
> > > > > >> > > been
> > > > > >> > > > >> > >> bumped on
> > > > > >> > > > >> > >> > > the
> > > > > >> > > > >> > >> > > > > > broker, it seems that the stucked
> message
> > will
> > > > > >> fail
> > > > > >> > the
> > > > > >> > > > >> > sequence
> > > > > >> > > > >> > >> > > > > validation
> > > > > >> > > > >> > >> > > > > > and will be ignored. If the producer
> > epoch has
> > > > > >> been
> > > > > >> > > > >> bumped, we
> > > > > >> > > > >> > >> > ignore
> > > > > >> > > > >> > >> > > > the
> > > > > >> > > > >> > >> > > > > > sequence check and the stuck message
> > could be
> > > > > >> > appended
> > > > > >> > > to
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> log.
> > > > > >> > > > >> > >> > > So,
> > > > > >> > > > >> > >> > > > is
> > > > > >> > > > >> > >> > > > > > the latter case that we want to guard?
> > > > > >> > > > >> > >> > > > > >
> > > > > >> > > > >> > >> > > > > > Thanks,
> > > > > >> > > > >> > >> > > > > >
> > > > > >> > > > >> > >> > > > > > Jun
> > > > > >> > > > >> > >> > > > > >
> > > > > >> > > > >> > >> > > > > > On Wed, Dec 14, 2022 at 10:44 AM Justine
> > > > Olshan
> > > > > >> > > > >> > >> > > > > > <jo...@confluent.io.invalid> wrote:
> > > > > >> > > > >> > >> > > > > >
> > > > > >> > > > >> > >> > > > > > > Matthias — thanks again for taking
> time
> > to
> > > > look
> > > > > >> a
> > > > > >> > > this.
> > > > > >> > > > >> You
> > > > > >> > > > >> > >> said:
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > > My proposal was only focusing to
> avoid
> > > > > >> dangling
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > transactions if records are added
> > without
> > > > > >> > registered
> > > > > >> > > > >> > >> partition.
> > > > > >> > > > >> > >> > --
> > > > > >> > > > >> > >> > > > > Maybe
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > you can add a few more details to the
> > KIP
> > > > about
> > > > > >> > this
> > > > > >> > > > >> > scenario
> > > > > >> > > > >> > >> for
> > > > > >> > > > >> > >> > > > > better
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > documentation purpose?
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > I'm not sure I understand what you
> mean
> > > > here.
> > > > > >> The
> > > > > >> > > > >> motivation
> > > > > >> > > > >> > >> > > section
> > > > > >> > > > >> > >> > > > > > > describes two scenarios about how the
> > record
> > > > > >> can be
> > > > > >> > > > added
> > > > > >> > > > >> > >> > without a
> > > > > >> > > > >> > >> > > > > > > registered partition:
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > > 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.
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > > Another way hanging transactions can
> > > > occur is
> > > > > >> > that
> > > > > >> > > a
> > > > > >> > > > >> > client
> > > > > >> > > > >> > >> is
> > > > > >> > > > >> > >> > > > buggy
> > > > > >> > > > >> > >> > > > > > and
> > > > > >> > > > >> > >> > > > > > > may somehow try to write to a
> partition
> > > > before
> > > > > >> it
> > > > > >> > > adds
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> > > partition
> > > > > >> > > > >> > >> > > > to
> > > > > >> > > > >> > >> > > > > > the
> > > > > >> > > > >> > >> > > > > > > transaction.
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > For the first example of this would it
> > be
> > > > > >> helpful
> > > > > >> > to
> > > > > >> > > > say
> > > > > >> > > > >> > that
> > > > > >> > > > >> > >> > this
> > > > > >> > > > >> > >> > > > > > message
> > > > > >> > > > >> > >> > > > > > > comes in after the abort, but before
> the
> > > > > >> partition
> > > > > >> > is
> > > > > >> > > > >> added
> > > > > >> > > > >> > to
> > > > > >> > > > >> > >> > the
> > > > > >> > > > >> > >> > > > next
> > > > > >> > > > >> > >> > > > > > > transaction so it becomes "hanging."
> > > > Perhaps the
> > > > > >> > next
> > > > > >> > > > >> > sentence
> > > > > >> > > > >> > >> > > > > describing
> > > > > >> > > > >> > >> > > > > > > the message becoming part of the next
> > > > > >> transaction
> > > > > >> > (a
> > > > > >> > > > >> > different
> > > > > >> > > > >> > >> > > case)
> > > > > >> > > > >> > >> > > > > was
> > > > > >> > > > >> > >> > > > > > > not properly differentiated.
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > Jun — thanks for reading the KIP.
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > 70. The int typing was a concern.
> > Currently
> > > > we
> > > > > >> > have a
> > > > > >> > > > >> > >> mechanism
> > > > > >> > > > >> > >> > in
> > > > > >> > > > >> > >> > > > > place
> > > > > >> > > > >> > >> > > > > > to
> > > > > >> > > > >> > >> > > > > > > fence the final epoch when the epoch
> is
> > > > about to
> > > > > >> > > > overflow
> > > > > >> > > > >> > and
> > > > > >> > > > >> > >> > > assign
> > > > > >> > > > >> > >> > > > a
> > > > > >> > > > >> > >> > > > > > new
> > > > > >> > > > >> > >> > > > > > > producer ID with epoch 0. Of course,
> > this
> > > > is a
> > > > > >> bit
> > > > > >> > > > tricky
> > > > > >> > > > >> > >> when it
> > > > > >> > > > >> > >> > > > comes
> > > > > >> > > > >> > >> > > > > > to
> > > > > >> > > > >> > >> > > > > > > the response back to the client.
> > > > > >> > > > >> > >> > > > > > > Making this a long could be another
> > option,
> > > > but
> > > > > >> I
> > > > > >> > > > wonder
> > > > > >> > > > >> are
> > > > > >> > > > >> > >> > there
> > > > > >> > > > >> > >> > > > any
> > > > > >> > > > >> > >> > > > > > > implications on changing this field if
> > the
> > > > > >> epoch is
> > > > > >> > > > >> > persisted
> > > > > >> > > > >> > >> to
> > > > > >> > > > >> > >> > > > disk?
> > > > > >> > > > >> > >> > > > > > I'd
> > > > > >> > > > >> > >> > > > > > > need to check the usages.
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > 71.This was something Matthias asked
> > about
> > > > as
> > > > > >> > well. I
> > > > > >> > > > was
> > > > > >> > > > >> > >> > > > considering a
> > > > > >> > > > >> > >> > > > > > > possible edge case where a produce
> > request
> > > > from
> > > > > >> a
> > > > > >> > new
> > > > > >> > > > >> > >> transaction
> > > > > >> > > > >> > >> > > > > somehow
> > > > > >> > > > >> > >> > > > > > > gets sent right after the marker is
> > > > written, but
> > > > > >> > > before
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> > > producer
> > > > > >> > > > >> > >> > > > is
> > > > > >> > > > >> > >> > > > > > > alerted of the newly bumped epoch. In
> > this
> > > > > >> case, we
> > > > > >> > > may
> > > > > >> > > > >> > >> include
> > > > > >> > > > >> > >> > > this
> > > > > >> > > > >> > >> > > > > > record
> > > > > >> > > > >> > >> > > > > > > when we don't want to. I suppose we
> > could
> > > > try
> > > > > >> to do
> > > > > >> > > > >> > something
> > > > > >> > > > >> > >> > > client
> > > > > >> > > > >> > >> > > > > side
> > > > > >> > > > >> > >> > > > > > > to bump the epoch after sending an
> > endTxn as
> > > > > >> well
> > > > > >> > in
> > > > > >> > > > this
> > > > > >> > > > >> > >> > scenario
> > > > > >> > > > >> > >> > > —
> > > > > >> > > > >> > >> > > > > but
> > > > > >> > > > >> > >> > > > > > I
> > > > > >> > > > >> > >> > > > > > > wonder how it would work when the
> > server is
> > > > > >> > aborting
> > > > > >> > > > >> based
> > > > > >> > > > >> > on
> > > > > >> > > > >> > >> a
> > > > > >> > > > >> > >> > > > > > server-side
> > > > > >> > > > >> > >> > > > > > > error. I could also be missing
> > something and
> > > > > >> this
> > > > > >> > > > >> scenario
> > > > > >> > > > >> > is
> > > > > >> > > > >> > >> > > > actually
> > > > > >> > > > >> > >> > > > > > not
> > > > > >> > > > >> > >> > > > > > > possible.
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > Thanks again to everyone reading and
> > > > commenting.
> > > > > >> > Let
> > > > > >> > > me
> > > > > >> > > > >> know
> > > > > >> > > > >> > >> > about
> > > > > >> > > > >> > >> > > > any
> > > > > >> > > > >> > >> > > > > > > further questions or comments.
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > Justine
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > On Wed, Dec 14, 2022 at 9:41 AM Jun
> Rao
> > > > > >> > > > >> > >> <jun@confluent.io.invalid
> > > > > >> > > > >> > >> > >
> > > > > >> > > > >> > >> > > > > > wrote:
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > > Hi, Justine,
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > Thanks for the KIP. A couple of
> > comments.
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > 70. Currently, the producer epoch is
> > an
> > > > int.
> > > > > >> I am
> > > > > >> > > not
> > > > > >> > > > >> sure
> > > > > >> > > > >> > >> if
> > > > > >> > > > >> > >> > > it's
> > > > > >> > > > >> > >> > > > > > enough
> > > > > >> > > > >> > >> > > > > > > > to accommodate all transactions in
> the
> > > > > >> lifetime
> > > > > >> > of
> > > > > >> > > a
> > > > > >> > > > >> > >> producer.
> > > > > >> > > > >> > >> > > > Should
> > > > > >> > > > >> > >> > > > > > we
> > > > > >> > > > >> > >> > > > > > > > change that to a long or add a new
> > long
> > > > field
> > > > > >> > like
> > > > > >> > > > >> txnId?
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > 71. "it will write the prepare
> commit
> > > > message
> > > > > >> > with
> > > > > >> > > a
> > > > > >> > > > >> > bumped
> > > > > >> > > > >> > >> > epoch
> > > > > >> > > > >> > >> > > > and
> > > > > >> > > > >> > >> > > > > > > send
> > > > > >> > > > >> > >> > > > > > > > WriteTxnMarkerRequests with the
> bumped
> > > > epoch."
> > > > > >> > Hmm,
> > > > > >> > > > the
> > > > > >> > > > >> > >> epoch
> > > > > >> > > > >> > >> > is
> > > > > >> > > > >> > >> > > > > > > associated
> > > > > >> > > > >> > >> > > > > > > > with the current txn right? So, it
> > seems
> > > > > >> weird to
> > > > > >> > > > >> write a
> > > > > >> > > > >> > >> > commit
> > > > > >> > > > >> > >> > > > > > message
> > > > > >> > > > >> > >> > > > > > > > with a bumped epoch. Should we only
> > bump
> > > > up
> > > > > >> the
> > > > > >> > > epoch
> > > > > >> > > > >> in
> > > > > >> > > > >> > >> > > > > EndTxnResponse
> > > > > >> > > > >> > >> > > > > > > and
> > > > > >> > > > >> > >> > > > > > > > rename the field to sth like
> > > > > >> nextProducerEpoch?
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > Thanks,
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > Jun
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > On Mon, Dec 12, 2022 at 8:54 PM
> > Matthias
> > > > J.
> > > > > >> Sax <
> > > > > >> > > > >> > >> > > mjsax@apache.org>
> > > > > >> > > > >> > >> > > > > > > wrote:
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > Thanks for the background.
> > > > > >> > > > >> > >> > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > 20/30: SGTM. My proposal was only
> > > > focusing
> > > > > >> to
> > > > > >> > > avoid
> > > > > >> > > > >> > >> dangling
> > > > > >> > > > >> > >> > > > > > > > > transactions if records are added
> > > > without
> > > > > >> > > > registered
> > > > > >> > > > >> > >> > partition.
> > > > > >> > > > >> > >> > > > --
> > > > > >> > > > >> > >> > > > > > > Maybe
> > > > > >> > > > >> > >> > > > > > > > > you can add a few more details to
> > the
> > > > KIP
> > > > > >> about
> > > > > >> > > > this
> > > > > >> > > > >> > >> scenario
> > > > > >> > > > >> > >> > > for
> > > > > >> > > > >> > >> > > > > > > better
> > > > > >> > > > >> > >> > > > > > > > > documentation purpose?
> > > > > >> > > > >> > >> > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > 40: I think you hit a fair point
> > about
> > > > race
> > > > > >> > > > >> conditions
> > > > > >> > > > >> > or
> > > > > >> > > > >> > >> > > client
> > > > > >> > > > >> > >> > > > > bugs
> > > > > >> > > > >> > >> > > > > > > > > (incorrectly not bumping the
> > epoch). The
> > > > > >> > > > >> > >> complexity/confusion
> > > > > >> > > > >> > >> > > for
> > > > > >> > > > >> > >> > > > > > using
> > > > > >> > > > >> > >> > > > > > > > > the bumped epoch I see, is mainly
> > for
> > > > > >> internal
> > > > > >> > > > >> > debugging,
> > > > > >> > > > >> > >> ie,
> > > > > >> > > > >> > >> > > > > > > inspecting
> > > > > >> > > > >> > >> > > > > > > > > log segment dumps -- it seems
> > harder to
> > > > > >> reason
> > > > > >> > > > about
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> > system
> > > > > >> > > > >> > >> > > > for
> > > > > >> > > > >> > >> > > > > > us
> > > > > >> > > > >> > >> > > > > > > > > humans. But if we get better
> > > > guarantees, it
> > > > > >> > would
> > > > > >> > > > be
> > > > > >> > > > >> > >> worth to
> > > > > >> > > > >> > >> > > use
> > > > > >> > > > >> > >> > > > > the
> > > > > >> > > > >> > >> > > > > > > > > bumped epoch.
> > > > > >> > > > >> > >> > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > 60: as I mentioned already, I
> don't
> > > > know the
> > > > > >> > > broker
> > > > > >> > > > >> > >> internals
> > > > > >> > > > >> > >> > > to
> > > > > >> > > > >> > >> > > > > > > provide
> > > > > >> > > > >> > >> > > > > > > > > more input. So if nobody else
> chimes
> > > > in, we
> > > > > >> > > should
> > > > > >> > > > >> just
> > > > > >> > > > >> > >> move
> > > > > >> > > > >> > >> > > > > forward
> > > > > >> > > > >> > >> > > > > > > > > with your proposal.
> > > > > >> > > > >> > >> > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > -Matthias
> > > > > >> > > > >> > >> > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > On 12/6/22 4:22 PM, Justine Olshan
> > > > wrote:
> > > > > >> > > > >> > >> > > > > > > > > > Hi all,
> > > > > >> > > > >> > >> > > > > > > > > > After Artem's questions about
> > error
> > > > > >> behavior,
> > > > > >> > > > I've
> > > > > >> > > > >> > >> > > re-evaluated
> > > > > >> > > > >> > >> > > > > the
> > > > > >> > > > >> > >> > > > > > > > > > unknown producer ID exception
> and
> > had
> > > > some
> > > > > >> > > > >> discussions
> > > > > >> > > > >> > >> > > offline.
> > > > > >> > > > >> > >> > > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > > I think generally it makes sense
> > to
> > > > > >> simplify
> > > > > >> > > > error
> > > > > >> > > > >> > >> handling
> > > > > >> > > > >> > >> > > in
> > > > > >> > > > >> > >> > > > > > cases
> > > > > >> > > > >> > >> > > > > > > > like
> > > > > >> > > > >> > >> > > > > > > > > > this and the UNKNOWN_PRODUCER_ID
> > error
> > > > > >> has a
> > > > > >> > > > pretty
> > > > > >> > > > >> > long
> > > > > >> > > > >> > >> > and
> > > > > >> > > > >> > >> > > > > > > > complicated
> > > > > >> > > > >> > >> > > > > > > > > > history. Because of this, I
> > propose
> > > > > >> adding a
> > > > > >> > > new
> > > > > >> > > > >> error
> > > > > >> > > > >> > >> code
> > > > > >> > > > >> > >> > > > > > > > > ABORTABLE_ERROR
> > > > > >> > > > >> > >> > > > > > > > > > that when encountered by new
> > clients
> > > > > >> (gated
> > > > > >> > by
> > > > > >> > > > the
> > > > > >> > > > >> > >> produce
> > > > > >> > > > >> > >> > > > > request
> > > > > >> > > > >> > >> > > > > > > > > version)
> > > > > >> > > > >> > >> > > > > > > > > > will simply abort the
> transaction.
> > > > This
> > > > > >> > allows
> > > > > >> > > > the
> > > > > >> > > > >> > >> server
> > > > > >> > > > >> > >> > to
> > > > > >> > > > >> > >> > > > have
> > > > > >> > > > >> > >> > > > > > > some
> > > > > >> > > > >> > >> > > > > > > > > say
> > > > > >> > > > >> > >> > > > > > > > > > in whether the client aborts and
> > makes
> > > > > >> > handling
> > > > > >> > > > >> much
> > > > > >> > > > >> > >> > simpler.
> > > > > >> > > > >> > >> > > > In
> > > > > >> > > > >> > >> > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > > future, we can also use this
> > error in
> > > > > >> other
> > > > > >> > > > >> situations
> > > > > >> > > > >> > >> > where
> > > > > >> > > > >> > >> > > we
> > > > > >> > > > >> > >> > > > > > want
> > > > > >> > > > >> > >> > > > > > > to
> > > > > >> > > > >> > >> > > > > > > > > > abort the transactions. We can
> > even
> > > > use on
> > > > > >> > > other
> > > > > >> > > > >> apis.
> > > > > >> > > > >> > >> > > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > > I've added this to the KIP. Let
> me
> > > > know if
> > > > > >> > > there
> > > > > >> > > > >> are
> > > > > >> > > > >> > any
> > > > > >> > > > >> > >> > > > > questions
> > > > > >> > > > >> > >> > > > > > or
> > > > > >> > > > >> > >> > > > > > > > > > issues.
> > > > > >> > > > >> > >> > > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > > Justine
> > > > > >> > > > >> > >> > > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > > On Fri, Dec 2, 2022 at 10:22 AM
> > > > Justine
> > > > > >> > Olshan
> > > > > >> > > <
> > > > > >> > > > >> > >> > > > > > jolshan@confluent.io
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > wrote:
> > > > > >> > > > >> > >> > > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > >> Hey Matthias,
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >> 20/30 — Maybe I also didn't
> > express
> > > > > >> myself
> > > > > >> > > > >> clearly.
> > > > > >> > > > >> > For
> > > > > >> > > > >> > >> > > older
> > > > > >> > > > >> > >> > > > > > > clients
> > > > > >> > > > >> > >> > > > > > > > we
> > > > > >> > > > >> > >> > > > > > > > > >> don't have a way to distinguish
> > > > between a
> > > > > >> > > > previous
> > > > > >> > > > >> > and
> > > > > >> > > > >> > >> the
> > > > > >> > > > >> > >> > > > > current
> > > > > >> > > > >> > >> > > > > > > > > >> transaction since we don't have
> > the
> > > > epoch
> > > > > >> > > bump.
> > > > > >> > > > >> This
> > > > > >> > > > >> > >> means
> > > > > >> > > > >> > >> > > > that
> > > > > >> > > > >> > >> > > > > a
> > > > > >> > > > >> > >> > > > > > > late
> > > > > >> > > > >> > >> > > > > > > > > >> message from the previous
> > transaction
> > > > > >> may be
> > > > > >> > > > >> added to
> > > > > >> > > > >> > >> the
> > > > > >> > > > >> > >> > > new
> > > > > >> > > > >> > >> > > > > one.
> > > > > >> > > > >> > >> > > > > > > > With
> > > > > >> > > > >> > >> > > > > > > > > >> older clients — we can't
> > guarantee
> > > > this
> > > > > >> > won't
> > > > > >> > > > >> happen
> > > > > >> > > > >> > >> if we
> > > > > >> > > > >> > >> > > > > already
> > > > > >> > > > >> > >> > > > > > > > sent
> > > > > >> > > > >> > >> > > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call (why we
> > make
> > > > > >> changes
> > > > > >> > > for
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> newer
> > > > > >> > > > >> > >> > > > > client)
> > > > > >> > > > >> > >> > > > > > > but
> > > > > >> > > > >> > >> > > > > > > > > we
> > > > > >> > > > >> > >> > > > > > > > > >> can at least gate some by
> > ensuring
> > > > that
> > > > > >> the
> > > > > >> > > > >> partition
> > > > > >> > > > >> > >> has
> > > > > >> > > > >> > >> > > been
> > > > > >> > > > >> > >> > > > > > added
> > > > > >> > > > >> > >> > > > > > > > to
> > > > > >> > > > >> > >> > > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >> transaction. The rationale here
> > is
> > > > that
> > > > > >> > there
> > > > > >> > > > are
> > > > > >> > > > >> > >> likely
> > > > > >> > > > >> > >> > > LESS
> > > > > >> > > > >> > >> > > > > late
> > > > > >> > > > >> > >> > > > > > > > > arrivals
> > > > > >> > > > >> > >> > > > > > > > > >> as time goes on, so hopefully
> > most
> > > > late
> > > > > >> > > arrivals
> > > > > >> > > > >> will
> > > > > >> > > > >> > >> come
> > > > > >> > > > >> > >> > > in
> > > > > >> > > > >> > >> > > > > > BEFORE
> > > > > >> > > > >> > >> > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call. Those
> > that
> > > > > >> arrive
> > > > > >> > > > before
> > > > > >> > > > >> > will
> > > > > >> > > > >> > >> be
> > > > > >> > > > >> > >> > > > > properly
> > > > > >> > > > >> > >> > > > > > > > gated
> > > > > >> > > > >> > >> > > > > > > > > >> with the describeTransactions
> > > > approach.
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >> If we take the approach you
> > > > suggested,
> > > > > >> ANY
> > > > > >> > > late
> > > > > >> > > > >> > arrival
> > > > > >> > > > >> > >> > > from a
> > > > > >> > > > >> > >> > > > > > > > previous
> > > > > >> > > > >> > >> > > > > > > > > >> transaction will be added. And
> we
> > > > don't
> > > > > >> want
> > > > > >> > > > >> that. I
> > > > > >> > > > >> > >> also
> > > > > >> > > > >> > >> > > > don't
> > > > > >> > > > >> > >> > > > > > see
> > > > > >> > > > >> > >> > > > > > > > any
> > > > > >> > > > >> > >> > > > > > > > > >> benefit in sending
> > addPartitionsToTxn
> > > > > >> over
> > > > > >> > the
> > > > > >> > > > >> > >> > describeTxns
> > > > > >> > > > >> > >> > > > > call.
> > > > > >> > > > >> > >> > > > > > > They
> > > > > >> > > > >> > >> > > > > > > > > will
> > > > > >> > > > >> > >> > > > > > > > > >> both be one extra RPC to the
> Txn
> > > > > >> > coordinator.
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >> To be clear — newer clients
> will
> > use
> > > > > >> > > > >> > addPartitionsToTxn
> > > > > >> > > > >> > >> > > > instead
> > > > > >> > > > >> > >> > > > > of
> > > > > >> > > > >> > >> > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >> DescribeTxns.
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >> 40)
> > > > > >> > > > >> > >> > > > > > > > > >> My concern is that if we have
> > some
> > > > delay
> > > > > >> in
> > > > > >> > > the
> > > > > >> > > > >> > client
> > > > > >> > > > >> > >> to
> > > > > >> > > > >> > >> > > bump
> > > > > >> > > > >> > >> > > > > the
> > > > > >> > > > >> > >> > > > > > > > > epoch,
> > > > > >> > > > >> > >> > > > > > > > > >> it could continue to send epoch
> > 73
> > > > and
> > > > > >> those
> > > > > >> > > > >> records
> > > > > >> > > > >> > >> would
> > > > > >> > > > >> > >> > > not
> > > > > >> > > > >> > >> > > > > be
> > > > > >> > > > >> > >> > > > > > > > > fenced.
> > > > > >> > > > >> > >> > > > > > > > > >> Perhaps this is not an issue if
> > we
> > > > don't
> > > > > >> > allow
> > > > > >> > > > the
> > > > > >> > > > >> > next
> > > > > >> > > > >> > >> > > > produce
> > > > > >> > > > >> > >> > > > > to
> > > > > >> > > > >> > >> > > > > > > go
> > > > > >> > > > >> > >> > > > > > > > > >> through before the EndTxn
> request
> > > > > >> returns.
> > > > > >> > I'm
> > > > > >> > > > >> also
> > > > > >> > > > >> > >> > thinking
> > > > > >> > > > >> > >> > > > > about
> > > > > >> > > > >> > >> > > > > > > > > cases of
> > > > > >> > > > >> > >> > > > > > > > > >> failure. I will need to think
> on
> > > > this a
> > > > > >> bit.
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >> I wasn't sure if it was that
> > > > confusing.
> > > > > >> But
> > > > > >> > if
> > > > > >> > > > we
> > > > > >> > > > >> > >> think it
> > > > > >> > > > >> > >> > > is,
> > > > > >> > > > >> > >> > > > > we
> > > > > >> > > > >> > >> > > > > > > can
> > > > > >> > > > >> > >> > > > > > > > > >> investigate other ways.
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >> 60)
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >> I'm not sure these are the same
> > > > > >> purgatories
> > > > > >> > > > since
> > > > > >> > > > >> one
> > > > > >> > > > >> > >> is a
> > > > > >> > > > >> > >> > > > > produce
> > > > > >> > > > >> > >> > > > > > > > > >> purgatory (I was planning on
> > using a
> > > > > >> > callback
> > > > > >> > > > >> rather
> > > > > >> > > > >> > >> than
> > > > > >> > > > >> > >> > > > > > purgatory)
> > > > > >> > > > >> > >> > > > > > > > and
> > > > > >> > > > >> > >> > > > > > > > > >> the other is simply a request
> to
> > > > append
> > > > > >> to
> > > > > >> > the
> > > > > >> > > > >> log.
> > > > > >> > > > >> > Not
> > > > > >> > > > >> > >> > sure
> > > > > >> > > > >> > >> > > > we
> > > > > >> > > > >> > >> > > > > > have
> > > > > >> > > > >> > >> > > > > > > > any
> > > > > >> > > > >> > >> > > > > > > > > >> structure here for ordering,
> but
> > my
> > > > > >> > > > understanding
> > > > > >> > > > >> is
> > > > > >> > > > >> > >> that
> > > > > >> > > > >> > >> > > the
> > > > > >> > > > >> > >> > > > > > broker
> > > > > >> > > > >> > >> > > > > > > > > could
> > > > > >> > > > >> > >> > > > > > > > > >> handle the write request before
> > it
> > > > hears
> > > > > >> > back
> > > > > >> > > > from
> > > > > >> > > > >> > the
> > > > > >> > > > >> > >> Txn
> > > > > >> > > > >> > >> > > > > > > > Coordinator.
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >> Let me know if I misunderstood
> > > > something
> > > > > >> or
> > > > > >> > > > >> something
> > > > > >> > > > >> > >> was
> > > > > >> > > > >> > >> > > > > unclear.
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >> Justine
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >> On Thu, Dec 1, 2022 at 12:15 PM
> > > > Matthias
> > > > > >> J.
> > > > > >> > > Sax
> > > > > >> > > > <
> > > > > >> > > > >> > >> > > > > mjsax@apache.org
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > wrote:
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >>> Thanks for the details
> Justine!
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>> 20)
> > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>> The client side change for 2
> is
> > > > > >> removing
> > > > > >> > the
> > > > > >> > > > >> > >> > addPartitions
> > > > > >> > > > >> > >> > > > to
> > > > > >> > > > >> > >> > > > > > > > > >>> transaction
> > > > > >> > > > >> > >> > > > > > > > > >>>> call. We don't need to make
> > this
> > > > from
> > > > > >> the
> > > > > >> > > > >> producer
> > > > > >> > > > >> > to
> > > > > >> > > > >> > >> > the
> > > > > >> > > > >> > >> > > > txn
> > > > > >> > > > >> > >> > > > > > > > > >>> coordinator,
> > > > > >> > > > >> > >> > > > > > > > > >>>> only server side.
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>> I think I did not express
> myself
> > > > > >> clearly. I
> > > > > >> > > > >> > understand
> > > > > >> > > > >> > >> > that
> > > > > >> > > > >> > >> > > > we
> > > > > >> > > > >> > >> > > > > > can
> > > > > >> > > > >> > >> > > > > > > > (and
> > > > > >> > > > >> > >> > > > > > > > > >>> should) change the producer to
> > not
> > > > send
> > > > > >> the
> > > > > >> > > > >> > >> > `addPartitions`
> > > > > >> > > > >> > >> > > > > > request
> > > > > >> > > > >> > >> > > > > > > > any
> > > > > >> > > > >> > >> > > > > > > > > >>> longer. But I don't thinks
> it's
> > > > > >> requirement
> > > > > >> > > to
> > > > > >> > > > >> > change
> > > > > >> > > > >> > >> the
> > > > > >> > > > >> > >> > > > > broker?
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>> What I am trying to say is:
> as a
> > > > > >> safe-guard
> > > > > >> > > and
> > > > > >> > > > >> > >> > improvement
> > > > > >> > > > >> > >> > > > for
> > > > > >> > > > >> > >> > > > > > > older
> > > > > >> > > > >> > >> > > > > > > > > >>> producers, the partition
> leader
> > can
> > > > just
> > > > > >> > send
> > > > > >> > > > the
> > > > > >> > > > >> > >> > > > > `addPartitions`
> > > > > >> > > > >> > >> > > > > > > > > >>> request to the TX-coordinator
> > in any
> > > > > >> case
> > > > > >> > --
> > > > > >> > > if
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> old
> > > > > >> > > > >> > >> > > > > producer
> > > > > >> > > > >> > >> > > > > > > > > >>> correctly did send the
> > > > `addPartition`
> > > > > >> > request
> > > > > >> > > > to
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> > > > > > TX-coordinator
> > > > > >> > > > >> > >> > > > > > > > > >>> already, the TX-coordinator
> can
> > just
> > > > > >> > "ignore"
> > > > > >> > > > is
> > > > > >> > > > >> as
> > > > > >> > > > >> > >> > > > idempotent.
> > > > > >> > > > >> > >> > > > > > > > > However,
> > > > > >> > > > >> > >> > > > > > > > > >>> if the old producer has a bug
> > and
> > > > did
> > > > > >> > forget
> > > > > >> > > to
> > > > > >> > > > >> sent
> > > > > >> > > > >> > >> the
> > > > > >> > > > >> > >> > > > > > > > `addPartition`
> > > > > >> > > > >> > >> > > > > > > > > >>> request, we would now ensure
> > that
> > > > the
> > > > > >> > > partition
> > > > > >> > > > >> is
> > > > > >> > > > >> > >> indeed
> > > > > >> > > > >> > >> > > > added
> > > > > >> > > > >> > >> > > > > > to
> > > > > >> > > > >> > >> > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> TX and thus fix a potential
> > > > producer bug
> > > > > >> > > (even
> > > > > >> > > > >> if we
> > > > > >> > > > >> > >> > don't
> > > > > >> > > > >> > >> > > > get
> > > > > >> > > > >> > >> > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> fencing via the bump epoch).
> --
> > It
> > > > > >> seems to
> > > > > >> > > be
> > > > > >> > > > a
> > > > > >> > > > >> > good
> > > > > >> > > > >> > >> > > > > > improvement?
> > > > > >> > > > >> > >> > > > > > > Or
> > > > > >> > > > >> > >> > > > > > > > > is
> > > > > >> > > > >> > >> > > > > > > > > >>> there a reason to not do this?
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>> 30)
> > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>> Transaction is ongoing =
> > partition
> > > > was
> > > > > >> > added
> > > > > >> > > > to
> > > > > >> > > > >> > >> > > transaction
> > > > > >> > > > >> > >> > > > > via
> > > > > >> > > > >> > >> > > > > > > > > >>>> addPartitionsToTxn. We check
> > this
> > > > with
> > > > > >> the
> > > > > >> > > > >> > >> > > > > DescribeTransactions
> > > > > >> > > > >> > >> > > > > > > > call.
> > > > > >> > > > >> > >> > > > > > > > > >>> Let
> > > > > >> > > > >> > >> > > > > > > > > >>>> me know if this wasn't
> > sufficiently
> > > > > >> > > explained
> > > > > >> > > > >> here:
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>> If we do what I propose in
> > (20), we
> > > > > >> don't
> > > > > >> > > > really
> > > > > >> > > > >> > need
> > > > > >> > > > >> > >> to
> > > > > >> > > > >> > >> > > make
> > > > > >> > > > >> > >> > > > > > this
> > > > > >> > > > >> > >> > > > > > > > > >>> `DescribeTransaction` call, as
> > the
> > > > > >> > partition
> > > > > >> > > > >> leader
> > > > > >> > > > >> > >> adds
> > > > > >> > > > >> > >> > > the
> > > > > >> > > > >> > >> > > > > > > > partition
> > > > > >> > > > >> > >> > > > > > > > > >>> for older clients and we get
> > this
> > > > check
> > > > > >> for
> > > > > >> > > > free.
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>> 40)
> > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>> The idea here is that if any
> > > > messages
> > > > > >> > > somehow
> > > > > >> > > > >> come
> > > > > >> > > > >> > in
> > > > > >> > > > >> > >> > > before
> > > > > >> > > > >> > >> > > > > we
> > > > > >> > > > >> > >> > > > > > > get
> > > > > >> > > > >> > >> > > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> new
> > > > > >> > > > >> > >> > > > > > > > > >>>> epoch to the producer, they
> > will be
> > > > > >> > fenced.
> > > > > >> > > > >> > However,
> > > > > >> > > > >> > >> if
> > > > > >> > > > >> > >> > we
> > > > > >> > > > >> > >> > > > > don't
> > > > > >> > > > >> > >> > > > > > > > think
> > > > > >> > > > >> > >> > > > > > > > > >>> this
> > > > > >> > > > >> > >> > > > > > > > > >>>> is necessary, it can be
> > discussed
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>> I agree that we should have
> > epoch
> > > > > >> fencing.
> > > > > >> > My
> > > > > >> > > > >> > >> question is
> > > > > >> > > > >> > >> > > > > > > different:
> > > > > >> > > > >> > >> > > > > > > > > >>> Assume we are at epoch 73, and
> > we
> > > > have
> > > > > >> an
> > > > > >> > > > ongoing
> > > > > >> > > > >> > >> > > > transaction,
> > > > > >> > > > >> > >> > > > > > that
> > > > > >> > > > >> > >> > > > > > > > is
> > > > > >> > > > >> > >> > > > > > > > > >>> committed. It seems natural to
> > > > write the
> > > > > >> > > > "prepare
> > > > > >> > > > >> > >> commit"
> > > > > >> > > > >> > >> > > > > marker
> > > > > >> > > > >> > >> > > > > > > and
> > > > > >> > > > >> > >> > > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` both
> with
> > > > epoch
> > > > > >> 73,
> > > > > >> > > too,
> > > > > >> > > > >> as
> > > > > >> > > > >> > it
> > > > > >> > > > >> > >> > > belongs
> > > > > >> > > > >> > >> > > > > to
> > > > > >> > > > >> > >> > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> current transaction. Of
> course,
> > we
> > > > now
> > > > > >> also
> > > > > >> > > > bump
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> > epoch
> > > > > >> > > > >> > >> > > > and
> > > > > >> > > > >> > >> > > > > > > expect
> > > > > >> > > > >> > >> > > > > > > > > >>> the next requests to have
> epoch
> > 74,
> > > > and
> > > > > >> > would
> > > > > >> > > > >> reject
> > > > > >> > > > >> > >> an
> > > > > >> > > > >> > >> > > > request
> > > > > >> > > > >> > >> > > > > > > with
> > > > > >> > > > >> > >> > > > > > > > > >>> epoch 73, as the corresponding
> > TX
> > > > for
> > > > > >> epoch
> > > > > >> > > 73
> > > > > >> > > > >> was
> > > > > >> > > > >> > >> > already
> > > > > >> > > > >> > >> > > > > > > committed.
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>> It seems you propose to write
> > the
> > > > > >> "prepare
> > > > > >> > > > commit
> > > > > >> > > > >> > >> marker"
> > > > > >> > > > >> > >> > > and
> > > > > >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` with
> > epoch 74
> > > > > >> > though,
> > > > > >> > > > what
> > > > > >> > > > >> > >> would
> > > > > >> > > > >> > >> > > work,
> > > > > >> > > > >> > >> > > > > but
> > > > > >> > > > >> > >> > > > > > > it
> > > > > >> > > > >> > >> > > > > > > > > >>> seems confusing. Is there a
> > reason
> > > > why
> > > > > >> we
> > > > > >> > > would
> > > > > >> > > > >> use
> > > > > >> > > > >> > >> the
> > > > > >> > > > >> > >> > > > bumped
> > > > > >> > > > >> > >> > > > > > > epoch
> > > > > >> > > > >> > >> > > > > > > > 74
> > > > > >> > > > >> > >> > > > > > > > > >>> instead of the current epoch
> 73?
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>> 60)
> > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>> When we are checking if the
> > > > > >> transaction is
> > > > > >> > > > >> ongoing,
> > > > > >> > > > >> > >> we
> > > > > >> > > > >> > >> > > need
> > > > > >> > > > >> > >> > > > to
> > > > > >> > > > >> > >> > > > > > > make
> > > > > >> > > > >> > >> > > > > > > > a
> > > > > >> > > > >> > >> > > > > > > > > >>> round
> > > > > >> > > > >> > >> > > > > > > > > >>>> trip from the leader
> partition
> > to
> > > > the
> > > > > >> > > > >> transaction
> > > > > >> > > > >> > >> > > > coordinator.
> > > > > >> > > > >> > >> > > > > > In
> > > > > >> > > > >> > >> > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> time
> > > > > >> > > > >> > >> > > > > > > > > >>>> we are waiting for this
> > message to
> > > > come
> > > > > >> > > back,
> > > > > >> > > > in
> > > > > >> > > > >> > >> theory
> > > > > >> > > > >> > >> > we
> > > > > >> > > > >> > >> > > > > could
> > > > > >> > > > >> > >> > > > > > > > have
> > > > > >> > > > >> > >> > > > > > > > > >>> sent
> > > > > >> > > > >> > >> > > > > > > > > >>>> a commit/abort call that
> would
> > > > make the
> > > > > >> > > > original
> > > > > >> > > > >> > >> result
> > > > > >> > > > >> > >> > of
> > > > > >> > > > >> > >> > > > the
> > > > > >> > > > >> > >> > > > > > > check
> > > > > >> > > > >> > >> > > > > > > > > >>> out of
> > > > > >> > > > >> > >> > > > > > > > > >>>> date. That is why we can
> check
> > the
> > > > > >> leader
> > > > > >> > > > state
> > > > > >> > > > >> > >> before
> > > > > >> > > > >> > >> > we
> > > > > >> > > > >> > >> > > > > write
> > > > > >> > > > >> > >> > > > > > to
> > > > > >> > > > >> > >> > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> log.
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>> Thanks. Got it.
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>> However, is this really an
> > issue?
> > > > We put
> > > > > >> > the
> > > > > >> > > > >> produce
> > > > > >> > > > >> > >> > > request
> > > > > >> > > > >> > >> > > > in
> > > > > >> > > > >> > >> > > > > > > > > >>> purgatory, so how could we
> > process
> > > > the
> > > > > >> > > > >> > >> > > > `WriteTxnMarkerRequest`
> > > > > >> > > > >> > >> > > > > > > first?
> > > > > >> > > > >> > >> > > > > > > > > >>> Don't we need to put the
> > > > > >> > > > `WriteTxnMarkerRequest`
> > > > > >> > > > >> > into
> > > > > >> > > > >> > >> > > > > purgatory,
> > > > > >> > > > >> > >> > > > > > > too,
> > > > > >> > > > >> > >> > > > > > > > > >>> for this case, and process
> both
> > > > request
> > > > > >> > > > in-order?
> > > > > >> > > > >> > >> (Again,
> > > > > >> > > > >> > >> > > my
> > > > > >> > > > >> > >> > > > > > broker
> > > > > >> > > > >> > >> > > > > > > > > >>> knowledge is limited and maybe
> > we
> > > > don't
> > > > > >> > > > maintain
> > > > > >> > > > >> > >> request
> > > > > >> > > > >> > >> > > > order
> > > > > >> > > > >> > >> > > > > > for
> > > > > >> > > > >> > >> > > > > > > > this
> > > > > >> > > > >> > >> > > > > > > > > >>> case, what seems to be an
> issue
> > > > IMHO,
> > > > > >> and I
> > > > > >> > > am
> > > > > >> > > > >> > >> wondering
> > > > > >> > > > >> > >> > if
> > > > > >> > > > >> > >> > > > > > > changing
> > > > > >> > > > >> > >> > > > > > > > > >>> request handling to preserve
> > order
> > > > for
> > > > > >> this
> > > > > >> > > > case
> > > > > >> > > > >> > >> might be
> > > > > >> > > > >> > >> > > the
> > > > > >> > > > >> > >> > > > > > > cleaner
> > > > > >> > > > >> > >> > > > > > > > > >>> solution?)
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>> -Matthias
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>> On 11/30/22 3:28 PM, Artem
> > Livshits
> > > > > >> wrote:
> > > > > >> > > > >> > >> > > > > > > > > >>>> Hi Justine,
> > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>> I think the interesting part
> is
> > > > not in
> > > > > >> > this
> > > > > >> > > > >> logic
> > > > > >> > > > >> > >> > (because
> > > > > >> > > > >> > >> > > > it
> > > > > >> > > > >> > >> > > > > > > tries
> > > > > >> > > > >> > >> > > > > > > > to
> > > > > >> > > > >> > >> > > > > > > > > >>>> figure out when
> > > > UNKNOWN_PRODUCER_ID is
> > > > > >> > > > retriable
> > > > > >> > > > >> > and
> > > > > >> > > > >> > >> if
> > > > > >> > > > >> > >> > > it's
> > > > > >> > > > >> > >> > > > > > > > > retryable,
> > > > > >> > > > >> > >> > > > > > > > > >>>> it's definitely not fatal),
> but
> > > > what
> > > > > >> > happens
> > > > > >> > > > >> when
> > > > > >> > > > >> > >> this
> > > > > >> > > > >> > >> > > logic
> > > > > >> > > > >> > >> > > > > > > doesn't
> > > > > >> > > > >> > >> > > > > > > > > >>> return
> > > > > >> > > > >> > >> > > > > > > > > >>>> 'true' and falls through.  In
> > the
> > > > old
> > > > > >> > > clients
> > > > > >> > > > it
> > > > > >> > > > >> > >> seems
> > > > > >> > > > >> > >> > to
> > > > > >> > > > >> > >> > > be
> > > > > >> > > > >> > >> > > > > > > fatal,
> > > > > >> > > > >> > >> > > > > > > > if
> > > > > >> > > > >> > >> > > > > > > > > >>> we
> > > > > >> > > > >> > >> > > > > > > > > >>>> keep the behavior in the new
> > > > clients,
> > > > > >> I'd
> > > > > >> > > > >> expect it
> > > > > >> > > > >> > >> > would
> > > > > >> > > > >> > >> > > be
> > > > > >> > > > >> > >> > > > > > fatal
> > > > > >> > > > >> > >> > > > > > > > as
> > > > > >> > > > >> > >> > > > > > > > > >>> well.
> > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>> -Artem
> > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>> On Tue, Nov 29, 2022 at 11:57
> > AM
> > > > > >> Justine
> > > > > >> > > > Olshan
> > > > > >> > > > >> > >> > > > > > > > > >>>> <jolshan@confluent.io.invalid
> >
> > > > wrote:
> > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> Hi Artem and Jeff,
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> Thanks for taking a look and
> > > > sorry for
> > > > > >> > the
> > > > > >> > > > slow
> > > > > >> > > > >> > >> > response.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> You both mentioned the
> change
> > to
> > > > > >> handle
> > > > > >> > > > >> > >> > > UNKNOWN_PRODUCER_ID
> > > > > >> > > > >> > >> > > > > > > errors.
> > > > > >> > > > >> > >> > > > > > > > > To
> > > > > >> > > > >> > >> > > > > > > > > >>> be
> > > > > >> > > > >> > >> > > > > > > > > >>>>> clear — this error code will
> > only
> > > > be
> > > > > >> sent
> > > > > >> > > > again
> > > > > >> > > > >> > when
> > > > > >> > > > >> > >> > the
> > > > > >> > > > >> > >> > > > > > client's
> > > > > >> > > > >> > >> > > > > > > > > >>> request
> > > > > >> > > > >> > >> > > > > > > > > >>>>> version is high enough to
> > ensure
> > > > we
> > > > > >> > handle
> > > > > >> > > it
> > > > > >> > > > >> > >> > correctly.
> > > > > >> > > > >> > >> > > > > > > > > >>>>> The current (Java) client
> > handles
> > > > > >> this by
> > > > > >> > > the
> > > > > >> > > > >> > >> following
> > > > > >> > > > >> > >> > > > > > (somewhat
> > > > > >> > > > >> > >> > > > > > > > > long)
> > > > > >> > > > >> > >> > > > > > > > > >>>>> code snippet:
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> // An UNKNOWN_PRODUCER_ID
> > means
> > > > that
> > > > > >> we
> > > > > >> > > have
> > > > > >> > > > >> lost
> > > > > >> > > > >> > >> the
> > > > > >> > > > >> > >> > > > > producer
> > > > > >> > > > >> > >> > > > > > > > state
> > > > > >> > > > >> > >> > > > > > > > > >>> on the
> > > > > >> > > > >> > >> > > > > > > > > >>>>> broker. Depending on the log
> > start
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> // offset, we may want to
> > retry
> > > > > >> these, as
> > > > > >> > > > >> > described
> > > > > >> > > > >> > >> for
> > > > > >> > > > >> > >> > > > each
> > > > > >> > > > >> > >> > > > > > case
> > > > > >> > > > >> > >> > > > > > > > > >>> below. If
> > > > > >> > > > >> > >> > > > > > > > > >>>>> none of those apply, then
> for
> > the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> // idempotent producer, we
> > will
> > > > > >> locally
> > > > > >> > > bump
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> epoch
> > > > > >> > > > >> > >> > > and
> > > > > >> > > > >> > >> > > > > > reset
> > > > > >> > > > >> > >> > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>>>> sequence numbers of
> in-flight
> > > > batches
> > > > > >> > from
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> // sequence 0, then retry
> the
> > > > failed
> > > > > >> > batch,
> > > > > >> > > > >> which
> > > > > >> > > > >> > >> > should
> > > > > >> > > > >> > >> > > > now
> > > > > >> > > > >> > >> > > > > > > > succeed.
> > > > > >> > > > >> > >> > > > > > > > > >>> For
> > > > > >> > > > >> > >> > > > > > > > > >>>>> the transactional producer,
> > allow
> > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> // batch to fail. When
> > processing
> > > > the
> > > > > >> > > failed
> > > > > >> > > > >> > batch,
> > > > > >> > > > >> > >> we
> > > > > >> > > > >> > >> > > will
> > > > > >> > > > >> > >> > > > > > > > > transition
> > > > > >> > > > >> > >> > > > > > > > > >>> to
> > > > > >> > > > >> > >> > > > > > > > > >>>>> an abortable error and set a
> > flag
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> // indicating that we need
> to
> > > > bump the
> > > > > >> > > epoch
> > > > > >> > > > >> (if
> > > > > >> > > > >> > >> > > supported
> > > > > >> > > > >> > >> > > > by
> > > > > >> > > > >> > >> > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> broker).
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> if (error ==
> > > > > >> > Errors.*UNKNOWN_PRODUCER_ID*)
> > > > > >> > > {
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>       if
> > (response.logStartOffset
> > > > ==
> > > > > >> -1)
> > > > > >> > {
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // We don't know
> > the log
> > > > > >> start
> > > > > >> > > > offset
> > > > > >> > > > >> > with
> > > > > >> > > > >> > >> > this
> > > > > >> > > > >> > >> > > > > > > response.
> > > > > >> > > > >> > >> > > > > > > > > We
> > > > > >> > > > >> > >> > > > > > > > > >>> should
> > > > > >> > > > >> > >> > > > > > > > > >>>>> just retry the request until
> > we
> > > > get
> > > > > >> it.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // The
> > > > UNKNOWN_PRODUCER_ID
> > > > > >> > error
> > > > > >> > > > code
> > > > > >> > > > >> > was
> > > > > >> > > > >> > >> > added
> > > > > >> > > > >> > >> > > > > along
> > > > > >> > > > >> > >> > > > > > > > with
> > > > > >> > > > >> > >> > > > > > > > > >>> the new
> > > > > >> > > > >> > >> > > > > > > > > >>>>> ProduceResponse which
> > includes the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // logStartOffset.
> > So
> > > > the
> > > > > >> '-1'
> > > > > >> > > > >> sentinel
> > > > > >> > > > >> > is
> > > > > >> > > > >> > >> > not
> > > > > >> > > > >> > >> > > > for
> > > > > >> > > > >> > >> > > > > > > > backward
> > > > > >> > > > >> > >> > > > > > > > > >>>>> compatibility. Instead, it
> is
> > > > possible
> > > > > >> > for
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // a broker to not
> > know
> > > > the
> > > > > >> > > > >> > >> logStartOffset at
> > > > > >> > > > >> > >> > > > when
> > > > > >> > > > >> > >> > > > > it
> > > > > >> > > > >> > >> > > > > > > is
> > > > > >> > > > >> > >> > > > > > > > > >>> returning
> > > > > >> > > > >> > >> > > > > > > > > >>>>> the response because the
> > partition
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // may have moved
> > away
> > > > from
> > > > > >> the
> > > > > >> > > > >> broker
> > > > > >> > > > >> > >> from
> > > > > >> > > > >> > >> > the
> > > > > >> > > > >> > >> > > > > time
> > > > > >> > > > >> > >> > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> error was
> > > > > >> > > > >> > >> > > > > > > > > >>>>> initially raised to the time
> > the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // response was
> > being
> > > > > >> > > constructed.
> > > > > >> > > > In
> > > > > >> > > > >> > >> these
> > > > > >> > > > >> > >> > > > cases,
> > > > > >> > > > >> > >> > > > > we
> > > > > >> > > > >> > >> > > > > > > > > should
> > > > > >> > > > >> > >> > > > > > > > > >>> just
> > > > > >> > > > >> > >> > > > > > > > > >>>>> retry the request: we are
> > > > guaranteed
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // to eventually
> > get a
> > > > > >> > > > logStartOffset
> > > > > >> > > > >> > once
> > > > > >> > > > >> > >> > > things
> > > > > >> > > > >> > >> > > > > > > settle
> > > > > >> > > > >> > >> > > > > > > > > down.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>       if
> > > > > >> (batch.sequenceHasBeenReset()) {
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // When the first
> > > > inflight
> > > > > >> > batch
> > > > > >> > > > >> fails
> > > > > >> > > > >> > >> due to
> > > > > >> > > > >> > >> > > the
> > > > > >> > > > >> > >> > > > > > > > > truncation
> > > > > >> > > > >> > >> > > > > > > > > >>> case,
> > > > > >> > > > >> > >> > > > > > > > > >>>>> then the sequences of all
> the
> > > > other
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // in flight
> batches
> > > > would
> > > > > >> have
> > > > > >> > > > been
> > > > > >> > > > >> > >> > restarted
> > > > > >> > > > >> > >> > > > from
> > > > > >> > > > >> > >> > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> beginning.
> > > > > >> > > > >> > >> > > > > > > > > >>>>> However, when those
> responses
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // come back from
> > the
> > > > > >> broker,
> > > > > >> > > they
> > > > > >> > > > >> would
> > > > > >> > > > >> > >> also
> > > > > >> > > > >> > >> > > > come
> > > > > >> > > > >> > >> > > > > > with
> > > > > >> > > > >> > >> > > > > > > > an
> > > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID error.
> In
> > this
> > > > > >> case,
> > > > > >> > we
> > > > > >> > > > >> should
> > > > > >> > > > >> > >> not
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // reset the
> > sequence
> > > > > >> numbers
> > > > > >> > to
> > > > > >> > > > the
> > > > > >> > > > >> > >> > beginning.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>       } else if
> > > > > >> > > > >> > >> > > > >
> > (lastAckedOffset(batch.topicPartition).orElse(
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > > > > >> > > > >> > >> > > > response.logStartOffset) {
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // The head of the
> > log
> > > > has
> > > > > >> been
> > > > > >> > > > >> removed,
> > > > > >> > > > >> > >> > > probably
> > > > > >> > > > >> > >> > > > > due
> > > > > >> > > > >> > >> > > > > > > to
> > > > > >> > > > >> > >> > > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>>>> retention time elapsing. In
> > this
> > > > case,
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // we expect to
> > lose the
> > > > > >> > producer
> > > > > >> > > > >> state.
> > > > > >> > > > >> > >> For
> > > > > >> > > > >> > >> > > the
> > > > > >> > > > >> > >> > > > > > > > > transactional
> > > > > >> > > > >> > >> > > > > > > > > >>>>> producer, reset the
> sequences
> > of
> > > > all
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // inflight
> batches
> > to
> > > > be
> > > > > >> from
> > > > > >> > > the
> > > > > >> > > > >> > >> beginning
> > > > > >> > > > >> > >> > > and
> > > > > >> > > > >> > >> > > > > > retry
> > > > > >> > > > >> > >> > > > > > > > > them,
> > > > > >> > > > >> > >> > > > > > > > > >>> so
> > > > > >> > > > >> > >> > > > > > > > > >>>>> that the transaction does
> not
> > > > need to
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // be aborted. For
> > the
> > > > > >> > idempotent
> > > > > >> > > > >> > >> producer,
> > > > > >> > > > >> > >> > > bump
> > > > > >> > > > >> > >> > > > > the
> > > > > >> > > > >> > >> > > > > > > > epoch
> > > > > >> > > > >> > >> > > > > > > > > to
> > > > > >> > > > >> > >> > > > > > > > > >>> avoid
> > > > > >> > > > >> > >> > > > > > > > > >>>>> reusing (sequence, epoch)
> > pairs
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           if
> > (isTransactional()) {
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > >
> > > > > >> > > >
> > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > > > > >> > > > >> > >> > > > > > > > > >>>>> this.producerIdAndEpoch);
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           } else {
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > >
> > > > > >> requestEpochBumpForPartition(batch.topicPartition);
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           }
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>       if
> (!isTransactional())
> > {
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // For the
> > idempotent
> > > > > >> producer,
> > > > > >> > > > >> always
> > > > > >> > > > >> > >> retry
> > > > > >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > > > >> > > > >> > >> > > > > > > > > >>>>> errors. If the batch has the
> > > > current
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // producer ID and
> > > > epoch,
> > > > > >> > > request a
> > > > > >> > > > >> bump
> > > > > >> > > > >> > >> of
> > > > > >> > > > >> > >> > the
> > > > > >> > > > >> > >> > > > > > epoch.
> > > > > >> > > > >> > >> > > > > > > > > >>> Otherwise
> > > > > >> > > > >> > >> > > > > > > > > >>>>> just retry the produce.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > >
> > > > requestEpochBumpForPartition(batch.topicPartition);
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> }
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> I was considering keeping
> this
> > > > > >> behavior —
> > > > > >> > > but
> > > > > >> > > > >> am
> > > > > >> > > > >> > >> open
> > > > > >> > > > >> > >> > to
> > > > > >> > > > >> > >> > > > > > > > simplifying
> > > > > >> > > > >> > >> > > > > > > > > >>> it.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> We are leaving changes to
> > older
> > > > > >> clients
> > > > > >> > off
> > > > > >> > > > the
> > > > > >> > > > >> > >> table
> > > > > >> > > > >> > >> > > here
> > > > > >> > > > >> > >> > > > > > since
> > > > > >> > > > >> > >> > > > > > > it
> > > > > >> > > > >> > >> > > > > > > > > >>> caused
> > > > > >> > > > >> > >> > > > > > > > > >>>>> many issues for clients in
> the
> > > > past.
> > > > > >> > > > Previously
> > > > > >> > > > >> > this
> > > > > >> > > > >> > >> > was
> > > > > >> > > > >> > >> > > a
> > > > > >> > > > >> > >> > > > > > fatal
> > > > > >> > > > >> > >> > > > > > > > > error
> > > > > >> > > > >> > >> > > > > > > > > >>> and
> > > > > >> > > > >> > >> > > > > > > > > >>>>> we didn't have the
> mechanisms
> > in
> > > > > >> place to
> > > > > >> > > > >> detect
> > > > > >> > > > >> > >> when
> > > > > >> > > > >> > >> > > this
> > > > > >> > > > >> > >> > > > > was
> > > > > >> > > > >> > >> > > > > > a
> > > > > >> > > > >> > >> > > > > > > > > >>> legitimate
> > > > > >> > > > >> > >> > > > > > > > > >>>>> case vs some bug or gap in
> the
> > > > > >> protocol.
> > > > > >> > > > >> Ensuring
> > > > > >> > > > >> > >> each
> > > > > >> > > > >> > >> > > > > > > transaction
> > > > > >> > > > >> > >> > > > > > > > > has
> > > > > >> > > > >> > >> > > > > > > > > >>> its
> > > > > >> > > > >> > >> > > > > > > > > >>>>> own epoch should close this
> > gap.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> And to address Jeff's second
> > > > point:
> > > > > >> > > > >> > >> > > > > > > > > >>>>> *does the typical produce
> > request
> > > > path
> > > > > >> > > append
> > > > > >> > > > >> > >> records
> > > > > >> > > > >> > >> > to
> > > > > >> > > > >> > >> > > > > local
> > > > > >> > > > >> > >> > > > > > > log
> > > > > >> > > > >> > >> > > > > > > > > >>> along*
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> *with the
> > currentTxnFirstOffset
> > > > > >> > > information?
> > > > > >> > > > I
> > > > > >> > > > >> > would
> > > > > >> > > > >> > >> > like
> > > > > >> > > > >> > >> > > > to
> > > > > >> > > > >> > >> > > > > > > > > >>> understand*
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> *when the field is written
> to
> > > > disk.*
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> Yes, the first produce
> request
> > > > > >> populates
> > > > > >> > > this
> > > > > >> > > > >> > field
> > > > > >> > > > >> > >> and
> > > > > >> > > > >> > >> > > > > writes
> > > > > >> > > > >> > >> > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> offset
> > > > > >> > > > >> > >> > > > > > > > > >>>>> as part of the record batch
> > and
> > > > also
> > > > > >> to
> > > > > >> > the
> > > > > >> > > > >> > producer
> > > > > >> > > > >> > >> > > state
> > > > > >> > > > >> > >> > > > > > > > snapshot.
> > > > > >> > > > >> > >> > > > > > > > > >>> When
> > > > > >> > > > >> > >> > > > > > > > > >>>>> we reload the records on
> > restart
> > > > > >> and/or
> > > > > >> > > > >> > >> reassignment,
> > > > > >> > > > >> > >> > we
> > > > > >> > > > >> > >> > > > > > > repopulate
> > > > > >> > > > >> > >> > > > > > > > > >>> this
> > > > > >> > > > >> > >> > > > > > > > > >>>>> field with the snapshot from
> > disk
> > > > > >> along
> > > > > >> > > with
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> rest
> > > > > >> > > > >> > >> > of
> > > > > >> > > > >> > >> > > > the
> > > > > >> > > > >> > >> > > > > > > > producer
> > > > > >> > > > >> > >> > > > > > > > > >>>>> state.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> Let me know if there are
> > further
> > > > > >> comments
> > > > > >> > > > >> and/or
> > > > > >> > > > >> > >> > > questions.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> Thanks,
> > > > > >> > > > >> > >> > > > > > > > > >>>>> Justine
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> On Tue, Nov 22, 2022 at 9:00
> > PM
> > > > Jeff
> > > > > >> Kim
> > > > > >> > > > >> > >> > > > > > > > > <jeff.kim@confluent.io.invalid
> > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> wrote:
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> Hi Justine,
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> Thanks for the KIP! I have
> > two
> > > > > >> > questions:
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> 1) For new clients, we can
> > once
> > > > again
> > > > > >> > > return
> > > > > >> > > > >> an
> > > > > >> > > > >> > >> error
> > > > > >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> for sequences
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> that are non-zero when
> there
> > is
> > > > no
> > > > > >> > > producer
> > > > > >> > > > >> state
> > > > > >> > > > >> > >> > > present
> > > > > >> > > > >> > >> > > > on
> > > > > >> > > > >> > >> > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> server.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> This will indicate we
> missed
> > the
> > > > 0
> > > > > >> > > sequence
> > > > > >> > > > >> and
> > > > > >> > > > >> > we
> > > > > >> > > > >> > >> > don't
> > > > > >> > > > >> > >> > > > yet
> > > > > >> > > > >> > >> > > > > > > want
> > > > > >> > > > >> > >> > > > > > > > to
> > > > > >> > > > >> > >> > > > > > > > > >>>>> write
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> to the log.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> I would like to understand
> > the
> > > > > >> current
> > > > > >> > > > >> behavior
> > > > > >> > > > >> > to
> > > > > >> > > > >> > >> > > handle
> > > > > >> > > > >> > >> > > > > > older
> > > > > >> > > > >> > >> > > > > > > > > >>> clients,
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> and if there are any
> changes
> > we
> > > > are
> > > > > >> > > making.
> > > > > >> > > > >> Maybe
> > > > > >> > > > >> > >> I'm
> > > > > >> > > > >> > >> > > > > missing
> > > > > >> > > > >> > >> > > > > > > > > >>> something,
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> but we would want to
> identify
> > > > > >> whether we
> > > > > >> > > > >> missed
> > > > > >> > > > >> > >> the 0
> > > > > >> > > > >> > >> > > > > sequence
> > > > > >> > > > >> > >> > > > > > > for
> > > > > >> > > > >> > >> > > > > > > > > >>> older
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> clients, no?
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> 2) Upon returning from the
> > > > > >> transaction
> > > > > >> > > > >> > >> coordinator, we
> > > > > >> > > > >> > >> > > can
> > > > > >> > > > >> > >> > > > > set
> > > > > >> > > > >> > >> > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> transaction
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> as ongoing on the leader by
> > > > > >> populating
> > > > > >> > > > >> > >> > > > currentTxnFirstOffset
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> through the typical produce
> > > > request
> > > > > >> > > > handling.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> does the typical produce
> > request
> > > > path
> > > > > >> > > append
> > > > > >> > > > >> > >> records
> > > > > >> > > > >> > >> > to
> > > > > >> > > > >> > >> > > > > local
> > > > > >> > > > >> > >> > > > > > > log
> > > > > >> > > > >> > >> > > > > > > > > >>> along
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> with the
> > currentTxnFirstOffset
> > > > > >> > > information?
> > > > > >> > > > I
> > > > > >> > > > >> > would
> > > > > >> > > > >> > >> > like
> > > > > >> > > > >> > >> > > > to
> > > > > >> > > > >> > >> > > > > > > > > understand
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> when the field is written
> to
> > > > disk.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> Thanks,
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> Jeff
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> On Tue, Nov 22, 2022 at
> 4:44
> > PM
> > > > Artem
> > > > > >> > > > Livshits
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> <alivshits@confluent.io
> > .invalid>
> > > > > >> wrote:
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> Hi Justine,
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> Thank you for the KIP.  I
> > have
> > > > one
> > > > > >> > > > question.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> 5) For new clients, we can
> > once
> > > > > >> again
> > > > > >> > > > return
> > > > > >> > > > >> an
> > > > > >> > > > >> > >> error
> > > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> I believe we had problems
> > in the
> > > > > >> past
> > > > > >> > > with
> > > > > >> > > > >> > >> returning
> > > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> because it was considered
> > fatal
> > > > and
> > > > > >> > > > required
> > > > > >> > > > >> > >> client
> > > > > >> > > > >> > >> > > > > restart.
> > > > > >> > > > >> > >> > > > > > > It
> > > > > >> > > > >> > >> > > > > > > > > >>> would
> > > > > >> > > > >> > >> > > > > > > > > >>>>> be
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> good to spell out the new
> > client
> > > > > >> > behavior
> > > > > >> > > > >> when
> > > > > >> > > > >> > it
> > > > > >> > > > >> > >> > > > receives
> > > > > >> > > > >> > >> > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > error.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> -Artem
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> On Tue, Nov 22, 2022 at
> > 10:00 AM
> > > > > >> > Justine
> > > > > >> > > > >> Olshan
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > <jo...@confluent.io.invalid>
> > > > > >> wrote:
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks for taking a look
> > > > Matthias.
> > > > > >> > I've
> > > > > >> > > > >> tried
> > > > > >> > > > >> > to
> > > > > >> > > > >> > >> > > answer
> > > > > >> > > > >> > >> > > > > your
> > > > > >> > > > >> > >> > > > > > > > > >>>>> questions
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> below:
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 10)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Right — so the hanging
> > > > transaction
> > > > > >> > only
> > > > > >> > > > >> occurs
> > > > > >> > > > >> > >> when
> > > > > >> > > > >> > >> > we
> > > > > >> > > > >> > >> > > > > have
> > > > > >> > > > >> > >> > > > > > a
> > > > > >> > > > >> > >> > > > > > > > late
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> message
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> come in and the partition
> > is
> > > > never
> > > > > >> > added
> > > > > >> > > > to
> > > > > >> > > > >> a
> > > > > >> > > > >> > >> > > > transaction
> > > > > >> > > > >> > >> > > > > > > again.
> > > > > >> > > > >> > >> > > > > > > > > If
> > > > > >> > > > >> > >> > > > > > > > > >>>>> we
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> never add the partition
> to
> > a
> > > > > >> > > transaction,
> > > > > >> > > > we
> > > > > >> > > > >> > will
> > > > > >> > > > >> > >> > > never
> > > > > >> > > > >> > >> > > > > > write
> > > > > >> > > > >> > >> > > > > > > a
> > > > > >> > > > >> > >> > > > > > > > > >>>>> marker
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> and
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> never advance the LSO.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> If we do end up adding
> the
> > > > > >> partition
> > > > > >> > to
> > > > > >> > > > the
> > > > > >> > > > >> > >> > > transaction
> > > > > >> > > > >> > >> > > > (I
> > > > > >> > > > >> > >> > > > > > > > suppose
> > > > > >> > > > >> > >> > > > > > > > > >>>>> this
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> can
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> happen before or after
> the
> > late
> > > > > >> > message
> > > > > >> > > > >> comes
> > > > > >> > > > >> > in)
> > > > > >> > > > >> > >> > then
> > > > > >> > > > >> > >> > > > we
> > > > > >> > > > >> > >> > > > > > will
> > > > > >> > > > >> > >> > > > > > > > > >>>>> include
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> late message in the next
> > > > > >> (incorrect)
> > > > > >> > > > >> > transaction.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> So perhaps it is clearer
> to
> > > > make
> > > > > >> the
> > > > > >> > > > >> > distinction
> > > > > >> > > > >> > >> > > between
> > > > > >> > > > >> > >> > > > > > > > messages
> > > > > >> > > > >> > >> > > > > > > > > >>>>> that
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> eventually get added to
> the
> > > > > >> > transaction
> > > > > >> > > > (but
> > > > > >> > > > >> > the
> > > > > >> > > > >> > >> > wrong
> > > > > >> > > > >> > >> > > > > one)
> > > > > >> > > > >> > >> > > > > > or
> > > > > >> > > > >> > >> > > > > > > > > >>>>> messages
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> that never get added and
> > become
> > > > > >> > hanging.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 20)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> The client side change
> for
> > 2 is
> > > > > >> > removing
> > > > > >> > > > the
> > > > > >> > > > >> > >> > > > addPartitions
> > > > > >> > > > >> > >> > > > > > to
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> transaction
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> call. We don't need to
> make
> > > > this
> > > > > >> from
> > > > > >> > > the
> > > > > >> > > > >> > >> producer
> > > > > >> > > > >> > >> > to
> > > > > >> > > > >> > >> > > > the
> > > > > >> > > > >> > >> > > > > > txn
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> coordinator,
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> only server side.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> In my opinion, the issue
> > with
> > > > the
> > > > > >> > > > >> > >> addPartitionsToTxn
> > > > > >> > > > >> > >> > > > call
> > > > > >> > > > >> > >> > > > > > for
> > > > > >> > > > >> > >> > > > > > > > > older
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> clients
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> is that we don't have the
> > epoch
> > > > > >> bump,
> > > > > >> > so
> > > > > >> > > > we
> > > > > >> > > > >> > don't
> > > > > >> > > > >> > >> > know
> > > > > >> > > > >> > >> > > > if
> > > > > >> > > > >> > >> > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> message
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> belongs to the previous
> > > > > >> transaction or
> > > > > >> > > > this
> > > > > >> > > > >> > one.
> > > > > >> > > > >> > >> We
> > > > > >> > > > >> > >> > > need
> > > > > >> > > > >> > >> > > > > to
> > > > > >> > > > >> > >> > > > > > > > check
> > > > > >> > > > >> > >> > > > > > > > > if
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> partition has been added
> to
> > > > this
> > > > > >> > > > >> transaction.
> > > > > >> > > > >> > Of
> > > > > >> > > > >> > >> > > course,
> > > > > >> > > > >> > >> > > > > > this
> > > > > >> > > > >> > >> > > > > > > > > means
> > > > > >> > > > >> > >> > > > > > > > > >>>>> we
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> won't completely cover
> the
> > case
> > > > > >> where
> > > > > >> > we
> > > > > >> > > > >> have a
> > > > > >> > > > >> > >> > really
> > > > > >> > > > >> > >> > > > > late
> > > > > >> > > > >> > >> > > > > > > > > message
> > > > > >> > > > >> > >> > > > > > > > > >>>>> and
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> we
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> have added the partition
> to
> > > > the new
> > > > > >> > > > >> > transaction,
> > > > > >> > > > >> > >> but
> > > > > >> > > > >> > >> > > > > that's
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> unfortunately
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> something we will need
> the
> > new
> > > > > >> clients
> > > > > >> > > to
> > > > > >> > > > >> > cover.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 30)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Transaction is ongoing =
> > > > partition
> > > > > >> was
> > > > > >> > > > >> added to
> > > > > >> > > > >> > >> > > > > transaction
> > > > > >> > > > >> > >> > > > > > > via
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> addPartitionsToTxn. We
> > check
> > > > this
> > > > > >> with
> > > > > >> > > the
> > > > > >> > > > >> > >> > > > > > > DescribeTransactions
> > > > > >> > > > >> > >> > > > > > > > > >>> call.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> Let
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> me know if this wasn't
> > > > sufficiently
> > > > > >> > > > >> explained
> > > > > >> > > > >> > >> here:
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > >
> > > > > >> > > > >> > >> > > > >
> > > > > >> > > > >> > >> > > >
> > > > > >> > > > >> > >> > >
> > > > > >> > > > >> > >> >
> > > > > >> > > > >> > >>
> > > > > >> > > > >> >
> > > > > >> > > > >>
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 40)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> The idea here is that if
> > any
> > > > > >> messages
> > > > > >> > > > >> somehow
> > > > > >> > > > >> > >> come
> > > > > >> > > > >> > >> > in
> > > > > >> > > > >> > >> > > > > before
> > > > > >> > > > >> > >> > > > > > > we
> > > > > >> > > > >> > >> > > > > > > > > get
> > > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> new
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> epoch to the producer,
> they
> > > > will be
> > > > > >> > > > fenced.
> > > > > >> > > > >> > >> However,
> > > > > >> > > > >> > >> > > if
> > > > > >> > > > >> > >> > > > we
> > > > > >> > > > >> > >> > > > > > > don't
> > > > > >> > > > >> > >> > > > > > > > > >>>>> think
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> this
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> is necessary, it can be
> > > > discussed
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 50)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> It should be synchronous
> > > > because
> > > > > >> if we
> > > > > >> > > > have
> > > > > >> > > > >> an
> > > > > >> > > > >> > >> event
> > > > > >> > > > >> > >> > > > (ie,
> > > > > >> > > > >> > >> > > > > an
> > > > > >> > > > >> > >> > > > > > > > > error)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> that
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> causes us to need to
> abort
> > the
> > > > > >> > > > transaction,
> > > > > >> > > > >> we
> > > > > >> > > > >> > >> need
> > > > > >> > > > >> > >> > to
> > > > > >> > > > >> > >> > > > > know
> > > > > >> > > > >> > >> > > > > > > > which
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> partitions to send
> > transaction
> > > > > >> markers
> > > > > >> > > to.
> > > > > >> > > > >> We
> > > > > >> > > > >> > >> know
> > > > > >> > > > >> > >> > the
> > > > > >> > > > >> > >> > > > > > > > partitions
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> because
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> we added them to the
> > > > coordinator
> > > > > >> via
> > > > > >> > the
> > > > > >> > > > >> > >> > > > > addPartitionsToTxn
> > > > > >> > > > >> > >> > > > > > > > call.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Previously we have had
> > > > asynchronous
> > > > > >> > > calls
> > > > > >> > > > in
> > > > > >> > > > >> > the
> > > > > >> > > > >> > >> > past
> > > > > >> > > > >> > >> > > > (ie,
> > > > > >> > > > >> > >> > > > > > > > writing
> > > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> commit markers when the
> > > > > >> transaction is
> > > > > >> > > > >> > completed)
> > > > > >> > > > >> > >> > but
> > > > > >> > > > >> > >> > > > > often
> > > > > >> > > > >> > >> > > > > > > this
> > > > > >> > > > >> > >> > > > > > > > > >>> just
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> causes confusion as we
> > need to
> > > > wait
> > > > > >> > for
> > > > > >> > > > some
> > > > > >> > > > >> > >> > > operations
> > > > > >> > > > >> > >> > > > to
> > > > > >> > > > >> > >> > > > > > > > > complete.
> > > > > >> > > > >> > >> > > > > > > > > >>>>> In
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> writing commit markers
> > case,
> > > > > >> clients
> > > > > >> > > often
> > > > > >> > > > >> see
> > > > > >> > > > >> > >> > > > > > > > > >>>>> CONCURRENT_TRANSACTIONs
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> error messages and that
> > can be
> > > > > >> > > confusing.
> > > > > >> > > > >> For
> > > > > >> > > > >> > >> that
> > > > > >> > > > >> > >> > > > reason,
> > > > > >> > > > >> > >> > > > > > it
> > > > > >> > > > >> > >> > > > > > > > may
> > > > > >> > > > >> > >> > > > > > > > > be
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> simpler to just have
> > > > synchronous
> > > > > >> > calls —
> > > > > >> > > > >> > >> especially
> > > > > >> > > > >> > >> > if
> > > > > >> > > > >> > >> > > > we
> > > > > >> > > > >> > >> > > > > > need
> > > > > >> > > > >> > >> > > > > > > > to
> > > > > >> > > > >> > >> > > > > > > > > >>>>> block
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> on
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> some operation's
> completion
> > > > anyway
> > > > > >> > > before
> > > > > >> > > > we
> > > > > >> > > > >> > can
> > > > > >> > > > >> > >> > start
> > > > > >> > > > >> > >> > > > the
> > > > > >> > > > >> > >> > > > > > > next
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> transaction. And yes, I
> > meant
> > > > > >> > > > coordinator. I
> > > > > >> > > > >> > will
> > > > > >> > > > >> > >> > fix
> > > > > >> > > > >> > >> > > > > that.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 60)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> When we are checking if
> the
> > > > > >> > transaction
> > > > > >> > > is
> > > > > >> > > > >> > >> ongoing,
> > > > > >> > > > >> > >> > we
> > > > > >> > > > >> > >> > > > > need
> > > > > >> > > > >> > >> > > > > > to
> > > > > >> > > > >> > >> > > > > > > > > make
> > > > > >> > > > >> > >> > > > > > > > > >>> a
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> round
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> trip from the leader
> > partition
> > > > to
> > > > > >> the
> > > > > >> > > > >> > transaction
> > > > > >> > > > >> > >> > > > > > coordinator.
> > > > > >> > > > >> > >> > > > > > > > In
> > > > > >> > > > >> > >> > > > > > > > > >>> the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> time
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> we are waiting for this
> > > > message to
> > > > > >> > come
> > > > > >> > > > >> back,
> > > > > >> > > > >> > in
> > > > > >> > > > >> > >> > > theory
> > > > > >> > > > >> > >> > > > we
> > > > > >> > > > >> > >> > > > > > > could
> > > > > >> > > > >> > >> > > > > > > > > >>> have
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> sent
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> a commit/abort call that
> > would
> > > > make
> > > > > >> > the
> > > > > >> > > > >> > original
> > > > > >> > > > >> > >> > > result
> > > > > >> > > > >> > >> > > > of
> > > > > >> > > > >> > >> > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > check
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> out
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> of
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> date. That is why we can
> > check
> > > > the
> > > > > >> > > leader
> > > > > >> > > > >> state
> > > > > >> > > > >> > >> > before
> > > > > >> > > > >> > >> > > > we
> > > > > >> > > > >> > >> > > > > > > write
> > > > > >> > > > >> > >> > > > > > > > to
> > > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> log.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> I'm happy to update the
> > KIP if
> > > > > >> some of
> > > > > >> > > > these
> > > > > >> > > > >> > >> things
> > > > > >> > > > >> > >> > > were
> > > > > >> > > > >> > >> > > > > not
> > > > > >> > > > >> > >> > > > > > > > > clear.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks,
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Justine
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> On Mon, Nov 21, 2022 at
> > 7:11 PM
> > > > > >> > Matthias
> > > > > >> > > > J.
> > > > > >> > > > >> > Sax <
> > > > > >> > > > >> > >> > > > > > > > mjsax@apache.org
> > > > > >> > > > >> > >> > > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> wrote:
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Thanks for the KIP.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Couple of clarification
> > > > questions
> > > > > >> (I
> > > > > >> > am
> > > > > >> > > > >> not a
> > > > > >> > > > >> > >> > broker
> > > > > >> > > > >> > >> > > > > expert
> > > > > >> > > > >> > >> > > > > > > do
> > > > > >> > > > >> > >> > > > > > > > > >>>>> maybe
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> some question are
> obvious
> > for
> > > > > >> others,
> > > > > >> > > but
> > > > > >> > > > >> not
> > > > > >> > > > >> > >> for
> > > > > >> > > > >> > >> > me
> > > > > >> > > > >> > >> > > > with
> > > > > >> > > > >> > >> > > > > > my
> > > > > >> > > > >> > >> > > > > > > > lack
> > > > > >> > > > >> > >> > > > > > > > > >>>>> of
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> broker knowledge).
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (10)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> 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.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> What happens if the
> > message
> > > > come
> > > > > >> in
> > > > > >> > > > before
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> next
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> addPartitionsToTxn
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> request? It seems the
> > broker
> > > > > >> hosting
> > > > > >> > > the
> > > > > >> > > > >> data
> > > > > >> > > > >> > >> > > > partitions
> > > > > >> > > > >> > >> > > > > > > won't
> > > > > >> > > > >> > >> > > > > > > > > know
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> anything about it and
> > append
> > > > it to
> > > > > >> > the
> > > > > >> > > > >> > >> partition,
> > > > > >> > > > >> > >> > > too?
> > > > > >> > > > >> > >> > > > > What
> > > > > >> > > > >> > >> > > > > > > is
> > > > > >> > > > >> > >> > > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> difference between both
> > cases?
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Also, it seems a TX
> would
> > only
> > > > > >> hang,
> > > > > >> > if
> > > > > >> > > > >> there
> > > > > >> > > > >> > >> is no
> > > > > >> > > > >> > >> > > > > > following
> > > > > >> > > > >> > >> > > > > > > > TX
> > > > > >> > > > >> > >> > > > > > > > > >>>>> that
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> is
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> either committer or
> > aborted?
> > > > Thus,
> > > > > >> > for
> > > > > >> > > > the
> > > > > >> > > > >> > case
> > > > > >> > > > >> > >> > > above,
> > > > > >> > > > >> > >> > > > > the
> > > > > >> > > > >> > >> > > > > > TX
> > > > > >> > > > >> > >> > > > > > > > > might
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> actually not hang (of
> > course,
> > > > we
> > > > > >> > might
> > > > > >> > > > get
> > > > > >> > > > >> an
> > > > > >> > > > >> > >> EOS
> > > > > >> > > > >> > >> > > > > violation
> > > > > >> > > > >> > >> > > > > > > if
> > > > > >> > > > >> > >> > > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> first
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> TX was aborted and the
> > second
> > > > > >> > > committed,
> > > > > >> > > > or
> > > > > >> > > > >> > the
> > > > > >> > > > >> > >> > other
> > > > > >> > > > >> > >> > > > way
> > > > > >> > > > >> > >> > > > > > > > > around).
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (20)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Of course, 1 and 2
> > require
> > > > > >> > client-side
> > > > > >> > > > >> > >> changes, so
> > > > > >> > > > >> > >> > > for
> > > > > >> > > > >> > >> > > > > > older
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> clients,
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> those approaches won’t
> > apply.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> For (1) I understand
> why a
> > > > client
> > > > > >> > > change
> > > > > >> > > > is
> > > > > >> > > > >> > >> > > necessary,
> > > > > >> > > > >> > >> > > > > but
> > > > > >> > > > >> > >> > > > > > > not
> > > > > >> > > > >> > >> > > > > > > > > sure
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> why
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> we need a client change
> > for
> > > > (2).
> > > > > >> Can
> > > > > >> > > you
> > > > > >> > > > >> > >> elaborate?
> > > > > >> > > > >> > >> > > --
> > > > > >> > > > >> > >> > > > > > Later
> > > > > >> > > > >> > >> > > > > > > > you
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> explain
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> that we should send a
> > > > > >> > > > >> > >> DescribeTransactionRequest,
> > > > > >> > > > >> > >> > > but I
> > > > > >> > > > >> > >> > > > > am
> > > > > >> > > > >> > >> > > > > > > not
> > > > > >> > > > >> > >> > > > > > > > > sure
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> why?
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Can't we not just do an
> > > > implicit
> > > > > >> > > > >> > >> AddPartiitonToTx,
> > > > > >> > > > >> > >> > > too?
> > > > > >> > > > >> > >> > > > > If
> > > > > >> > > > >> > >> > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > old
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> producer correctly
> > registered
> > > > the
> > > > > >> > > > partition
> > > > > >> > > > >> > >> > already,
> > > > > >> > > > >> > >> > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> TX-coordinator
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> can just ignore it as
> > it's an
> > > > > >> > > idempotent
> > > > > >> > > > >> > >> operation?
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (30)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> To cover older clients,
> > we
> > > > will
> > > > > >> > > ensure a
> > > > > >> > > > >> > >> > transaction
> > > > > >> > > > >> > >> > > > is
> > > > > >> > > > >> > >> > > > > > > > ongoing
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> before
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> we write to a
> transaction
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Not sure what you mean
> by
> > > > this?
> > > > > >> Can
> > > > > >> > you
> > > > > >> > > > >> > >> elaborate?
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (40)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> [the TX-coordinator]
> will
> > > > write
> > > > > >> the
> > > > > >> > > > >> prepare
> > > > > >> > > > >> > >> commit
> > > > > >> > > > >> > >> > > > > message
> > > > > >> > > > >> > >> > > > > > > > with
> > > > > >> > > > >> > >> > > > > > > > > a
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> bumped
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> epoch and send
> > > > > >> WriteTxnMarkerRequests
> > > > > >> > > > with
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> > bumped
> > > > > >> > > > >> > >> > > > > > epoch.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Why do we use the bumped
> > > > epoch for
> > > > > >> > > both?
> > > > > >> > > > It
> > > > > >> > > > >> > >> seems
> > > > > >> > > > >> > >> > > more
> > > > > >> > > > >> > >> > > > > > > > intuitive
> > > > > >> > > > >> > >> > > > > > > > > to
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> use
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> the current epoch, and
> > only
> > > > return
> > > > > >> > the
> > > > > >> > > > >> bumped
> > > > > >> > > > >> > >> epoch
> > > > > >> > > > >> > >> > > to
> > > > > >> > > > >> > >> > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>>>> producer?
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (50) "Implicit
> > > > > >> > > AddPartitionToTransaction"
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Why does the implicitly
> > sent
> > > > > >> request
> > > > > >> > > need
> > > > > >> > > > >> to
> > > > > >> > > > >> > be
> > > > > >> > > > >> > >> > > > > > synchronous?
> > > > > >> > > > >> > >> > > > > > > > The
> > > > > >> > > > >> > >> > > > > > > > > >>>>> KIP
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> also says
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> in case we need to
> abort
> > and
> > > > > >> need to
> > > > > >> > > > know
> > > > > >> > > > >> > which
> > > > > >> > > > >> > >> > > > > partitions
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> What do you mean by
> this?
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> we don’t want to write
> > to it
> > > > > >> before
> > > > > >> > we
> > > > > >> > > > >> store
> > > > > >> > > > >> > in
> > > > > >> > > > >> > >> > the
> > > > > >> > > > >> > >> > > > > > > > transaction
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> manager
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Do you mean
> TX-coordinator
> > > > > >> instead of
> > > > > >> > > > >> > "manager"?
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (60)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> For older clients and
> > ensuring
> > > > > >> that
> > > > > >> > the
> > > > > >> > > > TX
> > > > > >> > > > >> is
> > > > > >> > > > >> > >> > > ongoing,
> > > > > >> > > > >> > >> > > > > you
> > > > > >> > > > >> > >> > > > > > > > > >>>>> describe a
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> race condition. I am not
> > sure
> > > > if I
> > > > > >> > can
> > > > > >> > > > >> follow
> > > > > >> > > > >> > >> here.
> > > > > >> > > > >> > >> > > Can
> > > > > >> > > > >> > >> > > > > you
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> elaborate?
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> -Matthias
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> On 11/18/22 1:21 PM,
> > Justine
> > > > > >> Olshan
> > > > > >> > > > wrote:
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Hey all!
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> I'd like to start a
> > > > discussion
> > > > > >> on my
> > > > > >> > > > >> proposal
> > > > > >> > > > >> > >> to
> > > > > >> > > > >> > >> > add
> > > > > >> > > > >> > >> > > > > some
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> server-side
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> checks on transactions
> to
> > > > avoid
> > > > > >> > > hanging
> > > > > >> > > > >> > >> > > transactions.
> > > > > >> > > > >> > >> > > > I
> > > > > >> > > > >> > >> > > > > > know
> > > > > >> > > > >> > >> > > > > > > > > this
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> has
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> been
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> an issue for some time,
> > so I
> > > > > >> really
> > > > > >> > > hope
> > > > > >> > > > >> this
> > > > > >> > > > >> > >> KIP
> > > > > >> > > > >> > >> > > will
> > > > > >> > > > >> > >> > > > > be
> > > > > >> > > > >> > >> > > > > > > > > helpful
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> for
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> many
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> users of EOS.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> The KIP includes
> changes
> > that
> > > > > >> will
> > > > > >> > be
> > > > > >> > > > >> > >> compatible
> > > > > >> > > > >> > >> > > with
> > > > > >> > > > >> > >> > > > > old
> > > > > >> > > > >> > >> > > > > > > > > clients
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> and
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> changes to improve
> > > > performance
> > > > > >> and
> > > > > >> > > > >> > correctness
> > > > > >> > > > >> > >> on
> > > > > >> > > > >> > >> > > new
> > > > > >> > > > >> > >> > > > > > > clients.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Please take a look and
> > leave
> > > > any
> > > > > >> > > > comments
> > > > > >> > > > >> you
> > > > > >> > > > >> > >> may
> > > > > >> > > > >> > >> > > > have!
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> KIP:
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > >
> > > > > >> > > > >> > >> > > > >
> > > > > >> > > > >> > >> > > >
> > > > > >> > > > >> > >> > >
> > > > > >> > > > >> > >> >
> > > > > >> > > > >> > >>
> > > > > >> > > > >> >
> > > > > >> > > > >>
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> JIRA:
> > > > > >> > > > >> > >> > > >
> > https://issues.apache.org/jira/browse/KAFKA-14402
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Thanks!
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Justine
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > >
> > > > > >> > > > >> > >> > > > >
> > > > > >> > > > >> > >> > > >
> > > > > >> > > > >> > >> > >
> > > > > >> > > > >> > >> >
> > > > > >> > > > >> > >>
> > > > > >> > > > >> > >
> > > > > >> > > > >> >
> > > > > >> > > > >>
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > > >
> > > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Artem Livshits <al...@confluent.io.INVALID>.
> The code that references Short.MAX_VALUE is on the client

Do you mean this code?
https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java#L474
This is for an idempotent producer, not transactional.

> Are you concerned about non-Java clients here or something else?

I'm sure we'll figure out the correct behavior in the Java client :-), but
Kafka is a public API and we have multiple clients that need to adapt to
the new API correctly.

-Artem

On Fri, Jan 20, 2023 at 3:22 PM Justine Olshan <jo...@confluent.io.invalid>
wrote:

> Artem -- I'm a bit confused because the epoch bump is done from the client
> side. The code that references Short.MAX_VALUE is on the client. Are you
> concerned about non-Java clients here or something else?
> I can add some explicit text in the KIP about this. But my understanding is
> that it could work something like the following:
> 1. server returns max value after transaction completes
> 2. client will check epoch and if it is max value, allocate new producer
> ID.
>
> The only difference here is instead of client bumping epoch itself and
> doing this check, it must do the check when handling the response from
> committing the transaction.
>
> Guozhang -- yup that makes sense. I see the same. I just wanted to confirm
> your suggestion was to shut down the producer totally and not just fail the
> batch. I think we are aligned here.
>
> Thanks again for the discussion all!
> Justine
>
> On Fri, Jan 20, 2023 at 2:57 PM Guozhang Wang <gu...@gmail.com>
> wrote:
>
> > For InvalidRecord -- when I did the research on KIP-691 I saw from the
> > code we would treat them as abortableError, not fatalError inside
> > Producer's TxnManager (again sorry for raising such implementation
> > details in KIP discussions), so I think, from client's perspective,
> > they will not cause the client to be shutdown automatically?
> >
> > On Fri, Jan 20, 2023 at 2:52 PM Justine Olshan
> > <jo...@confluent.io.invalid> wrote:
> > >
> > > That's a fair point about other clients.
> > >
> > > I think the abortable error case is interesting because I'm curious how
> > > other clients would handle this. I assume they would need to implement
> > > handling for the error code unless they did something like "any unknown
> > > error codes/any codes that aren't x,y,z are retriable." I would hope
> that
> > > unknown error codes were fatal, and if the code was implemented it
> would
> > > abort the transaction. But I will think on this too.
> > >
> > > As for InvalidRecord -- you mentioned it was not fatal, but I'm taking
> a
> > > look through the code. We would see this on handling the produce
> > response.
> > > If I recall correctly, we check if errors are retriable. I think this
> > error
> > > would not be retriable. But I guess the concern here is that it is not
> > > enough for just that batch to fail. I guess I hadn't considered fully
> > > fencing the old producer but there are valid arguments here why we
> would
> > > want to.
> > >
> > > Thanks,
> > > Justine
> > >
> > > On Fri, Jan 20, 2023 at 2:35 PM Guozhang Wang <
> > guozhang.wang.us@gmail.com>
> > > wrote:
> > >
> > > > Thanks Justine for the replies! I agree with most of your thoughts.
> > > >
> > > > Just for 3/7), though I agree for our own AK producer, since we do
> > > > "nextRequest(boolean hasIncompleteBatches)", we guarantee the end-txn
> > > > would not be sent until we've effectively flushed, but I was
> referring
> > > > to any future bugs or other buggy clients that the same client may
> get
> > > > into this situation, in which case we should give the client a clear
> > > > msg that "you did something wrong, and hence now you should fatally
> > > > close yourself". What I'm concerned about is that, by seeing an
> > > > "abortable error" or in some rare cases an "invalid record", the
> > > > client could not realize "something that's really bad happened". So
> > > > it's not about adding a new error, it's mainly about those real buggy
> > > > situations causing such "should never happen" cases, the errors
> return
> > > > would not be informative enough.
> > > >
> > > > Thinking in other ways, if we believe that for most cases such error
> > > > codes would not reach the original clients since they would be
> > > > disconnected or even gone by that time, and only in some rare cases
> > > > they would still be seen by the sending clients, then why not make
> > > > them more fatal and more specific than generic.
> > > >
> > > > Guozhang
> > > >
> > > > On Fri, Jan 20, 2023 at 1:59 PM Justine Olshan
> > > > <jo...@confluent.io.invalid> wrote:
> > > > >
> > > > > Hey Guozhang. Thanks for taking a look and for the detailed
> comments!
> > > > I'll
> > > > > do my best to address below.
> > > > >
> > > > > 1. I see what you are saying here, but I think I need to look
> > through the
> > > > > sequence of events you mention. Typically we've seen this issue in
> a
> > few
> > > > > cases.
> > > > >
> > > > >  One is when we have a producer disconnect when trying to produce.
> > > > > Typically in these cases, we abort the transaction. We've seen that
> > after
> > > > > the markers are written, the disconnection can sometimes cause the
> > > > request
> > > > > to get flushed to the broker. In this case, we don't need client
> > handling
> > > > > because the producer we are responding to is gone. We just needed
> to
> > make
> > > > > sure we didn't write to the log on the broker side. I'm trying to
> > think
> > > > of
> > > > > a case where we do have the client to return to. I'd think the same
> > > > client
> > > > > couldn't progress to committing the transaction unless the produce
> > > > request
> > > > > returned right? Of course, there is the incorrectly written clients
> > case.
> > > > > I'll think on this a bit more and let you know if I come up with
> > another
> > > > > scenario when we would return to an active client when the
> > transaction is
> > > > > no longer ongoing.
> > > > >
> > > > > I was not aware that we checked the result of a send after we
> commit
> > > > > though. I'll need to look into that a bit more.
> > > > >
> > > > > 2. There were some questions about this in the discussion. The plan
> > is to
> > > > > handle overflow with the mechanism we currently have in the
> > producer. If
> > > > we
> > > > > try to bump and the epoch will overflow, we actually allocate a new
> > > > > producer ID. I need to confirm the fencing logic on the last epoch
> > (ie,
> > > > we
> > > > > probably shouldn't allow any records to be produced with the final
> > epoch
> > > > > since we can never properly fence that one).
> > > > >
> > > > > 3. I can agree with you that the current error handling is messy. I
> > > > recall
> > > > > taking a look at your KIP a while back, but I think I mostly saw
> the
> > > > > section about how the errors were wrapped. Maybe I need to take
> > another
> > > > > look. As for abortable error, the idea was that the handling would
> be
> > > > > simple -- if this error is seen, the transaction should be aborted
> > -- no
> > > > > other logic about previous state or requests necessary. Is your
> > concern
> > > > > simply about adding new errors? We were hoping to have an error
> that
> > > > would
> > > > > have one meaning and many of the current errors have a history of
> > meaning
> > > > > different things on different client versions. That was the main
> > > > motivation
> > > > > for adding a new error.
> > > > >
> > > > > 4. This is a good point about record timestamp reordering.
> Timestamps
> > > > don't
> > > > > affect compaction, but they do affect retention deletion. For that,
> > kafka
> > > > > considers the largest timestamp in the segment, so I think a small
> > amount
> > > > > of reordering (hopefully on the order of milliseconds or even
> > seconds)
> > > > will
> > > > > be ok. We take timestamps from clients so there is already a
> > possibility
> > > > > for some drift and non-monotonically increasing timestamps.
> > > > >
> > > > > 5. Thanks for catching. The error is there, but it's actually that
> > those
> > > > > fields should be 4+! Due to how the message generator works, I
> > actually
> > > > > have to redefine those fields inside the
> > `"AddPartitionsToTxnTransaction`
> > > > > block for it to build correctly. I'll fix it to be correct.
> > > > >
> > > > > 6. Correct -- we will only add the request to purgatory if the
> cache
> > has
> > > > no
> > > > > ongoing transaction. I can change the wording to make that clearer
> > that
> > > > we
> > > > > only place the request in purgatory if we need to contact the
> > transaction
> > > > > coordinator.
> > > > >
> > > > > 7. We did take a look at some of the errors and it was hard to come
> > up
> > > > with
> > > > > a good one. I agree that InvalidTxnStateException is ideal except
> > for the
> > > > > fact that it hasn't been returned on Produce requests before. The
> > error
> > > > > handling for clients is a bit vague (which is why I opened
> > KAFKA-14439
> > > > > <https://issues.apache.org/jira/browse/KAFKA-14439>), but the
> > decision
> > > > we
> > > > > made here was to only return errors that have been previously
> > returned to
> > > > > producers. As for not being fatal, I think part of the theory was
> > that in
> > > > > many cases, the producer would be disconnected. (See point 1) and
> > this
> > > > > would just be an error to return from the server. I did plan to
> think
> > > > about
> > > > > other cases, so let me know if you think of any as well!
> > > > >
> > > > > Lots to say! Let me know if you have further thoughts!
> > > > > Justine
> > > > >
> > > > > On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang <
> > > > guozhang.wang.us@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hello Justine,
> > > > > >
> > > > > > Thanks for the great write-up! I made a quick pass through it and
> > here
> > > > > > are some thoughts (I have not been able to read through this
> > thread so
> > > > > > pardon me if they have overlapped or subsumed by previous
> > comments):
> > > > > >
> > > > > > First are some meta ones:
> > > > > >
> > > > > > 1. I think we need to also improve the client's experience once
> we
> > > > > > have this defence in place. More concretely, say a user's
> producer
> > > > > > code is like following:
> > > > > >
> > > > > > future = producer.send();
> > > > > > // producer.flush();
> > > > > > producer.commitTransaction();
> > > > > > future.get();
> > > > > >
> > > > > > Which resulted in the order of a) produce-request sent by
> > producer, b)
> > > > > > end-txn-request sent by producer, c) end-txn-response sent back,
> d)
> > > > > > txn-marker-request sent from coordinator to partition leader, e)
> > > > > > produce-request finally received by the partition leader, before
> > this
> > > > > > KIP e) step would be accepted causing a dangling txn; now it
> would
> > be
> > > > > > rejected in step e) which is good. But from the client's point of
> > view
> > > > > > now it becomes confusing since the `commitTransaction()` returns
> > > > > > successfully, but the "future" throws an invalid-epoch error, and
> > they
> > > > > > are not sure if the transaction did succeed or not. In fact, it
> > > > > > "partially succeeded" with some msgs being rejected but others
> > > > > > committed successfully.
> > > > > >
> > > > > > Of course the easy way to avoid this is, always call
> > > > > > "producer.flush()" before commitTxn and that's what we do
> > ourselves,
> > > > > > and what we recommend users do. But I suspect not everyone does
> > it. In
> > > > > > fact I just checked the javadoc in KafkaProducer and our code
> > snippet
> > > > > > does not include a `flush()` call. So I'm thinking maybe we can
> in
> > > > > > side the `commitTxn` code to enforce flushing before sending the
> > > > > > end-txn request.
> > > > > >
> > > > > > 2. I'd like to clarify a bit details on "just add partitions to
> the
> > > > > > transaction on the first produce request during a transaction".
> My
> > > > > > understanding is that the partition leader's cache has the
> > producer id
> > > > > > / sequence / epoch for the latest txn, either on-going or is
> > completed
> > > > > > (upon receiving the marker request from coordinator). When a
> > produce
> > > > > > request is received, if
> > > > > >
> > > > > > * producer's epoch < cached epoch, or producer's epoch == cached
> > epoch
> > > > > > but the latest txn is completed, leader directly reject with
> > > > > > invalid-epoch.
> > > > > > * producer's epoch > cached epoch, park the the request and send
> > > > > > add-partitions request to coordinator.
> > > > > >
> > > > > > In order to do it, does the coordinator need to bump the sequence
> > and
> > > > > > reset epoch to 0 when the next epoch is going to overflow? If no
> > need
> > > > > > to do so, then how we handle the (admittedly rare, but still may
> > > > > > happen) epoch overflow situation?
> > > > > >
> > > > > > 3. I'm a bit concerned about adding a generic "ABORTABLE_ERROR"
> > given
> > > > > > we already have a pretty messy error classification and error
> > handling
> > > > > > on the producer clients side --- I have a summary about the
> issues
> > and
> > > > > > a proposal to address this in
> > > > > >
> > > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
> > > > > > -- I understand we do not want to use "UNKNOWN_PRODUCER_ID"
> anymore
> > > > > > and in fact we intend to deprecate it in KIP-360 and eventually
> > remove
> > > > > > it; but I'm wondering can we still use specific error codes. E.g.
> > what
> > > > > > about "InvalidProducerEpochException" since for new clients, the
> > > > > > actual reason this would actually be rejected is indeed because
> the
> > > > > > epoch on the coordinator caused the add-partitions-request from
> the
> > > > > > brokers to be rejected anyways?
> > > > > >
> > > > > > 4. It seems we put the producer request into purgatory before we
> > ever
> > > > > > append the records, while other producer's records may still be
> > > > > > appended during the time; and that potentially may result in some
> > > > > > re-ordering compared with reception order. I'm not super
> concerned
> > > > > > about it since Kafka does not guarantee reception ordering across
> > > > > > producers anyways, but it may make the timestamps of records
> > inside a
> > > > > > partition to be more out-of-ordered. Are we aware of any
> scenarios
> > > > > > such as future enhancements on log compactions that may be
> > affected by
> > > > > > this effect?
> > > > > >
> > > > > > Below are just minor comments:
> > > > > >
> > > > > > 5. In "AddPartitionsToTxnTransaction" field of
> > > > > > "AddPartitionsToTxnRequest" RPC, the versions of those inner
> fields
> > > > > > are "0-3" while I thought they should be "0+" still?
> > > > > >
> > > > > > 6. Regarding "we can place the request in a purgatory of sorts
> and
> > > > > > check if there is any state for the transaction on the broker": i
> > > > > > think at this time when we just do the checks against the cached
> > > > > > state, we do not need to put the request to purgatory yet?
> > > > > >
> > > > > > 7. This is related to 3) above. I feel using
> > "InvalidRecordException"
> > > > > > for older clients may also be a bit confusing, and also it is not
> > > > > > fatal -- for old clients, it better to be fatal since this
> > indicates
> > > > > > the clients is doing something wrong and hence it should be
> closed.
> > > > > > And in general I'd prefer to use slightly more specific meaning
> > error
> > > > > > codes for clients. That being said, I also feel
> > > > > > "InvalidProducerEpochException" is not suitable for old versioned
> > > > > > clients, and we'd have to pick one that old clients recognize.
> I'd
> > > > > > prefer "InvalidTxnStateException" but that one is supposed to be
> > > > > > returned from txn coordinators only today. I'd suggest we do a
> > quick
> > > > > > check in the current client's code path and see if that one would
> > be
> > > > > > handled if it's from a produce-response, and if yes, use this
> one;
> > > > > > otherwise, use "ProducerFencedException" which is much less
> > meaningful
> > > > > > but it's still a fatal error.
> > > > > >
> > > > > >
> > > > > > Thanks,
> > > > > > Guozhang
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
> > > > > > <jo...@confluent.io.invalid> wrote:
> > > > > > >
> > > > > > > Yeah -- looks like we already have code to handle bumping the
> > epoch
> > > > and
> > > > > > > when the epoch is Short.MAX_VALUE, we get a new producer ID.
> > Since
> > > > this
> > > > > > is
> > > > > > > already the behavior, do we want to change it further?
> > > > > > >
> > > > > > > Justine
> > > > > > >
> > > > > > > On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <
> > jolshan@confluent.io
> > > > >
> > > > > > wrote:
> > > > > > >
> > > > > > > > Hey all, just wanted to quickly update and say I've modified
> > the
> > > > KIP to
> > > > > > > > explicitly mention that AddOffsetCommitsToTxnRequest will be
> > > > replaced
> > > > > > by
> > > > > > > > a coordinator-side (inter-broker) AddPartitionsToTxn implicit
> > > > request.
> > > > > > This
> > > > > > > > mirrors the user partitions and will implicitly add offset
> > > > partitions
> > > > > > to
> > > > > > > > transactions when we commit offsets on them. We will
> deprecate
> > > > > > AddOffsetCommitsToTxnRequest
> > > > > > > > for new clients.
> > > > > > > >
> > > > > > > > Also to address Artem's comments --
> > > > > > > > I'm a bit unsure if the changes here will change the previous
> > > > behavior
> > > > > > for
> > > > > > > > fencing producers. In the case you mention in the first
> > paragraph,
> > > > are
> > > > > > you
> > > > > > > > saying we bump the epoch before we try to abort the
> > transaction? I
> > > > > > think I
> > > > > > > > need to understand the scenarios you mention a bit better.
> > > > > > > >
> > > > > > > > As for the second part -- I think it makes sense to have some
> > sort
> > > > of
> > > > > > > > "sentinel" epoch to signal epoch is about to overflow (I
> think
> > we
> > > > sort
> > > > > > of
> > > > > > > > have this value in place in some ways) so we can codify it in
> > the
> > > > KIP.
> > > > > > I'll
> > > > > > > > look into that and try to update soon.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Justine.
> > > > > > > >
> > > > > > > > On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
> > > > > > > > <al...@confluent.io.invalid> wrote:
> > > > > > > >
> > > > > > > >> It's good to know that KIP-588 addressed some of the issues.
> > > > Looking
> > > > > > at
> > > > > > > >> the code, it still looks like there are some cases that
> would
> > > > result
> > > > > > in
> > > > > > > >> fatal error, e.g. PRODUCER_FENCED is issued by the
> transaction
> > > > > > coordinator
> > > > > > > >> if epoch doesn't match, and the client treats it as a fatal
> > error
> > > > > > (code in
> > > > > > > >> TransactionManager request handling).  If we consider, for
> > > > example,
> > > > > > > >> committing a transaction that returns a timeout, but
> actually
> > > > > > succeeds,
> > > > > > > >> trying to abort it or re-commit may result in
> PRODUCER_FENCED
> > > > error
> > > > > > > >> (because of epoch bump).
> > > > > > > >>
> > > > > > > >> For failed commits, specifically, we need to know the actual
> > > > outcome,
> > > > > > > >> because if we return an error the application may think that
> > the
> > > > > > > >> transaction is aborted and redo the work, leading to
> > duplicates.
> > > > > > > >>
> > > > > > > >> Re: overflowing epoch.  We could either do it on the TC and
> > return
> > > > > > both
> > > > > > > >> producer id and epoch (e.g. change the protocol), or signal
> > the
> > > > client
> > > > > > > >> that
> > > > > > > >> it needs to get a new producer id.  Checking for max epoch
> > could
> > > > be a
> > > > > > > >> reasonable signal, the value to check should probably be
> > present
> > > > in
> > > > > > the
> > > > > > > >> KIP
> > > > > > > >> as this is effectively a part of the contract.  Also, the TC
> > > > should
> > > > > > > >> probably return an error if the client didn't change
> producer
> > id
> > > > after
> > > > > > > >> hitting max epoch.
> > > > > > > >>
> > > > > > > >> -Artem
> > > > > > > >>
> > > > > > > >>
> > > > > > > >> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
> > > > > > > >> <jo...@confluent.io.invalid> wrote:
> > > > > > > >>
> > > > > > > >> > Thanks for the discussion Artem.
> > > > > > > >> >
> > > > > > > >> > With respect to the handling of fenced producers, we have
> > some
> > > > > > behavior
> > > > > > > >> > already in place. As of KIP-588:
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >>
> > > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> > > > > > > >> > ,
> > > > > > > >> > we handle timeouts more gracefully. The producer can
> > recover.
> > > > > > > >> >
> > > > > > > >> > Produce requests can also recover from epoch fencing by
> > > > aborting the
> > > > > > > >> > transaction and starting over.
> > > > > > > >> >
> > > > > > > >> > What other cases were you considering that would cause us
> to
> > > > have a
> > > > > > > >> fenced
> > > > > > > >> > epoch but we'd want to recover?
> > > > > > > >> >
> > > > > > > >> > The first point about handling epoch overflows is fair. I
> > think
> > > > > > there is
> > > > > > > >> > some logic we'd need to consider. (ie, if we are one away
> > from
> > > > the
> > > > > > max
> > > > > > > >> > epoch, we need to reset the producer ID.) I'm still
> > wondering if
> > > > > > there
> > > > > > > >> is a
> > > > > > > >> > way to direct this from the response, or if everything
> > should be
> > > > > > done on
> > > > > > > >> > the client side. Let me know if you have any thoughts
> here.
> > > > > > > >> >
> > > > > > > >> > Thanks,
> > > > > > > >> > Justine
> > > > > > > >> >
> > > > > > > >> > On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
> > > > > > > >> > <al...@confluent.io.invalid> wrote:
> > > > > > > >> >
> > > > > > > >> > > There are some workflows in the client that are implied
> by
> > > > > > protocol
> > > > > > > >> > > changes, e.g.:
> > > > > > > >> > >
> > > > > > > >> > > - for new clients, epoch changes with every transaction
> > and
> > > > can
> > > > > > > >> overflow,
> > > > > > > >> > > in old clients this condition was handled transparently,
> > > > because
> > > > > > epoch
> > > > > > > >> > was
> > > > > > > >> > > bumped in InitProducerId and it would return a new
> > producer
> > > > id if
> > > > > > > >> epoch
> > > > > > > >> > > overflows, the new clients would need to implement some
> > > > workflow
> > > > > > to
> > > > > > > >> > refresh
> > > > > > > >> > > producer id
> > > > > > > >> > > - how to handle fenced producers, for new clients epoch
> > > > changes
> > > > > > with
> > > > > > > >> > every
> > > > > > > >> > > transaction, so in presence of failures during commits /
> > > > aborts,
> > > > > > the
> > > > > > > >> > > producer could get easily fenced, old clients would
> pretty
> > > > much
> > > > > > would
> > > > > > > >> get
> > > > > > > >> > > fenced when a new incarnation of the producer was
> > initialized
> > > > with
> > > > > > > >> > > InitProducerId so it's ok to treat as a fatal error, the
> > new
> > > > > > clients
> > > > > > > >> > would
> > > > > > > >> > > need to implement some workflow to handle that error,
> > > > otherwise
> > > > > > they
> > > > > > > >> > could
> > > > > > > >> > > get fenced by themselves
> > > > > > > >> > > - in particular (as a subset of the previous issue),
> what
> > > > would
> > > > > > the
> > > > > > > >> > client
> > > > > > > >> > > do if it got a timeout during commit?  commit could've
> > > > succeeded
> > > > > > or
> > > > > > > >> > failed
> > > > > > > >> > >
> > > > > > > >> > > Not sure if this has to be defined in the KIP as
> > implementing
> > > > > > those
> > > > > > > >> > > probably wouldn't require protocol changes, but we have
> > > > multiple
> > > > > > > >> > > implementations of Kafka clients, so probably would be
> > good to
> > > > > > have
> > > > > > > >> some
> > > > > > > >> > > client implementation guidance.  Could also be done as a
> > > > separate
> > > > > > doc.
> > > > > > > >> > >
> > > > > > > >> > > -Artem
> > > > > > > >> > >
> > > > > > > >> > > On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
> > > > > > > >> > <jolshan@confluent.io.invalid
> > > > > > > >> > > >
> > > > > > > >> > > wrote:
> > > > > > > >> > >
> > > > > > > >> > > > Hey all, I've updated the KIP to incorporate Jason's
> > > > > > suggestions.
> > > > > > > >> > > >
> > > > > > > >> > > >
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >>
> > > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > > > >> > > >
> > > > > > > >> > > >
> > > > > > > >> > > > 1. Use AddPartitionsToTxn + verify flag to check on
> old
> > > > clients
> > > > > > > >> > > > 2. Updated AddPartitionsToTxn API to support
> transaction
> > > > > > batching
> > > > > > > >> > > > 3. Mention IBP bump
> > > > > > > >> > > > 4. Mention auth change on new AddPartitionsToTxn
> > version.
> > > > > > > >> > > >
> > > > > > > >> > > > I'm planning on opening a vote soon.
> > > > > > > >> > > > Thanks,
> > > > > > > >> > > > Justine
> > > > > > > >> > > >
> > > > > > > >> > > > On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <
> > > > > > jolshan@confluent.io
> > > > > > > >> >
> > > > > > > >> > > > wrote:
> > > > > > > >> > > >
> > > > > > > >> > > > > Thanks Jason. Those changes make sense to me. I will
> > > > update
> > > > > > the
> > > > > > > >> KIP.
> > > > > > > >> > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > > > On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
> > > > > > > >> > > > <ja...@confluent.io.invalid>
> > > > > > > >> > > > > wrote:
> > > > > > > >> > > > >
> > > > > > > >> > > > >> Hey Justine,
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> > I was wondering about compatibility here. When we
> > send
> > > > > > requests
> > > > > > > >> > > > >> between brokers, we want to ensure that the
> receiving
> > > > broker
> > > > > > > >> > > understands
> > > > > > > >> > > > >> the request (specifically the new fields).
> Typically
> > > > this is
> > > > > > done
> > > > > > > >> > via
> > > > > > > >> > > > >> IBP/metadata version.
> > > > > > > >> > > > >> I'm trying to think if there is a way around it but
> > I'm
> > > > not
> > > > > > sure
> > > > > > > >> > there
> > > > > > > >> > > > is.
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> Yes. I think we would gate usage of this behind an
> > IBP
> > > > bump.
> > > > > > Does
> > > > > > > >> > that
> > > > > > > >> > > > >> seem
> > > > > > > >> > > > >> reasonable?
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> > As for the improvements -- can you clarify how
> the
> > > > multiple
> > > > > > > >> > > > >> transactional
> > > > > > > >> > > > >> IDs would help here? Were you thinking of a case
> > where we
> > > > > > > >> wait/batch
> > > > > > > >> > > > >> multiple produce requests together? My
> understanding
> > for
> > > > now
> > > > > > was
> > > > > > > >> 1
> > > > > > > >> > > > >> transactional ID and one validation per 1 produce
> > > > request.
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> Each call to `AddPartitionsToTxn` is essentially a
> > write
> > > > to
> > > > > > the
> > > > > > > >> > > > >> transaction
> > > > > > > >> > > > >> log and must block on replication. The more we can
> > fit
> > > > into a
> > > > > > > >> single
> > > > > > > >> > > > >> request, the more writes we can do in parallel. The
> > > > > > alternative
> > > > > > > >> is
> > > > > > > >> > to
> > > > > > > >> > > > make
> > > > > > > >> > > > >> use of more connections, but usually we prefer
> > batching
> > > > > > since the
> > > > > > > >> > > > network
> > > > > > > >> > > > >> stack is not really optimized for high
> > connection/request
> > > > > > loads.
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> > Finally with respect to the authorizations, I
> > think it
> > > > > > makes
> > > > > > > >> sense
> > > > > > > >> > > to
> > > > > > > >> > > > >> skip
> > > > > > > >> > > > >> topic authorizations, but I'm a bit confused by the
> > > > "leader
> > > > > > ID"
> > > > > > > >> > field.
> > > > > > > >> > > > >> Wouldn't we just want to flag the request as from a
> > > > broker
> > > > > > (does
> > > > > > > >> it
> > > > > > > >> > > > matter
> > > > > > > >> > > > >> which one?).
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> We could also make it version-based. For the next
> > > > version, we
> > > > > > > >> could
> > > > > > > >> > > > >> require
> > > > > > > >> > > > >> CLUSTER auth. So clients would not be able to use
> > the API
> > > > > > > >> anymore,
> > > > > > > >> > > which
> > > > > > > >> > > > >> is
> > > > > > > >> > > > >> probably what we want.
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> -Jason
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> > > > > > > >> > > > >> <jo...@confluent.io.invalid>
> > > > > > > >> > > > >> wrote:
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> > As a follow up, I was just thinking about the
> > batching
> > > > a
> > > > > > bit
> > > > > > > >> more.
> > > > > > > >> > > > >> > I suppose if we have one request in flight and we
> > > > queue up
> > > > > > the
> > > > > > > >> > other
> > > > > > > >> > > > >> > produce requests in some sort of purgatory, we
> > could
> > > > send
> > > > > > > >> > > information
> > > > > > > >> > > > >> out
> > > > > > > >> > > > >> > for all of them rather than one by one. So that
> > would
> > > > be a
> > > > > > > >> benefit
> > > > > > > >> > > of
> > > > > > > >> > > > >> > batching partitions to add per transaction.
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > I'll need to think a bit more on the design of
> this
> > > > part
> > > > > > of the
> > > > > > > >> > KIP,
> > > > > > > >> > > > and
> > > > > > > >> > > > >> > will update the KIP in the next few days.
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > Thanks,
> > > > > > > >> > > > >> > Justine
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan <
> > > > > > > >> > > jolshan@confluent.io>
> > > > > > > >> > > > >> > wrote:
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > > Hey Jason -- thanks for the input -- I was just
> > > > digging
> > > > > > a bit
> > > > > > > >> > > deeper
> > > > > > > >> > > > >> into
> > > > > > > >> > > > >> > > the design + implementation of the validation
> > calls
> > > > here
> > > > > > and
> > > > > > > >> > what
> > > > > > > >> > > > you
> > > > > > > >> > > > >> say
> > > > > > > >> > > > >> > > makes sense.
> > > > > > > >> > > > >> > >
> > > > > > > >> > > > >> > > I was wondering about compatibility here. When
> we
> > > > send
> > > > > > > >> requests
> > > > > > > >> > > > >> > > between brokers, we want to ensure that the
> > receiving
> > > > > > broker
> > > > > > > >> > > > >> understands
> > > > > > > >> > > > >> > > the request (specifically the new fields).
> > Typically
> > > > > > this is
> > > > > > > >> > done
> > > > > > > >> > > > via
> > > > > > > >> > > > >> > > IBP/metadata version.
> > > > > > > >> > > > >> > > I'm trying to think if there is a way around it
> > but
> > > > I'm
> > > > > > not
> > > > > > > >> sure
> > > > > > > >> > > > there
> > > > > > > >> > > > >> > is.
> > > > > > > >> > > > >> > >
> > > > > > > >> > > > >> > > As for the improvements -- can you clarify how
> > the
> > > > > > multiple
> > > > > > > >> > > > >> transactional
> > > > > > > >> > > > >> > > IDs would help here? Were you thinking of a
> case
> > > > where we
> > > > > > > >> > > wait/batch
> > > > > > > >> > > > >> > > multiple produce requests together? My
> > understanding
> > > > for
> > > > > > now
> > > > > > > >> > was 1
> > > > > > > >> > > > >> > > transactional ID and one validation per 1
> produce
> > > > > > request.
> > > > > > > >> > > > >> > >
> > > > > > > >> > > > >> > > Finally with respect to the authorizations, I
> > think
> > > > it
> > > > > > makes
> > > > > > > >> > sense
> > > > > > > >> > > > to
> > > > > > > >> > > > >> > skip
> > > > > > > >> > > > >> > > topic authorizations, but I'm a bit confused by
> > the
> > > > > > "leader
> > > > > > > >> ID"
> > > > > > > >> > > > field.
> > > > > > > >> > > > >> > > Wouldn't we just want to flag the request as
> > from a
> > > > > > broker
> > > > > > > >> (does
> > > > > > > >> > > it
> > > > > > > >> > > > >> > matter
> > > > > > > >> > > > >> > > which one?).
> > > > > > > >> > > > >> > >
> > > > > > > >> > > > >> > > I think I want to adopt these suggestions, just
> > had
> > > > a few
> > > > > > > >> > > questions
> > > > > > > >> > > > on
> > > > > > > >> > > > >> > the
> > > > > > > >> > > > >> > > details.
> > > > > > > >> > > > >> > >
> > > > > > > >> > > > >> > > Thanks,
> > > > > > > >> > > > >> > > Justine
> > > > > > > >> > > > >> > >
> > > > > > > >> > > > >> > > On Thu, Jan 5, 2023 at 5:05 PM Jason Gustafson
> > > > > > > >> > > > >> > <ja...@confluent.io.invalid>
> > > > > > > >> > > > >> > > wrote:
> > > > > > > >> > > > >> > >
> > > > > > > >> > > > >> > >> Hi Justine,
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> > >> Thanks for the proposal.
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> > >> I was thinking about the implementation a
> little
> > > > bit.
> > > > > > In the
> > > > > > > >> > > > current
> > > > > > > >> > > > >> > >> proposal, the behavior depends on whether we
> > have an
> > > > > > old or
> > > > > > > >> new
> > > > > > > >> > > > >> client.
> > > > > > > >> > > > >> > >> For
> > > > > > > >> > > > >> > >> old clients, we send `DescribeTransactions`
> and
> > > > verify
> > > > > > the
> > > > > > > >> > result
> > > > > > > >> > > > and
> > > > > > > >> > > > >> > for
> > > > > > > >> > > > >> > >> new clients, we send `AddPartitionsToTxn`. We
> > might
> > > > be
> > > > > > able
> > > > > > > >> to
> > > > > > > >> > > > >> simplify
> > > > > > > >> > > > >> > >> the
> > > > > > > >> > > > >> > >> implementation if we can use the same request
> > type.
> > > > For
> > > > > > > >> > example,
> > > > > > > >> > > > >> what if
> > > > > > > >> > > > >> > >> we
> > > > > > > >> > > > >> > >> bump the protocol version for
> > `AddPartitionsToTxn`
> > > > and
> > > > > > add a
> > > > > > > >> > > > >> > >> `validateOnly`
> > > > > > > >> > > > >> > >> flag? For older versions, we can set
> > > > > > `validateOnly=true` so
> > > > > > > >> > that
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > >> request only returns successfully if the
> > partition
> > > > had
> > > > > > > >> already
> > > > > > > >> > > been
> > > > > > > >> > > > >> > added.
> > > > > > > >> > > > >> > >> For new versions, we can set
> > `validateOnly=false`
> > > > and
> > > > > > the
> > > > > > > >> > > partition
> > > > > > > >> > > > >> will
> > > > > > > >> > > > >> > >> be
> > > > > > > >> > > > >> > >> added to the transaction. The other slightly
> > > > annoying
> > > > > > thing
> > > > > > > >> > that
> > > > > > > >> > > > this
> > > > > > > >> > > > >> > >> would
> > > > > > > >> > > > >> > >> get around is the need to collect the
> > transaction
> > > > state
> > > > > > for
> > > > > > > >> all
> > > > > > > >> > > > >> > partitions
> > > > > > > >> > > > >> > >> even when we only care about a subset.
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> > >> Some additional improvements to consider:
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> > >> - We can give `AddPartitionsToTxn` better
> batch
> > > > support
> > > > > > for
> > > > > > > >> > > > >> inter-broker
> > > > > > > >> > > > >> > >> usage. Currently we only allow one
> > > > `TransactionalId` to
> > > > > > be
> > > > > > > >> > > > specified,
> > > > > > > >> > > > >> > but
> > > > > > > >> > > > >> > >> the broker may get some benefit being able to
> > batch
> > > > > > across
> > > > > > > >> > > multiple
> > > > > > > >> > > > >> > >> transactions.
> > > > > > > >> > > > >> > >> - Another small improvement is skipping topic
> > > > > > authorization
> > > > > > > >> > > checks
> > > > > > > >> > > > >> for
> > > > > > > >> > > > >> > >> `AddPartitionsToTxn` when the request is from
> a
> > > > broker.
> > > > > > > >> Perhaps
> > > > > > > >> > > we
> > > > > > > >> > > > >> can
> > > > > > > >> > > > >> > add
> > > > > > > >> > > > >> > >> a field for the `LeaderId` or something like
> > that
> > > > and
> > > > > > > >> require
> > > > > > > >> > > > CLUSTER
> > > > > > > >> > > > >> > >> permission when set.
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> > >> Best,
> > > > > > > >> > > > >> > >> Jason
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> > >> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
> > > > > > > >> > <jun@confluent.io.invalid
> > > > > > > >> > > >
> > > > > > > >> > > > >> > wrote:
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> > >> > Hi, Justine,
> > > > > > > >> > > > >> > >> >
> > > > > > > >> > > > >> > >> > Thanks for the explanation. It makes sense
> to
> > me
> > > > now.
> > > > > > > >> > > > >> > >> >
> > > > > > > >> > > > >> > >> > Jun
> > > > > > > >> > > > >> > >> >
> > > > > > > >> > > > >> > >> > On Mon, Dec 19, 2022 at 1:42 PM Justine
> Olshan
> > > > > > > >> > > > >> > >> > <jo...@confluent.io.invalid>
> > > > > > > >> > > > >> > >> > wrote:
> > > > > > > >> > > > >> > >> >
> > > > > > > >> > > > >> > >> > > Hi Jun,
> > > > > > > >> > > > >> > >> > >
> > > > > > > >> > > > >> > >> > > My understanding of the mechanism is that
> > when
> > > > we
> > > > > > get to
> > > > > > > >> > the
> > > > > > > >> > > > last
> > > > > > > >> > > > >> > >> epoch,
> > > > > > > >> > > > >> > >> > we
> > > > > > > >> > > > >> > >> > > increment to the fencing/last epoch and if
> > any
> > > > > > further
> > > > > > > >> > > requests
> > > > > > > >> > > > >> come
> > > > > > > >> > > > >> > >> in
> > > > > > > >> > > > >> > >> > for
> > > > > > > >> > > > >> > >> > > this producer ID they are fenced. Then the
> > > > producer
> > > > > > > >> gets a
> > > > > > > >> > > new
> > > > > > > >> > > > ID
> > > > > > > >> > > > >> > and
> > > > > > > >> > > > >> > >> > > restarts with epoch/sequence 0. The fenced
> > epoch
> > > > > > sticks
> > > > > > > >> > > around
> > > > > > > >> > > > >> for
> > > > > > > >> > > > >> > the
> > > > > > > >> > > > >> > >> > > duration of producer.id.expiration.ms and
> > > > blocks
> > > > > > any
> > > > > > > >> late
> > > > > > > >> > > > >> messages
> > > > > > > >> > > > >> > >> > there.
> > > > > > > >> > > > >> > >> > > The new ID will get to take advantage of
> the
> > > > > > improved
> > > > > > > >> > > semantics
> > > > > > > >> > > > >> > around
> > > > > > > >> > > > >> > >> > > non-zero start sequences. So I think we
> are
> > > > covered.
> > > > > > > >> > > > >> > >> > >
> > > > > > > >> > > > >> > >> > > The only potential issue is overloading
> the
> > > > cache,
> > > > > > but
> > > > > > > >> > > > hopefully
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> > > improvements (lowered
> > producer.id.expiration.ms
> > > > )
> > > > > > will
> > > > > > > >> help
> > > > > > > >> > > > with
> > > > > > > >> > > > >> > that.
> > > > > > > >> > > > >> > >> > Let
> > > > > > > >> > > > >> > >> > > me know if you still have concerns.
> > > > > > > >> > > > >> > >> > >
> > > > > > > >> > > > >> > >> > > Thanks,
> > > > > > > >> > > > >> > >> > > Justine
> > > > > > > >> > > > >> > >> > >
> > > > > > > >> > > > >> > >> > > On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
> > > > > > > >> > > > >> <ju...@confluent.io.invalid>
> > > > > > > >> > > > >> > >> > wrote:
> > > > > > > >> > > > >> > >> > >
> > > > > > > >> > > > >> > >> > > > Hi, Justine,
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > > >> > >> > > > Thanks for the explanation.
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > > >> > >> > > > 70. The proposed fencing logic doesn't
> > apply
> > > > when
> > > > > > pid
> > > > > > > >> > > > changes,
> > > > > > > >> > > > >> is
> > > > > > > >> > > > >> > >> that
> > > > > > > >> > > > >> > >> > > > right? If so, I am not sure how complete
> > we
> > > > are
> > > > > > > >> > addressing
> > > > > > > >> > > > this
> > > > > > > >> > > > >> > >> issue
> > > > > > > >> > > > >> > >> > if
> > > > > > > >> > > > >> > >> > > > the pid changes more frequently.
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > > >> > >> > > > Thanks,
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > > >> > >> > > > Jun
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > > >> > >> > > > On Fri, Dec 16, 2022 at 9:16 AM Justine
> > Olshan
> > > > > > > >> > > > >> > >> > > > <jo...@confluent.io.invalid>
> > > > > > > >> > > > >> > >> > > > wrote:
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > > >> > >> > > > > Hi Jun,
> > > > > > > >> > > > >> > >> > > > >
> > > > > > > >> > > > >> > >> > > > > Thanks for replying!
> > > > > > > >> > > > >> > >> > > > >
> > > > > > > >> > > > >> > >> > > > > 70.We already do the overflow
> > mechanism, so
> > > > my
> > > > > > > >> change
> > > > > > > >> > > would
> > > > > > > >> > > > >> just
> > > > > > > >> > > > >> > >> make
> > > > > > > >> > > > >> > >> > > it
> > > > > > > >> > > > >> > >> > > > > happen more often.
> > > > > > > >> > > > >> > >> > > > > I was also not suggesting a new field
> > in the
> > > > > > log,
> > > > > > > >> but
> > > > > > > >> > in
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > >> > response,
> > > > > > > >> > > > >> > >> > > > > which would be gated by the client
> > version.
> > > > > > Sorry if
> > > > > > > >> > > > >> something
> > > > > > > >> > > > >> > >> there
> > > > > > > >> > > > >> > >> > is
> > > > > > > >> > > > >> > >> > > > > unclear. I think we are starting to
> > diverge.
> > > > > > > >> > > > >> > >> > > > > The goal of this KIP is to not change
> > to the
> > > > > > marker
> > > > > > > >> > > format
> > > > > > > >> > > > at
> > > > > > > >> > > > >> > all.
> > > > > > > >> > > > >> > >> > > > >
> > > > > > > >> > > > >> > >> > > > > 71. Yes, I guess I was going under the
> > > > > > assumption
> > > > > > > >> that
> > > > > > > >> > > the
> > > > > > > >> > > > >> log
> > > > > > > >> > > > >> > >> would
> > > > > > > >> > > > >> > >> > > just
> > > > > > > >> > > > >> > >> > > > > look at its last epoch and treat it as
> > the
> > > > > > current
> > > > > > > >> > > epoch. I
> > > > > > > >> > > > >> > >> suppose
> > > > > > > >> > > > >> > >> > we
> > > > > > > >> > > > >> > >> > > > can
> > > > > > > >> > > > >> > >> > > > > have some special logic that if the
> last
> > > > epoch
> > > > > > was
> > > > > > > >> on a
> > > > > > > >> > > > >> marker
> > > > > > > >> > > > >> > we
> > > > > > > >> > > > >> > >> > > > actually
> > > > > > > >> > > > >> > >> > > > > expect the next epoch or something
> like
> > > > that. We
> > > > > > > >> just
> > > > > > > >> > > need
> > > > > > > >> > > > to
> > > > > > > >> > > > >> > >> > > distinguish
> > > > > > > >> > > > >> > >> > > > > based on whether we had a commit/abort
> > > > marker.
> > > > > > > >> > > > >> > >> > > > >
> > > > > > > >> > > > >> > >> > > > > 72.
> > > > > > > >> > > > >> > >> > > > > > if the producer epoch hasn't been
> > bumped
> > > > on
> > > > > > the
> > > > > > > >> > > > >> > >> > > > > broker, it seems that the stucked
> > message
> > > > will
> > > > > > fail
> > > > > > > >> the
> > > > > > > >> > > > >> sequence
> > > > > > > >> > > > >> > >> > > > validation
> > > > > > > >> > > > >> > >> > > > > and will be ignored. If the producer
> > epoch
> > > > has
> > > > > > been
> > > > > > > >> > > bumped,
> > > > > > > >> > > > >> we
> > > > > > > >> > > > >> > >> ignore
> > > > > > > >> > > > >> > >> > > the
> > > > > > > >> > > > >> > >> > > > > sequence check and the stuck message
> > could
> > > > be
> > > > > > > >> appended
> > > > > > > >> > to
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > log.
> > > > > > > >> > > > >> > >> > So,
> > > > > > > >> > > > >> > >> > > is
> > > > > > > >> > > > >> > >> > > > > the latter case that we want to guard?
> > > > > > > >> > > > >> > >> > > > >
> > > > > > > >> > > > >> > >> > > > > I'm not sure I follow that "the
> message
> > will
> > > > > > fail
> > > > > > > >> the
> > > > > > > >> > > > >> sequence
> > > > > > > >> > > > >> > >> > > > validation".
> > > > > > > >> > > > >> > >> > > > > In some of these cases, we had an
> abort
> > > > marker
> > > > > > (due
> > > > > > > >> to
> > > > > > > >> > an
> > > > > > > >> > > > >> error)
> > > > > > > >> > > > >> > >> and
> > > > > > > >> > > > >> > >> > > then
> > > > > > > >> > > > >> > >> > > > > the late message comes in with the
> > correct
> > > > > > sequence
> > > > > > > >> > > number.
> > > > > > > >> > > > >> This
> > > > > > > >> > > > >> > >> is a
> > > > > > > >> > > > >> > >> > > > case
> > > > > > > >> > > > >> > >> > > > > covered by the KIP.
> > > > > > > >> > > > >> > >> > > > > The latter case is actually not
> > something
> > > > we've
> > > > > > > >> > > considered
> > > > > > > >> > > > >> > here. I
> > > > > > > >> > > > >> > >> > > think
> > > > > > > >> > > > >> > >> > > > > generally when we bump the epoch, we
> are
> > > > > > accepting
> > > > > > > >> that
> > > > > > > >> > > the
> > > > > > > >> > > > >> > >> sequence
> > > > > > > >> > > > >> > >> > > does
> > > > > > > >> > > > >> > >> > > > > not need to be checked anymore. My
> > > > > > understanding is
> > > > > > > >> > also
> > > > > > > >> > > > >> that we
> > > > > > > >> > > > >> > >> > don't
> > > > > > > >> > > > >> > >> > > > > typically bump epoch mid transaction
> > (based
> > > > on a
> > > > > > > >> quick
> > > > > > > >> > > look
> > > > > > > >> > > > >> at
> > > > > > > >> > > > >> > the
> > > > > > > >> > > > >> > >> > > code)
> > > > > > > >> > > > >> > >> > > > > but let me know if that is the case.
> > > > > > > >> > > > >> > >> > > > >
> > > > > > > >> > > > >> > >> > > > > Thanks,
> > > > > > > >> > > > >> > >> > > > > Justine
> > > > > > > >> > > > >> > >> > > > >
> > > > > > > >> > > > >> > >> > > > > On Thu, Dec 15, 2022 at 12:23 PM Jun
> Rao
> > > > > > > >> > > > >> > <jun@confluent.io.invalid
> > > > > > > >> > > > >> > >> >
> > > > > > > >> > > > >> > >> > > > wrote:
> > > > > > > >> > > > >> > >> > > > >
> > > > > > > >> > > > >> > >> > > > > > Hi, Justine,
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> > > > >> > >> > > > > > Thanks for the reply.
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> > > > >> > >> > > > > > 70. Assigning a new pid on int
> > overflow
> > > > seems
> > > > > > a
> > > > > > > >> bit
> > > > > > > >> > > > hacky.
> > > > > > > >> > > > >> If
> > > > > > > >> > > > >> > we
> > > > > > > >> > > > >> > >> > > need a
> > > > > > > >> > > > >> > >> > > > > txn
> > > > > > > >> > > > >> > >> > > > > > level id, it will be better to model
> > this
> > > > > > > >> explicitly.
> > > > > > > >> > > > >> Adding a
> > > > > > > >> > > > >> > >> new
> > > > > > > >> > > > >> > >> > > > field
> > > > > > > >> > > > >> > >> > > > > > would require a bit more work since
> it
> > > > > > requires a
> > > > > > > >> new
> > > > > > > >> > > txn
> > > > > > > >> > > > >> > marker
> > > > > > > >> > > > >> > >> > > format
> > > > > > > >> > > > >> > >> > > > > in
> > > > > > > >> > > > >> > >> > > > > > the log. So, we probably need to
> > guard it
> > > > > > with an
> > > > > > > >> IBP
> > > > > > > >> > > or
> > > > > > > >> > > > >> > >> metadata
> > > > > > > >> > > > >> > >> > > > version
> > > > > > > >> > > > >> > >> > > > > > and document the impact on downgrade
> > once
> > > > the
> > > > > > new
> > > > > > > >> > > format
> > > > > > > >> > > > is
> > > > > > > >> > > > >> > >> written
> > > > > > > >> > > > >> > >> > > to
> > > > > > > >> > > > >> > >> > > > > the
> > > > > > > >> > > > >> > >> > > > > > log.
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> > > > >> > >> > > > > > 71. Hmm, once the marker is written,
> > the
> > > > > > partition
> > > > > > > >> > will
> > > > > > > >> > > > >> expect
> > > > > > > >> > > > >> > >> the
> > > > > > > >> > > > >> > >> > > next
> > > > > > > >> > > > >> > >> > > > > > append to be on the next epoch. Does
> > that
> > > > > > cover
> > > > > > > >> the
> > > > > > > >> > > case
> > > > > > > >> > > > >> you
> > > > > > > >> > > > >> > >> > > mentioned?
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> > > > >> > >> > > > > > 72. Also, just to be clear on the
> > stucked
> > > > > > message
> > > > > > > >> > issue
> > > > > > > >> > > > >> > >> described
> > > > > > > >> > > > >> > >> > in
> > > > > > > >> > > > >> > >> > > > the
> > > > > > > >> > > > >> > >> > > > > > motivation. With EoS, we also
> > validate the
> > > > > > > >> sequence
> > > > > > > >> > id
> > > > > > > >> > > > for
> > > > > > > >> > > > >> > >> > > idempotency.
> > > > > > > >> > > > >> > >> > > > > So,
> > > > > > > >> > > > >> > >> > > > > > with the current logic, if the
> > producer
> > > > epoch
> > > > > > > >> hasn't
> > > > > > > >> > > been
> > > > > > > >> > > > >> > >> bumped on
> > > > > > > >> > > > >> > >> > > the
> > > > > > > >> > > > >> > >> > > > > > broker, it seems that the stucked
> > message
> > > > will
> > > > > > > >> fail
> > > > > > > >> > the
> > > > > > > >> > > > >> > sequence
> > > > > > > >> > > > >> > >> > > > > validation
> > > > > > > >> > > > >> > >> > > > > > and will be ignored. If the producer
> > > > epoch has
> > > > > > > >> been
> > > > > > > >> > > > >> bumped, we
> > > > > > > >> > > > >> > >> > ignore
> > > > > > > >> > > > >> > >> > > > the
> > > > > > > >> > > > >> > >> > > > > > sequence check and the stuck message
> > > > could be
> > > > > > > >> > appended
> > > > > > > >> > > to
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> log.
> > > > > > > >> > > > >> > >> > > So,
> > > > > > > >> > > > >> > >> > > > is
> > > > > > > >> > > > >> > >> > > > > > the latter case that we want to
> guard?
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> > > > >> > >> > > > > > Thanks,
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> > > > >> > >> > > > > > Jun
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> > > > >> > >> > > > > > On Wed, Dec 14, 2022 at 10:44 AM
> > Justine
> > > > > > Olshan
> > > > > > > >> > > > >> > >> > > > > > <jo...@confluent.io.invalid>
> wrote:
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> > > > >> > >> > > > > > > Matthias — thanks again for taking
> > time
> > > > to
> > > > > > look
> > > > > > > >> a
> > > > > > > >> > > this.
> > > > > > > >> > > > >> You
> > > > > > > >> > > > >> > >> said:
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > My proposal was only focusing to
> > avoid
> > > > > > > >> dangling
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > transactions if records are added
> > > > without
> > > > > > > >> > registered
> > > > > > > >> > > > >> > >> partition.
> > > > > > > >> > > > >> > >> > --
> > > > > > > >> > > > >> > >> > > > > Maybe
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > you can add a few more details to
> > the
> > > > KIP
> > > > > > about
> > > > > > > >> > this
> > > > > > > >> > > > >> > scenario
> > > > > > > >> > > > >> > >> for
> > > > > > > >> > > > >> > >> > > > > better
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > documentation purpose?
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > I'm not sure I understand what you
> > mean
> > > > > > here.
> > > > > > > >> The
> > > > > > > >> > > > >> motivation
> > > > > > > >> > > > >> > >> > > section
> > > > > > > >> > > > >> > >> > > > > > > describes two scenarios about how
> > the
> > > > record
> > > > > > > >> can be
> > > > > > > >> > > > added
> > > > > > > >> > > > >> > >> > without a
> > > > > > > >> > > > >> > >> > > > > > > registered partition:
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > 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.
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > Another way hanging transactions
> > can
> > > > > > occur is
> > > > > > > >> > that
> > > > > > > >> > > a
> > > > > > > >> > > > >> > client
> > > > > > > >> > > > >> > >> is
> > > > > > > >> > > > >> > >> > > > buggy
> > > > > > > >> > > > >> > >> > > > > > and
> > > > > > > >> > > > >> > >> > > > > > > may somehow try to write to a
> > partition
> > > > > > before
> > > > > > > >> it
> > > > > > > >> > > adds
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> > > partition
> > > > > > > >> > > > >> > >> > > > to
> > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > transaction.
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > For the first example of this
> would
> > it
> > > > be
> > > > > > > >> helpful
> > > > > > > >> > to
> > > > > > > >> > > > say
> > > > > > > >> > > > >> > that
> > > > > > > >> > > > >> > >> > this
> > > > > > > >> > > > >> > >> > > > > > message
> > > > > > > >> > > > >> > >> > > > > > > comes in after the abort, but
> > before the
> > > > > > > >> partition
> > > > > > > >> > is
> > > > > > > >> > > > >> added
> > > > > > > >> > > > >> > to
> > > > > > > >> > > > >> > >> > the
> > > > > > > >> > > > >> > >> > > > next
> > > > > > > >> > > > >> > >> > > > > > > transaction so it becomes
> "hanging."
> > > > > > Perhaps the
> > > > > > > >> > next
> > > > > > > >> > > > >> > sentence
> > > > > > > >> > > > >> > >> > > > > describing
> > > > > > > >> > > > >> > >> > > > > > > the message becoming part of the
> > next
> > > > > > > >> transaction
> > > > > > > >> > (a
> > > > > > > >> > > > >> > different
> > > > > > > >> > > > >> > >> > > case)
> > > > > > > >> > > > >> > >> > > > > was
> > > > > > > >> > > > >> > >> > > > > > > not properly differentiated.
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > Jun — thanks for reading the KIP.
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > 70. The int typing was a concern.
> > > > Currently
> > > > > > we
> > > > > > > >> > have a
> > > > > > > >> > > > >> > >> mechanism
> > > > > > > >> > > > >> > >> > in
> > > > > > > >> > > > >> > >> > > > > place
> > > > > > > >> > > > >> > >> > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > fence the final epoch when the
> > epoch is
> > > > > > about to
> > > > > > > >> > > > overflow
> > > > > > > >> > > > >> > and
> > > > > > > >> > > > >> > >> > > assign
> > > > > > > >> > > > >> > >> > > > a
> > > > > > > >> > > > >> > >> > > > > > new
> > > > > > > >> > > > >> > >> > > > > > > producer ID with epoch 0. Of
> course,
> > > > this
> > > > > > is a
> > > > > > > >> bit
> > > > > > > >> > > > tricky
> > > > > > > >> > > > >> > >> when it
> > > > > > > >> > > > >> > >> > > > comes
> > > > > > > >> > > > >> > >> > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > the response back to the client.
> > > > > > > >> > > > >> > >> > > > > > > Making this a long could be
> another
> > > > option,
> > > > > > but
> > > > > > > >> I
> > > > > > > >> > > > wonder
> > > > > > > >> > > > >> are
> > > > > > > >> > > > >> > >> > there
> > > > > > > >> > > > >> > >> > > > any
> > > > > > > >> > > > >> > >> > > > > > > implications on changing this
> field
> > if
> > > > the
> > > > > > > >> epoch is
> > > > > > > >> > > > >> > persisted
> > > > > > > >> > > > >> > >> to
> > > > > > > >> > > > >> > >> > > > disk?
> > > > > > > >> > > > >> > >> > > > > > I'd
> > > > > > > >> > > > >> > >> > > > > > > need to check the usages.
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > 71.This was something Matthias
> asked
> > > > about
> > > > > > as
> > > > > > > >> > well. I
> > > > > > > >> > > > was
> > > > > > > >> > > > >> > >> > > > considering a
> > > > > > > >> > > > >> > >> > > > > > > possible edge case where a produce
> > > > request
> > > > > > from
> > > > > > > >> a
> > > > > > > >> > new
> > > > > > > >> > > > >> > >> transaction
> > > > > > > >> > > > >> > >> > > > > somehow
> > > > > > > >> > > > >> > >> > > > > > > gets sent right after the marker
> is
> > > > > > written, but
> > > > > > > >> > > before
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> > > producer
> > > > > > > >> > > > >> > >> > > > is
> > > > > > > >> > > > >> > >> > > > > > > alerted of the newly bumped epoch.
> > In
> > > > this
> > > > > > > >> case, we
> > > > > > > >> > > may
> > > > > > > >> > > > >> > >> include
> > > > > > > >> > > > >> > >> > > this
> > > > > > > >> > > > >> > >> > > > > > record
> > > > > > > >> > > > >> > >> > > > > > > when we don't want to. I suppose
> we
> > > > could
> > > > > > try
> > > > > > > >> to do
> > > > > > > >> > > > >> > something
> > > > > > > >> > > > >> > >> > > client
> > > > > > > >> > > > >> > >> > > > > side
> > > > > > > >> > > > >> > >> > > > > > > to bump the epoch after sending an
> > > > endTxn as
> > > > > > > >> well
> > > > > > > >> > in
> > > > > > > >> > > > this
> > > > > > > >> > > > >> > >> > scenario
> > > > > > > >> > > > >> > >> > > —
> > > > > > > >> > > > >> > >> > > > > but
> > > > > > > >> > > > >> > >> > > > > > I
> > > > > > > >> > > > >> > >> > > > > > > wonder how it would work when the
> > > > server is
> > > > > > > >> > aborting
> > > > > > > >> > > > >> based
> > > > > > > >> > > > >> > on
> > > > > > > >> > > > >> > >> a
> > > > > > > >> > > > >> > >> > > > > > server-side
> > > > > > > >> > > > >> > >> > > > > > > error. I could also be missing
> > > > something and
> > > > > > > >> this
> > > > > > > >> > > > >> scenario
> > > > > > > >> > > > >> > is
> > > > > > > >> > > > >> > >> > > > actually
> > > > > > > >> > > > >> > >> > > > > > not
> > > > > > > >> > > > >> > >> > > > > > > possible.
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > Thanks again to everyone reading
> and
> > > > > > commenting.
> > > > > > > >> > Let
> > > > > > > >> > > me
> > > > > > > >> > > > >> know
> > > > > > > >> > > > >> > >> > about
> > > > > > > >> > > > >> > >> > > > any
> > > > > > > >> > > > >> > >> > > > > > > further questions or comments.
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > Justine
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > On Wed, Dec 14, 2022 at 9:41 AM
> Jun
> > Rao
> > > > > > > >> > > > >> > >> <jun@confluent.io.invalid
> > > > > > > >> > > > >> > >> > >
> > > > > > > >> > > > >> > >> > > > > > wrote:
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > Hi, Justine,
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > Thanks for the KIP. A couple of
> > > > comments.
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > 70. Currently, the producer
> epoch
> > is
> > > > an
> > > > > > int.
> > > > > > > >> I am
> > > > > > > >> > > not
> > > > > > > >> > > > >> sure
> > > > > > > >> > > > >> > >> if
> > > > > > > >> > > > >> > >> > > it's
> > > > > > > >> > > > >> > >> > > > > > enough
> > > > > > > >> > > > >> > >> > > > > > > > to accommodate all transactions
> > in the
> > > > > > > >> lifetime
> > > > > > > >> > of
> > > > > > > >> > > a
> > > > > > > >> > > > >> > >> producer.
> > > > > > > >> > > > >> > >> > > > Should
> > > > > > > >> > > > >> > >> > > > > > we
> > > > > > > >> > > > >> > >> > > > > > > > change that to a long or add a
> new
> > > > long
> > > > > > field
> > > > > > > >> > like
> > > > > > > >> > > > >> txnId?
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > 71. "it will write the prepare
> > commit
> > > > > > message
> > > > > > > >> > with
> > > > > > > >> > > a
> > > > > > > >> > > > >> > bumped
> > > > > > > >> > > > >> > >> > epoch
> > > > > > > >> > > > >> > >> > > > and
> > > > > > > >> > > > >> > >> > > > > > > send
> > > > > > > >> > > > >> > >> > > > > > > > WriteTxnMarkerRequests with the
> > bumped
> > > > > > epoch."
> > > > > > > >> > Hmm,
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > >> epoch
> > > > > > > >> > > > >> > >> > is
> > > > > > > >> > > > >> > >> > > > > > > associated
> > > > > > > >> > > > >> > >> > > > > > > > with the current txn right? So,
> it
> > > > seems
> > > > > > > >> weird to
> > > > > > > >> > > > >> write a
> > > > > > > >> > > > >> > >> > commit
> > > > > > > >> > > > >> > >> > > > > > message
> > > > > > > >> > > > >> > >> > > > > > > > with a bumped epoch. Should we
> > only
> > > > bump
> > > > > > up
> > > > > > > >> the
> > > > > > > >> > > epoch
> > > > > > > >> > > > >> in
> > > > > > > >> > > > >> > >> > > > > EndTxnResponse
> > > > > > > >> > > > >> > >> > > > > > > and
> > > > > > > >> > > > >> > >> > > > > > > > rename the field to sth like
> > > > > > > >> nextProducerEpoch?
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > Thanks,
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > Jun
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > On Mon, Dec 12, 2022 at 8:54 PM
> > > > Matthias
> > > > > > J.
> > > > > > > >> Sax <
> > > > > > > >> > > > >> > >> > > mjsax@apache.org>
> > > > > > > >> > > > >> > >> > > > > > > wrote:
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > Thanks for the background.
> > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > 20/30: SGTM. My proposal was
> > only
> > > > > > focusing
> > > > > > > >> to
> > > > > > > >> > > avoid
> > > > > > > >> > > > >> > >> dangling
> > > > > > > >> > > > >> > >> > > > > > > > > transactions if records are
> > added
> > > > > > without
> > > > > > > >> > > > registered
> > > > > > > >> > > > >> > >> > partition.
> > > > > > > >> > > > >> > >> > > > --
> > > > > > > >> > > > >> > >> > > > > > > Maybe
> > > > > > > >> > > > >> > >> > > > > > > > > you can add a few more details
> > to
> > > > the
> > > > > > KIP
> > > > > > > >> about
> > > > > > > >> > > > this
> > > > > > > >> > > > >> > >> scenario
> > > > > > > >> > > > >> > >> > > for
> > > > > > > >> > > > >> > >> > > > > > > better
> > > > > > > >> > > > >> > >> > > > > > > > > documentation purpose?
> > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > 40: I think you hit a fair
> point
> > > > about
> > > > > > race
> > > > > > > >> > > > >> conditions
> > > > > > > >> > > > >> > or
> > > > > > > >> > > > >> > >> > > client
> > > > > > > >> > > > >> > >> > > > > bugs
> > > > > > > >> > > > >> > >> > > > > > > > > (incorrectly not bumping the
> > > > epoch). The
> > > > > > > >> > > > >> > >> complexity/confusion
> > > > > > > >> > > > >> > >> > > for
> > > > > > > >> > > > >> > >> > > > > > using
> > > > > > > >> > > > >> > >> > > > > > > > > the bumped epoch I see, is
> > mainly
> > > > for
> > > > > > > >> internal
> > > > > > > >> > > > >> > debugging,
> > > > > > > >> > > > >> > >> ie,
> > > > > > > >> > > > >> > >> > > > > > > inspecting
> > > > > > > >> > > > >> > >> > > > > > > > > log segment dumps -- it seems
> > > > harder to
> > > > > > > >> reason
> > > > > > > >> > > > about
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> > system
> > > > > > > >> > > > >> > >> > > > for
> > > > > > > >> > > > >> > >> > > > > > us
> > > > > > > >> > > > >> > >> > > > > > > > > humans. But if we get better
> > > > > > guarantees, it
> > > > > > > >> > would
> > > > > > > >> > > > be
> > > > > > > >> > > > >> > >> worth to
> > > > > > > >> > > > >> > >> > > use
> > > > > > > >> > > > >> > >> > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > bumped epoch.
> > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > 60: as I mentioned already, I
> > don't
> > > > > > know the
> > > > > > > >> > > broker
> > > > > > > >> > > > >> > >> internals
> > > > > > > >> > > > >> > >> > > to
> > > > > > > >> > > > >> > >> > > > > > > provide
> > > > > > > >> > > > >> > >> > > > > > > > > more input. So if nobody else
> > chimes
> > > > > > in, we
> > > > > > > >> > > should
> > > > > > > >> > > > >> just
> > > > > > > >> > > > >> > >> move
> > > > > > > >> > > > >> > >> > > > > forward
> > > > > > > >> > > > >> > >> > > > > > > > > with your proposal.
> > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > -Matthias
> > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > On 12/6/22 4:22 PM, Justine
> > Olshan
> > > > > > wrote:
> > > > > > > >> > > > >> > >> > > > > > > > > > Hi all,
> > > > > > > >> > > > >> > >> > > > > > > > > > After Artem's questions
> about
> > > > error
> > > > > > > >> behavior,
> > > > > > > >> > > > I've
> > > > > > > >> > > > >> > >> > > re-evaluated
> > > > > > > >> > > > >> > >> > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > > unknown producer ID
> exception
> > and
> > > > had
> > > > > > some
> > > > > > > >> > > > >> discussions
> > > > > > > >> > > > >> > >> > > offline.
> > > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > > I think generally it makes
> > sense
> > > > to
> > > > > > > >> simplify
> > > > > > > >> > > > error
> > > > > > > >> > > > >> > >> handling
> > > > > > > >> > > > >> > >> > > in
> > > > > > > >> > > > >> > >> > > > > > cases
> > > > > > > >> > > > >> > >> > > > > > > > like
> > > > > > > >> > > > >> > >> > > > > > > > > > this and the
> > UNKNOWN_PRODUCER_ID
> > > > error
> > > > > > > >> has a
> > > > > > > >> > > > pretty
> > > > > > > >> > > > >> > long
> > > > > > > >> > > > >> > >> > and
> > > > > > > >> > > > >> > >> > > > > > > > complicated
> > > > > > > >> > > > >> > >> > > > > > > > > > history. Because of this, I
> > > > propose
> > > > > > > >> adding a
> > > > > > > >> > > new
> > > > > > > >> > > > >> error
> > > > > > > >> > > > >> > >> code
> > > > > > > >> > > > >> > >> > > > > > > > > ABORTABLE_ERROR
> > > > > > > >> > > > >> > >> > > > > > > > > > that when encountered by new
> > > > clients
> > > > > > > >> (gated
> > > > > > > >> > by
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > >> produce
> > > > > > > >> > > > >> > >> > > > > request
> > > > > > > >> > > > >> > >> > > > > > > > > version)
> > > > > > > >> > > > >> > >> > > > > > > > > > will simply abort the
> > transaction.
> > > > > > This
> > > > > > > >> > allows
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > >> server
> > > > > > > >> > > > >> > >> > to
> > > > > > > >> > > > >> > >> > > > have
> > > > > > > >> > > > >> > >> > > > > > > some
> > > > > > > >> > > > >> > >> > > > > > > > > say
> > > > > > > >> > > > >> > >> > > > > > > > > > in whether the client aborts
> > and
> > > > makes
> > > > > > > >> > handling
> > > > > > > >> > > > >> much
> > > > > > > >> > > > >> > >> > simpler.
> > > > > > > >> > > > >> > >> > > > In
> > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > > future, we can also use this
> > > > error in
> > > > > > > >> other
> > > > > > > >> > > > >> situations
> > > > > > > >> > > > >> > >> > where
> > > > > > > >> > > > >> > >> > > we
> > > > > > > >> > > > >> > >> > > > > > want
> > > > > > > >> > > > >> > >> > > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > > abort the transactions. We
> can
> > > > even
> > > > > > use on
> > > > > > > >> > > other
> > > > > > > >> > > > >> apis.
> > > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > > I've added this to the KIP.
> > Let me
> > > > > > know if
> > > > > > > >> > > there
> > > > > > > >> > > > >> are
> > > > > > > >> > > > >> > any
> > > > > > > >> > > > >> > >> > > > > questions
> > > > > > > >> > > > >> > >> > > > > > or
> > > > > > > >> > > > >> > >> > > > > > > > > > issues.
> > > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > > Justine
> > > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > > On Fri, Dec 2, 2022 at 10:22
> > AM
> > > > > > Justine
> > > > > > > >> > Olshan
> > > > > > > >> > > <
> > > > > > > >> > > > >> > >> > > > > > jolshan@confluent.io
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > wrote:
> > > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > >> Hey Matthias,
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >> 20/30 — Maybe I also didn't
> > > > express
> > > > > > > >> myself
> > > > > > > >> > > > >> clearly.
> > > > > > > >> > > > >> > For
> > > > > > > >> > > > >> > >> > > older
> > > > > > > >> > > > >> > >> > > > > > > clients
> > > > > > > >> > > > >> > >> > > > > > > > we
> > > > > > > >> > > > >> > >> > > > > > > > > >> don't have a way to
> > distinguish
> > > > > > between a
> > > > > > > >> > > > previous
> > > > > > > >> > > > >> > and
> > > > > > > >> > > > >> > >> the
> > > > > > > >> > > > >> > >> > > > > current
> > > > > > > >> > > > >> > >> > > > > > > > > >> transaction since we don't
> > have
> > > > the
> > > > > > epoch
> > > > > > > >> > > bump.
> > > > > > > >> > > > >> This
> > > > > > > >> > > > >> > >> means
> > > > > > > >> > > > >> > >> > > > that
> > > > > > > >> > > > >> > >> > > > > a
> > > > > > > >> > > > >> > >> > > > > > > late
> > > > > > > >> > > > >> > >> > > > > > > > > >> message from the previous
> > > > transaction
> > > > > > > >> may be
> > > > > > > >> > > > >> added to
> > > > > > > >> > > > >> > >> the
> > > > > > > >> > > > >> > >> > > new
> > > > > > > >> > > > >> > >> > > > > one.
> > > > > > > >> > > > >> > >> > > > > > > > With
> > > > > > > >> > > > >> > >> > > > > > > > > >> older clients — we can't
> > > > guarantee
> > > > > > this
> > > > > > > >> > won't
> > > > > > > >> > > > >> happen
> > > > > > > >> > > > >> > >> if we
> > > > > > > >> > > > >> > >> > > > > already
> > > > > > > >> > > > >> > >> > > > > > > > sent
> > > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call
> (why
> > we
> > > > make
> > > > > > > >> changes
> > > > > > > >> > > for
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> newer
> > > > > > > >> > > > >> > >> > > > > client)
> > > > > > > >> > > > >> > >> > > > > > > but
> > > > > > > >> > > > >> > >> > > > > > > > > we
> > > > > > > >> > > > >> > >> > > > > > > > > >> can at least gate some by
> > > > ensuring
> > > > > > that
> > > > > > > >> the
> > > > > > > >> > > > >> partition
> > > > > > > >> > > > >> > >> has
> > > > > > > >> > > > >> > >> > > been
> > > > > > > >> > > > >> > >> > > > > > added
> > > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >> transaction. The rationale
> > here
> > > > is
> > > > > > that
> > > > > > > >> > there
> > > > > > > >> > > > are
> > > > > > > >> > > > >> > >> likely
> > > > > > > >> > > > >> > >> > > LESS
> > > > > > > >> > > > >> > >> > > > > late
> > > > > > > >> > > > >> > >> > > > > > > > > arrivals
> > > > > > > >> > > > >> > >> > > > > > > > > >> as time goes on, so
> hopefully
> > > > most
> > > > > > late
> > > > > > > >> > > arrivals
> > > > > > > >> > > > >> will
> > > > > > > >> > > > >> > >> come
> > > > > > > >> > > > >> > >> > > in
> > > > > > > >> > > > >> > >> > > > > > BEFORE
> > > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call.
> > Those
> > > > that
> > > > > > > >> arrive
> > > > > > > >> > > > before
> > > > > > > >> > > > >> > will
> > > > > > > >> > > > >> > >> be
> > > > > > > >> > > > >> > >> > > > > properly
> > > > > > > >> > > > >> > >> > > > > > > > gated
> > > > > > > >> > > > >> > >> > > > > > > > > >> with the
> describeTransactions
> > > > > > approach.
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >> If we take the approach you
> > > > > > suggested,
> > > > > > > >> ANY
> > > > > > > >> > > late
> > > > > > > >> > > > >> > arrival
> > > > > > > >> > > > >> > >> > > from a
> > > > > > > >> > > > >> > >> > > > > > > > previous
> > > > > > > >> > > > >> > >> > > > > > > > > >> transaction will be added.
> > And we
> > > > > > don't
> > > > > > > >> want
> > > > > > > >> > > > >> that. I
> > > > > > > >> > > > >> > >> also
> > > > > > > >> > > > >> > >> > > > don't
> > > > > > > >> > > > >> > >> > > > > > see
> > > > > > > >> > > > >> > >> > > > > > > > any
> > > > > > > >> > > > >> > >> > > > > > > > > >> benefit in sending
> > > > addPartitionsToTxn
> > > > > > > >> over
> > > > > > > >> > the
> > > > > > > >> > > > >> > >> > describeTxns
> > > > > > > >> > > > >> > >> > > > > call.
> > > > > > > >> > > > >> > >> > > > > > > They
> > > > > > > >> > > > >> > >> > > > > > > > > will
> > > > > > > >> > > > >> > >> > > > > > > > > >> both be one extra RPC to
> the
> > Txn
> > > > > > > >> > coordinator.
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >> To be clear — newer clients
> > will
> > > > use
> > > > > > > >> > > > >> > addPartitionsToTxn
> > > > > > > >> > > > >> > >> > > > instead
> > > > > > > >> > > > >> > >> > > > > of
> > > > > > > >> > > > >> > >> > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >> DescribeTxns.
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >> 40)
> > > > > > > >> > > > >> > >> > > > > > > > > >> My concern is that if we
> have
> > > > some
> > > > > > delay
> > > > > > > >> in
> > > > > > > >> > > the
> > > > > > > >> > > > >> > client
> > > > > > > >> > > > >> > >> to
> > > > > > > >> > > > >> > >> > > bump
> > > > > > > >> > > > >> > >> > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > epoch,
> > > > > > > >> > > > >> > >> > > > > > > > > >> it could continue to send
> > epoch
> > > > 73
> > > > > > and
> > > > > > > >> those
> > > > > > > >> > > > >> records
> > > > > > > >> > > > >> > >> would
> > > > > > > >> > > > >> > >> > > not
> > > > > > > >> > > > >> > >> > > > > be
> > > > > > > >> > > > >> > >> > > > > > > > > fenced.
> > > > > > > >> > > > >> > >> > > > > > > > > >> Perhaps this is not an
> issue
> > if
> > > > we
> > > > > > don't
> > > > > > > >> > allow
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > next
> > > > > > > >> > > > >> > >> > > > produce
> > > > > > > >> > > > >> > >> > > > > to
> > > > > > > >> > > > >> > >> > > > > > > go
> > > > > > > >> > > > >> > >> > > > > > > > > >> through before the EndTxn
> > request
> > > > > > > >> returns.
> > > > > > > >> > I'm
> > > > > > > >> > > > >> also
> > > > > > > >> > > > >> > >> > thinking
> > > > > > > >> > > > >> > >> > > > > about
> > > > > > > >> > > > >> > >> > > > > > > > > cases of
> > > > > > > >> > > > >> > >> > > > > > > > > >> failure. I will need to
> > think on
> > > > > > this a
> > > > > > > >> bit.
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >> I wasn't sure if it was
> that
> > > > > > confusing.
> > > > > > > >> But
> > > > > > > >> > if
> > > > > > > >> > > > we
> > > > > > > >> > > > >> > >> think it
> > > > > > > >> > > > >> > >> > > is,
> > > > > > > >> > > > >> > >> > > > > we
> > > > > > > >> > > > >> > >> > > > > > > can
> > > > > > > >> > > > >> > >> > > > > > > > > >> investigate other ways.
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >> 60)
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >> I'm not sure these are the
> > same
> > > > > > > >> purgatories
> > > > > > > >> > > > since
> > > > > > > >> > > > >> one
> > > > > > > >> > > > >> > >> is a
> > > > > > > >> > > > >> > >> > > > > produce
> > > > > > > >> > > > >> > >> > > > > > > > > >> purgatory (I was planning
> on
> > > > using a
> > > > > > > >> > callback
> > > > > > > >> > > > >> rather
> > > > > > > >> > > > >> > >> than
> > > > > > > >> > > > >> > >> > > > > > purgatory)
> > > > > > > >> > > > >> > >> > > > > > > > and
> > > > > > > >> > > > >> > >> > > > > > > > > >> the other is simply a
> > request to
> > > > > > append
> > > > > > > >> to
> > > > > > > >> > the
> > > > > > > >> > > > >> log.
> > > > > > > >> > > > >> > Not
> > > > > > > >> > > > >> > >> > sure
> > > > > > > >> > > > >> > >> > > > we
> > > > > > > >> > > > >> > >> > > > > > have
> > > > > > > >> > > > >> > >> > > > > > > > any
> > > > > > > >> > > > >> > >> > > > > > > > > >> structure here for
> ordering,
> > but
> > > > my
> > > > > > > >> > > > understanding
> > > > > > > >> > > > >> is
> > > > > > > >> > > > >> > >> that
> > > > > > > >> > > > >> > >> > > the
> > > > > > > >> > > > >> > >> > > > > > broker
> > > > > > > >> > > > >> > >> > > > > > > > > could
> > > > > > > >> > > > >> > >> > > > > > > > > >> handle the write request
> > before
> > > > it
> > > > > > hears
> > > > > > > >> > back
> > > > > > > >> > > > from
> > > > > > > >> > > > >> > the
> > > > > > > >> > > > >> > >> Txn
> > > > > > > >> > > > >> > >> > > > > > > > Coordinator.
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >> Let me know if I
> > misunderstood
> > > > > > something
> > > > > > > >> or
> > > > > > > >> > > > >> something
> > > > > > > >> > > > >> > >> was
> > > > > > > >> > > > >> > >> > > > > unclear.
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >> Justine
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >> On Thu, Dec 1, 2022 at
> 12:15
> > PM
> > > > > > Matthias
> > > > > > > >> J.
> > > > > > > >> > > Sax
> > > > > > > >> > > > <
> > > > > > > >> > > > >> > >> > > > > mjsax@apache.org
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > wrote:
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >>> Thanks for the details
> > Justine!
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> 20)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> The client side change
> for
> > 2 is
> > > > > > > >> removing
> > > > > > > >> > the
> > > > > > > >> > > > >> > >> > addPartitions
> > > > > > > >> > > > >> > >> > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > >>> transaction
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> call. We don't need to
> make
> > > > this
> > > > > > from
> > > > > > > >> the
> > > > > > > >> > > > >> producer
> > > > > > > >> > > > >> > to
> > > > > > > >> > > > >> > >> > the
> > > > > > > >> > > > >> > >> > > > txn
> > > > > > > >> > > > >> > >> > > > > > > > > >>> coordinator,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> only server side.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>> I think I did not express
> > myself
> > > > > > > >> clearly. I
> > > > > > > >> > > > >> > understand
> > > > > > > >> > > > >> > >> > that
> > > > > > > >> > > > >> > >> > > > we
> > > > > > > >> > > > >> > >> > > > > > can
> > > > > > > >> > > > >> > >> > > > > > > > (and
> > > > > > > >> > > > >> > >> > > > > > > > > >>> should) change the
> producer
> > to
> > > > not
> > > > > > send
> > > > > > > >> the
> > > > > > > >> > > > >> > >> > `addPartitions`
> > > > > > > >> > > > >> > >> > > > > > request
> > > > > > > >> > > > >> > >> > > > > > > > any
> > > > > > > >> > > > >> > >> > > > > > > > > >>> longer. But I don't thinks
> > it's
> > > > > > > >> requirement
> > > > > > > >> > > to
> > > > > > > >> > > > >> > change
> > > > > > > >> > > > >> > >> the
> > > > > > > >> > > > >> > >> > > > > broker?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>> What I am trying to say
> is:
> > as a
> > > > > > > >> safe-guard
> > > > > > > >> > > and
> > > > > > > >> > > > >> > >> > improvement
> > > > > > > >> > > > >> > >> > > > for
> > > > > > > >> > > > >> > >> > > > > > > older
> > > > > > > >> > > > >> > >> > > > > > > > > >>> producers, the partition
> > leader
> > > > can
> > > > > > just
> > > > > > > >> > send
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > >> > > > > `addPartitions`
> > > > > > > >> > > > >> > >> > > > > > > > > >>> request to the
> > TX-coordinator
> > > > in any
> > > > > > > >> case
> > > > > > > >> > --
> > > > > > > >> > > if
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> old
> > > > > > > >> > > > >> > >> > > > > producer
> > > > > > > >> > > > >> > >> > > > > > > > > >>> correctly did send the
> > > > > > `addPartition`
> > > > > > > >> > request
> > > > > > > >> > > > to
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> > > > > > TX-coordinator
> > > > > > > >> > > > >> > >> > > > > > > > > >>> already, the
> TX-coordinator
> > can
> > > > just
> > > > > > > >> > "ignore"
> > > > > > > >> > > > is
> > > > > > > >> > > > >> as
> > > > > > > >> > > > >> > >> > > > idempotent.
> > > > > > > >> > > > >> > >> > > > > > > > > However,
> > > > > > > >> > > > >> > >> > > > > > > > > >>> if the old producer has a
> > bug
> > > > and
> > > > > > did
> > > > > > > >> > forget
> > > > > > > >> > > to
> > > > > > > >> > > > >> sent
> > > > > > > >> > > > >> > >> the
> > > > > > > >> > > > >> > >> > > > > > > > `addPartition`
> > > > > > > >> > > > >> > >> > > > > > > > > >>> request, we would now
> ensure
> > > > that
> > > > > > the
> > > > > > > >> > > partition
> > > > > > > >> > > > >> is
> > > > > > > >> > > > >> > >> indeed
> > > > > > > >> > > > >> > >> > > > added
> > > > > > > >> > > > >> > >> > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> TX and thus fix a
> potential
> > > > > > producer bug
> > > > > > > >> > > (even
> > > > > > > >> > > > >> if we
> > > > > > > >> > > > >> > >> > don't
> > > > > > > >> > > > >> > >> > > > get
> > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> fencing via the bump
> > epoch). --
> > > > It
> > > > > > > >> seems to
> > > > > > > >> > > be
> > > > > > > >> > > > a
> > > > > > > >> > > > >> > good
> > > > > > > >> > > > >> > >> > > > > > improvement?
> > > > > > > >> > > > >> > >> > > > > > > Or
> > > > > > > >> > > > >> > >> > > > > > > > > is
> > > > > > > >> > > > >> > >> > > > > > > > > >>> there a reason to not do
> > this?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> 30)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> Transaction is ongoing =
> > > > partition
> > > > > > was
> > > > > > > >> > added
> > > > > > > >> > > > to
> > > > > > > >> > > > >> > >> > > transaction
> > > > > > > >> > > > >> > >> > > > > via
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> addPartitionsToTxn. We
> > check
> > > > this
> > > > > > with
> > > > > > > >> the
> > > > > > > >> > > > >> > >> > > > > DescribeTransactions
> > > > > > > >> > > > >> > >> > > > > > > > call.
> > > > > > > >> > > > >> > >> > > > > > > > > >>> Let
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> me know if this wasn't
> > > > sufficiently
> > > > > > > >> > > explained
> > > > > > > >> > > > >> here:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>> If we do what I propose in
> > > > (20), we
> > > > > > > >> don't
> > > > > > > >> > > > really
> > > > > > > >> > > > >> > need
> > > > > > > >> > > > >> > >> to
> > > > > > > >> > > > >> > >> > > make
> > > > > > > >> > > > >> > >> > > > > > this
> > > > > > > >> > > > >> > >> > > > > > > > > >>> `DescribeTransaction`
> call,
> > as
> > > > the
> > > > > > > >> > partition
> > > > > > > >> > > > >> leader
> > > > > > > >> > > > >> > >> adds
> > > > > > > >> > > > >> > >> > > the
> > > > > > > >> > > > >> > >> > > > > > > > partition
> > > > > > > >> > > > >> > >> > > > > > > > > >>> for older clients and we
> get
> > > > this
> > > > > > check
> > > > > > > >> for
> > > > > > > >> > > > free.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> 40)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> The idea here is that if
> > any
> > > > > > messages
> > > > > > > >> > > somehow
> > > > > > > >> > > > >> come
> > > > > > > >> > > > >> > in
> > > > > > > >> > > > >> > >> > > before
> > > > > > > >> > > > >> > >> > > > > we
> > > > > > > >> > > > >> > >> > > > > > > get
> > > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> new
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> epoch to the producer,
> they
> > > > will be
> > > > > > > >> > fenced.
> > > > > > > >> > > > >> > However,
> > > > > > > >> > > > >> > >> if
> > > > > > > >> > > > >> > >> > we
> > > > > > > >> > > > >> > >> > > > > don't
> > > > > > > >> > > > >> > >> > > > > > > > think
> > > > > > > >> > > > >> > >> > > > > > > > > >>> this
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> is necessary, it can be
> > > > discussed
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>> I agree that we should
> have
> > > > epoch
> > > > > > > >> fencing.
> > > > > > > >> > My
> > > > > > > >> > > > >> > >> question is
> > > > > > > >> > > > >> > >> > > > > > > different:
> > > > > > > >> > > > >> > >> > > > > > > > > >>> Assume we are at epoch 73,
> > and
> > > > we
> > > > > > have
> > > > > > > >> an
> > > > > > > >> > > > ongoing
> > > > > > > >> > > > >> > >> > > > transaction,
> > > > > > > >> > > > >> > >> > > > > > that
> > > > > > > >> > > > >> > >> > > > > > > > is
> > > > > > > >> > > > >> > >> > > > > > > > > >>> committed. It seems
> natural
> > to
> > > > > > write the
> > > > > > > >> > > > "prepare
> > > > > > > >> > > > >> > >> commit"
> > > > > > > >> > > > >> > >> > > > > marker
> > > > > > > >> > > > >> > >> > > > > > > and
> > > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest`
> both
> > with
> > > > > > epoch
> > > > > > > >> 73,
> > > > > > > >> > > too,
> > > > > > > >> > > > >> as
> > > > > > > >> > > > >> > it
> > > > > > > >> > > > >> > >> > > belongs
> > > > > > > >> > > > >> > >> > > > > to
> > > > > > > >> > > > >> > >> > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> current transaction. Of
> > course,
> > > > we
> > > > > > now
> > > > > > > >> also
> > > > > > > >> > > > bump
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> > epoch
> > > > > > > >> > > > >> > >> > > > and
> > > > > > > >> > > > >> > >> > > > > > > expect
> > > > > > > >> > > > >> > >> > > > > > > > > >>> the next requests to have
> > epoch
> > > > 74,
> > > > > > and
> > > > > > > >> > would
> > > > > > > >> > > > >> reject
> > > > > > > >> > > > >> > >> an
> > > > > > > >> > > > >> > >> > > > request
> > > > > > > >> > > > >> > >> > > > > > > with
> > > > > > > >> > > > >> > >> > > > > > > > > >>> epoch 73, as the
> > corresponding
> > > > TX
> > > > > > for
> > > > > > > >> epoch
> > > > > > > >> > > 73
> > > > > > > >> > > > >> was
> > > > > > > >> > > > >> > >> > already
> > > > > > > >> > > > >> > >> > > > > > > committed.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>> It seems you propose to
> > write
> > > > the
> > > > > > > >> "prepare
> > > > > > > >> > > > commit
> > > > > > > >> > > > >> > >> marker"
> > > > > > > >> > > > >> > >> > > and
> > > > > > > >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest`
> with
> > > > epoch 74
> > > > > > > >> > though,
> > > > > > > >> > > > what
> > > > > > > >> > > > >> > >> would
> > > > > > > >> > > > >> > >> > > work,
> > > > > > > >> > > > >> > >> > > > > but
> > > > > > > >> > > > >> > >> > > > > > > it
> > > > > > > >> > > > >> > >> > > > > > > > > >>> seems confusing. Is there
> a
> > > > reason
> > > > > > why
> > > > > > > >> we
> > > > > > > >> > > would
> > > > > > > >> > > > >> use
> > > > > > > >> > > > >> > >> the
> > > > > > > >> > > > >> > >> > > > bumped
> > > > > > > >> > > > >> > >> > > > > > > epoch
> > > > > > > >> > > > >> > >> > > > > > > > 74
> > > > > > > >> > > > >> > >> > > > > > > > > >>> instead of the current
> > epoch 73?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> 60)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> When we are checking if
> the
> > > > > > > >> transaction is
> > > > > > > >> > > > >> ongoing,
> > > > > > > >> > > > >> > >> we
> > > > > > > >> > > > >> > >> > > need
> > > > > > > >> > > > >> > >> > > > to
> > > > > > > >> > > > >> > >> > > > > > > make
> > > > > > > >> > > > >> > >> > > > > > > > a
> > > > > > > >> > > > >> > >> > > > > > > > > >>> round
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> trip from the leader
> > partition
> > > > to
> > > > > > the
> > > > > > > >> > > > >> transaction
> > > > > > > >> > > > >> > >> > > > coordinator.
> > > > > > > >> > > > >> > >> > > > > > In
> > > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> time
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> we are waiting for this
> > > > message to
> > > > > > come
> > > > > > > >> > > back,
> > > > > > > >> > > > in
> > > > > > > >> > > > >> > >> theory
> > > > > > > >> > > > >> > >> > we
> > > > > > > >> > > > >> > >> > > > > could
> > > > > > > >> > > > >> > >> > > > > > > > have
> > > > > > > >> > > > >> > >> > > > > > > > > >>> sent
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> a commit/abort call that
> > would
> > > > > > make the
> > > > > > > >> > > > original
> > > > > > > >> > > > >> > >> result
> > > > > > > >> > > > >> > >> > of
> > > > > > > >> > > > >> > >> > > > the
> > > > > > > >> > > > >> > >> > > > > > > check
> > > > > > > >> > > > >> > >> > > > > > > > > >>> out of
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> date. That is why we can
> > check
> > > > the
> > > > > > > >> leader
> > > > > > > >> > > > state
> > > > > > > >> > > > >> > >> before
> > > > > > > >> > > > >> > >> > we
> > > > > > > >> > > > >> > >> > > > > write
> > > > > > > >> > > > >> > >> > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> log.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>> Thanks. Got it.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>> However, is this really an
> > > > issue?
> > > > > > We put
> > > > > > > >> > the
> > > > > > > >> > > > >> produce
> > > > > > > >> > > > >> > >> > > request
> > > > > > > >> > > > >> > >> > > > in
> > > > > > > >> > > > >> > >> > > > > > > > > >>> purgatory, so how could we
> > > > process
> > > > > > the
> > > > > > > >> > > > >> > >> > > > `WriteTxnMarkerRequest`
> > > > > > > >> > > > >> > >> > > > > > > first?
> > > > > > > >> > > > >> > >> > > > > > > > > >>> Don't we need to put the
> > > > > > > >> > > > `WriteTxnMarkerRequest`
> > > > > > > >> > > > >> > into
> > > > > > > >> > > > >> > >> > > > > purgatory,
> > > > > > > >> > > > >> > >> > > > > > > too,
> > > > > > > >> > > > >> > >> > > > > > > > > >>> for this case, and process
> > both
> > > > > > request
> > > > > > > >> > > > in-order?
> > > > > > > >> > > > >> > >> (Again,
> > > > > > > >> > > > >> > >> > > my
> > > > > > > >> > > > >> > >> > > > > > broker
> > > > > > > >> > > > >> > >> > > > > > > > > >>> knowledge is limited and
> > maybe
> > > > we
> > > > > > don't
> > > > > > > >> > > > maintain
> > > > > > > >> > > > >> > >> request
> > > > > > > >> > > > >> > >> > > > order
> > > > > > > >> > > > >> > >> > > > > > for
> > > > > > > >> > > > >> > >> > > > > > > > this
> > > > > > > >> > > > >> > >> > > > > > > > > >>> case, what seems to be an
> > issue
> > > > > > IMHO,
> > > > > > > >> and I
> > > > > > > >> > > am
> > > > > > > >> > > > >> > >> wondering
> > > > > > > >> > > > >> > >> > if
> > > > > > > >> > > > >> > >> > > > > > > changing
> > > > > > > >> > > > >> > >> > > > > > > > > >>> request handling to
> preserve
> > > > order
> > > > > > for
> > > > > > > >> this
> > > > > > > >> > > > case
> > > > > > > >> > > > >> > >> might be
> > > > > > > >> > > > >> > >> > > the
> > > > > > > >> > > > >> > >> > > > > > > cleaner
> > > > > > > >> > > > >> > >> > > > > > > > > >>> solution?)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>> -Matthias
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>> On 11/30/22 3:28 PM, Artem
> > > > Livshits
> > > > > > > >> wrote:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> Hi Justine,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> I think the interesting
> > part is
> > > > > > not in
> > > > > > > >> > this
> > > > > > > >> > > > >> logic
> > > > > > > >> > > > >> > >> > (because
> > > > > > > >> > > > >> > >> > > > it
> > > > > > > >> > > > >> > >> > > > > > > tries
> > > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> figure out when
> > > > > > UNKNOWN_PRODUCER_ID is
> > > > > > > >> > > > retriable
> > > > > > > >> > > > >> > and
> > > > > > > >> > > > >> > >> if
> > > > > > > >> > > > >> > >> > > it's
> > > > > > > >> > > > >> > >> > > > > > > > > retryable,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> it's definitely not
> > fatal), but
> > > > > > what
> > > > > > > >> > happens
> > > > > > > >> > > > >> when
> > > > > > > >> > > > >> > >> this
> > > > > > > >> > > > >> > >> > > logic
> > > > > > > >> > > > >> > >> > > > > > > doesn't
> > > > > > > >> > > > >> > >> > > > > > > > > >>> return
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> 'true' and falls through.
> > In
> > > > the
> > > > > > old
> > > > > > > >> > > clients
> > > > > > > >> > > > it
> > > > > > > >> > > > >> > >> seems
> > > > > > > >> > > > >> > >> > to
> > > > > > > >> > > > >> > >> > > be
> > > > > > > >> > > > >> > >> > > > > > > fatal,
> > > > > > > >> > > > >> > >> > > > > > > > if
> > > > > > > >> > > > >> > >> > > > > > > > > >>> we
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> keep the behavior in the
> > new
> > > > > > clients,
> > > > > > > >> I'd
> > > > > > > >> > > > >> expect it
> > > > > > > >> > > > >> > >> > would
> > > > > > > >> > > > >> > >> > > be
> > > > > > > >> > > > >> > >> > > > > > fatal
> > > > > > > >> > > > >> > >> > > > > > > > as
> > > > > > > >> > > > >> > >> > > > > > > > > >>> well.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> -Artem
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>> On Tue, Nov 29, 2022 at
> > 11:57
> > > > AM
> > > > > > > >> Justine
> > > > > > > >> > > > Olshan
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > <jo...@confluent.io.invalid>
> > > > > > wrote:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> Hi Artem and Jeff,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> Thanks for taking a look
> > and
> > > > > > sorry for
> > > > > > > >> > the
> > > > > > > >> > > > slow
> > > > > > > >> > > > >> > >> > response.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> You both mentioned the
> > change
> > > > to
> > > > > > > >> handle
> > > > > > > >> > > > >> > >> > > UNKNOWN_PRODUCER_ID
> > > > > > > >> > > > >> > >> > > > > > > errors.
> > > > > > > >> > > > >> > >> > > > > > > > > To
> > > > > > > >> > > > >> > >> > > > > > > > > >>> be
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> clear — this error code
> > will
> > > > only
> > > > > > be
> > > > > > > >> sent
> > > > > > > >> > > > again
> > > > > > > >> > > > >> > when
> > > > > > > >> > > > >> > >> > the
> > > > > > > >> > > > >> > >> > > > > > client's
> > > > > > > >> > > > >> > >> > > > > > > > > >>> request
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> version is high enough
> to
> > > > ensure
> > > > > > we
> > > > > > > >> > handle
> > > > > > > >> > > it
> > > > > > > >> > > > >> > >> > correctly.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> The current (Java)
> client
> > > > handles
> > > > > > > >> this by
> > > > > > > >> > > the
> > > > > > > >> > > > >> > >> following
> > > > > > > >> > > > >> > >> > > > > > (somewhat
> > > > > > > >> > > > >> > >> > > > > > > > > long)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> code snippet:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> // An
> UNKNOWN_PRODUCER_ID
> > > > means
> > > > > > that
> > > > > > > >> we
> > > > > > > >> > > have
> > > > > > > >> > > > >> lost
> > > > > > > >> > > > >> > >> the
> > > > > > > >> > > > >> > >> > > > > producer
> > > > > > > >> > > > >> > >> > > > > > > > state
> > > > > > > >> > > > >> > >> > > > > > > > > >>> on the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> broker. Depending on the
> > log
> > > > start
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> // offset, we may want
> to
> > > > retry
> > > > > > > >> these, as
> > > > > > > >> > > > >> > described
> > > > > > > >> > > > >> > >> for
> > > > > > > >> > > > >> > >> > > > each
> > > > > > > >> > > > >> > >> > > > > > case
> > > > > > > >> > > > >> > >> > > > > > > > > >>> below. If
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> none of those apply,
> then
> > for
> > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> // idempotent producer,
> we
> > > > will
> > > > > > > >> locally
> > > > > > > >> > > bump
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> epoch
> > > > > > > >> > > > >> > >> > > and
> > > > > > > >> > > > >> > >> > > > > > reset
> > > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> sequence numbers of
> > in-flight
> > > > > > batches
> > > > > > > >> > from
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> // sequence 0, then
> retry
> > the
> > > > > > failed
> > > > > > > >> > batch,
> > > > > > > >> > > > >> which
> > > > > > > >> > > > >> > >> > should
> > > > > > > >> > > > >> > >> > > > now
> > > > > > > >> > > > >> > >> > > > > > > > succeed.
> > > > > > > >> > > > >> > >> > > > > > > > > >>> For
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> the transactional
> > producer,
> > > > allow
> > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> // batch to fail. When
> > > > processing
> > > > > > the
> > > > > > > >> > > failed
> > > > > > > >> > > > >> > batch,
> > > > > > > >> > > > >> > >> we
> > > > > > > >> > > > >> > >> > > will
> > > > > > > >> > > > >> > >> > > > > > > > > transition
> > > > > > > >> > > > >> > >> > > > > > > > > >>> to
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> an abortable error and
> > set a
> > > > flag
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> // indicating that we
> > need to
> > > > > > bump the
> > > > > > > >> > > epoch
> > > > > > > >> > > > >> (if
> > > > > > > >> > > > >> > >> > > supported
> > > > > > > >> > > > >> > >> > > > by
> > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> broker).
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> if (error ==
> > > > > > > >> > Errors.*UNKNOWN_PRODUCER_ID*)
> > > > > > > >> > > {
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>       if
> > > > (response.logStartOffset
> > > > > > ==
> > > > > > > >> -1)
> > > > > > > >> > {
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // We don't
> know
> > > > the log
> > > > > > > >> start
> > > > > > > >> > > > offset
> > > > > > > >> > > > >> > with
> > > > > > > >> > > > >> > >> > this
> > > > > > > >> > > > >> > >> > > > > > > response.
> > > > > > > >> > > > >> > >> > > > > > > > > We
> > > > > > > >> > > > >> > >> > > > > > > > > >>> should
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> just retry the request
> > until
> > > > we
> > > > > > get
> > > > > > > >> it.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // The
> > > > > > UNKNOWN_PRODUCER_ID
> > > > > > > >> > error
> > > > > > > >> > > > code
> > > > > > > >> > > > >> > was
> > > > > > > >> > > > >> > >> > added
> > > > > > > >> > > > >> > >> > > > > along
> > > > > > > >> > > > >> > >> > > > > > > > with
> > > > > > > >> > > > >> > >> > > > > > > > > >>> the new
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> ProduceResponse which
> > > > includes the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           //
> > logStartOffset.
> > > > So
> > > > > > the
> > > > > > > >> '-1'
> > > > > > > >> > > > >> sentinel
> > > > > > > >> > > > >> > is
> > > > > > > >> > > > >> > >> > not
> > > > > > > >> > > > >> > >> > > > for
> > > > > > > >> > > > >> > >> > > > > > > > backward
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> compatibility. Instead,
> > it is
> > > > > > possible
> > > > > > > >> > for
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // a broker to
> > not
> > > > know
> > > > > > the
> > > > > > > >> > > > >> > >> logStartOffset at
> > > > > > > >> > > > >> > >> > > > when
> > > > > > > >> > > > >> > >> > > > > it
> > > > > > > >> > > > >> > >> > > > > > > is
> > > > > > > >> > > > >> > >> > > > > > > > > >>> returning
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> the response because the
> > > > partition
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // may have
> > moved
> > > > away
> > > > > > from
> > > > > > > >> the
> > > > > > > >> > > > >> broker
> > > > > > > >> > > > >> > >> from
> > > > > > > >> > > > >> > >> > the
> > > > > > > >> > > > >> > >> > > > > time
> > > > > > > >> > > > >> > >> > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> error was
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> initially raised to the
> > time
> > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // response
> was
> > > > being
> > > > > > > >> > > constructed.
> > > > > > > >> > > > In
> > > > > > > >> > > > >> > >> these
> > > > > > > >> > > > >> > >> > > > cases,
> > > > > > > >> > > > >> > >> > > > > we
> > > > > > > >> > > > >> > >> > > > > > > > > should
> > > > > > > >> > > > >> > >> > > > > > > > > >>> just
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> retry the request: we
> are
> > > > > > guaranteed
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // to
> eventually
> > > > get a
> > > > > > > >> > > > logStartOffset
> > > > > > > >> > > > >> > once
> > > > > > > >> > > > >> > >> > > things
> > > > > > > >> > > > >> > >> > > > > > > settle
> > > > > > > >> > > > >> > >> > > > > > > > > down.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>       if
> > > > > > > >> (batch.sequenceHasBeenReset()) {
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // When the
> > first
> > > > > > inflight
> > > > > > > >> > batch
> > > > > > > >> > > > >> fails
> > > > > > > >> > > > >> > >> due to
> > > > > > > >> > > > >> > >> > > the
> > > > > > > >> > > > >> > >> > > > > > > > > truncation
> > > > > > > >> > > > >> > >> > > > > > > > > >>> case,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> then the sequences of
> all
> > the
> > > > > > other
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // in flight
> > batches
> > > > > > would
> > > > > > > >> have
> > > > > > > >> > > > been
> > > > > > > >> > > > >> > >> > restarted
> > > > > > > >> > > > >> > >> > > > from
> > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> beginning.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> However, when those
> > responses
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // come back
> > from
> > > > the
> > > > > > > >> broker,
> > > > > > > >> > > they
> > > > > > > >> > > > >> would
> > > > > > > >> > > > >> > >> also
> > > > > > > >> > > > >> > >> > > > come
> > > > > > > >> > > > >> > >> > > > > > with
> > > > > > > >> > > > >> > >> > > > > > > > an
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > error. In
> > > > this
> > > > > > > >> case,
> > > > > > > >> > we
> > > > > > > >> > > > >> should
> > > > > > > >> > > > >> > >> not
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // reset the
> > > > sequence
> > > > > > > >> numbers
> > > > > > > >> > to
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > >> > beginning.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>       } else if
> > > > > > > >> > > > >> > >> > > > >
> > > > (lastAckedOffset(batch.topicPartition).orElse(
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > > > > > > >> > > > >> > >> > > > response.logStartOffset) {
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // The head of
> > the
> > > > log
> > > > > > has
> > > > > > > >> been
> > > > > > > >> > > > >> removed,
> > > > > > > >> > > > >> > >> > > probably
> > > > > > > >> > > > >> > >> > > > > due
> > > > > > > >> > > > >> > >> > > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> retention time elapsing.
> > In
> > > > this
> > > > > > case,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // we expect
> to
> > > > lose the
> > > > > > > >> > producer
> > > > > > > >> > > > >> state.
> > > > > > > >> > > > >> > >> For
> > > > > > > >> > > > >> > >> > > the
> > > > > > > >> > > > >> > >> > > > > > > > > transactional
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> producer, reset the
> > sequences
> > > > of
> > > > > > all
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // inflight
> > batches
> > > > to
> > > > > > be
> > > > > > > >> from
> > > > > > > >> > > the
> > > > > > > >> > > > >> > >> beginning
> > > > > > > >> > > > >> > >> > > and
> > > > > > > >> > > > >> > >> > > > > > retry
> > > > > > > >> > > > >> > >> > > > > > > > > them,
> > > > > > > >> > > > >> > >> > > > > > > > > >>> so
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> that the transaction
> does
> > not
> > > > > > need to
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // be aborted.
> > For
> > > > the
> > > > > > > >> > idempotent
> > > > > > > >> > > > >> > >> producer,
> > > > > > > >> > > > >> > >> > > bump
> > > > > > > >> > > > >> > >> > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > epoch
> > > > > > > >> > > > >> > >> > > > > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > >>> avoid
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> reusing (sequence,
> epoch)
> > > > pairs
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           if
> > > > (isTransactional()) {
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > >
> > > > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> this.producerIdAndEpoch);
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           } else {
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> requestEpochBumpForPartition(batch.topicPartition);
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           }
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>       if
> > (!isTransactional())
> > > > {
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // For the
> > > > idempotent
> > > > > > > >> producer,
> > > > > > > >> > > > >> always
> > > > > > > >> > > > >> > >> retry
> > > > > > > >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> errors. If the batch has
> > the
> > > > > > current
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // producer ID
> > and
> > > > > > epoch,
> > > > > > > >> > > request a
> > > > > > > >> > > > >> bump
> > > > > > > >> > > > >> > >> of
> > > > > > > >> > > > >> > >> > the
> > > > > > > >> > > > >> > >> > > > > > epoch.
> > > > > > > >> > > > >> > >> > > > > > > > > >>> Otherwise
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> just retry the produce.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > >
> > > > > > requestEpochBumpForPartition(batch.topicPartition);
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> }
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> I was considering
> keeping
> > this
> > > > > > > >> behavior —
> > > > > > > >> > > but
> > > > > > > >> > > > >> am
> > > > > > > >> > > > >> > >> open
> > > > > > > >> > > > >> > >> > to
> > > > > > > >> > > > >> > >> > > > > > > > simplifying
> > > > > > > >> > > > >> > >> > > > > > > > > >>> it.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> We are leaving changes
> to
> > > > older
> > > > > > > >> clients
> > > > > > > >> > off
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > >> table
> > > > > > > >> > > > >> > >> > > here
> > > > > > > >> > > > >> > >> > > > > > since
> > > > > > > >> > > > >> > >> > > > > > > it
> > > > > > > >> > > > >> > >> > > > > > > > > >>> caused
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> many issues for clients
> > in the
> > > > > > past.
> > > > > > > >> > > > Previously
> > > > > > > >> > > > >> > this
> > > > > > > >> > > > >> > >> > was
> > > > > > > >> > > > >> > >> > > a
> > > > > > > >> > > > >> > >> > > > > > fatal
> > > > > > > >> > > > >> > >> > > > > > > > > error
> > > > > > > >> > > > >> > >> > > > > > > > > >>> and
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> we didn't have the
> > mechanisms
> > > > in
> > > > > > > >> place to
> > > > > > > >> > > > >> detect
> > > > > > > >> > > > >> > >> when
> > > > > > > >> > > > >> > >> > > this
> > > > > > > >> > > > >> > >> > > > > was
> > > > > > > >> > > > >> > >> > > > > > a
> > > > > > > >> > > > >> > >> > > > > > > > > >>> legitimate
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> case vs some bug or gap
> > in the
> > > > > > > >> protocol.
> > > > > > > >> > > > >> Ensuring
> > > > > > > >> > > > >> > >> each
> > > > > > > >> > > > >> > >> > > > > > > transaction
> > > > > > > >> > > > >> > >> > > > > > > > > has
> > > > > > > >> > > > >> > >> > > > > > > > > >>> its
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> own epoch should close
> > this
> > > > gap.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> And to address Jeff's
> > second
> > > > > > point:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> *does the typical
> produce
> > > > request
> > > > > > path
> > > > > > > >> > > append
> > > > > > > >> > > > >> > >> records
> > > > > > > >> > > > >> > >> > to
> > > > > > > >> > > > >> > >> > > > > local
> > > > > > > >> > > > >> > >> > > > > > > log
> > > > > > > >> > > > >> > >> > > > > > > > > >>> along*
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> *with the
> > > > currentTxnFirstOffset
> > > > > > > >> > > information?
> > > > > > > >> > > > I
> > > > > > > >> > > > >> > would
> > > > > > > >> > > > >> > >> > like
> > > > > > > >> > > > >> > >> > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > >>> understand*
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> *when the field is
> > written to
> > > > > > disk.*
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> Yes, the first produce
> > request
> > > > > > > >> populates
> > > > > > > >> > > this
> > > > > > > >> > > > >> > field
> > > > > > > >> > > > >> > >> and
> > > > > > > >> > > > >> > >> > > > > writes
> > > > > > > >> > > > >> > >> > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> offset
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> as part of the record
> > batch
> > > > and
> > > > > > also
> > > > > > > >> to
> > > > > > > >> > the
> > > > > > > >> > > > >> > producer
> > > > > > > >> > > > >> > >> > > state
> > > > > > > >> > > > >> > >> > > > > > > > snapshot.
> > > > > > > >> > > > >> > >> > > > > > > > > >>> When
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> we reload the records on
> > > > restart
> > > > > > > >> and/or
> > > > > > > >> > > > >> > >> reassignment,
> > > > > > > >> > > > >> > >> > we
> > > > > > > >> > > > >> > >> > > > > > > repopulate
> > > > > > > >> > > > >> > >> > > > > > > > > >>> this
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> field with the snapshot
> > from
> > > > disk
> > > > > > > >> along
> > > > > > > >> > > with
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> rest
> > > > > > > >> > > > >> > >> > of
> > > > > > > >> > > > >> > >> > > > the
> > > > > > > >> > > > >> > >> > > > > > > > producer
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> state.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> Let me know if there are
> > > > further
> > > > > > > >> comments
> > > > > > > >> > > > >> and/or
> > > > > > > >> > > > >> > >> > > questions.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> Thanks,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> Justine
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> On Tue, Nov 22, 2022 at
> > 9:00
> > > > PM
> > > > > > Jeff
> > > > > > > >> Kim
> > > > > > > >> > > > >> > >> > > > > > > > > <jeff.kim@confluent.io.invalid
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> wrote:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Hi Justine,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Thanks for the KIP! I
> > have
> > > > two
> > > > > > > >> > questions:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> 1) For new clients, we
> > can
> > > > once
> > > > > > again
> > > > > > > >> > > return
> > > > > > > >> > > > >> an
> > > > > > > >> > > > >> > >> error
> > > > > > > >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> for sequences
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> that are non-zero when
> > there
> > > > is
> > > > > > no
> > > > > > > >> > > producer
> > > > > > > >> > > > >> state
> > > > > > > >> > > > >> > >> > > present
> > > > > > > >> > > > >> > >> > > > on
> > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> server.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> This will indicate we
> > missed
> > > > the
> > > > > > 0
> > > > > > > >> > > sequence
> > > > > > > >> > > > >> and
> > > > > > > >> > > > >> > we
> > > > > > > >> > > > >> > >> > don't
> > > > > > > >> > > > >> > >> > > > yet
> > > > > > > >> > > > >> > >> > > > > > > want
> > > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> write
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> to the log.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> I would like to
> > understand
> > > > the
> > > > > > > >> current
> > > > > > > >> > > > >> behavior
> > > > > > > >> > > > >> > to
> > > > > > > >> > > > >> > >> > > handle
> > > > > > > >> > > > >> > >> > > > > > older
> > > > > > > >> > > > >> > >> > > > > > > > > >>> clients,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> and if there are any
> > changes
> > > > we
> > > > > > are
> > > > > > > >> > > making.
> > > > > > > >> > > > >> Maybe
> > > > > > > >> > > > >> > >> I'm
> > > > > > > >> > > > >> > >> > > > > missing
> > > > > > > >> > > > >> > >> > > > > > > > > >>> something,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> but we would want to
> > identify
> > > > > > > >> whether we
> > > > > > > >> > > > >> missed
> > > > > > > >> > > > >> > >> the 0
> > > > > > > >> > > > >> > >> > > > > sequence
> > > > > > > >> > > > >> > >> > > > > > > for
> > > > > > > >> > > > >> > >> > > > > > > > > >>> older
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> clients, no?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> 2) Upon returning from
> > the
> > > > > > > >> transaction
> > > > > > > >> > > > >> > >> coordinator, we
> > > > > > > >> > > > >> > >> > > can
> > > > > > > >> > > > >> > >> > > > > set
> > > > > > > >> > > > >> > >> > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> transaction
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> as ongoing on the
> leader
> > by
> > > > > > > >> populating
> > > > > > > >> > > > >> > >> > > > currentTxnFirstOffset
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> through the typical
> > produce
> > > > > > request
> > > > > > > >> > > > handling.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> does the typical
> produce
> > > > request
> > > > > > path
> > > > > > > >> > > append
> > > > > > > >> > > > >> > >> records
> > > > > > > >> > > > >> > >> > to
> > > > > > > >> > > > >> > >> > > > > local
> > > > > > > >> > > > >> > >> > > > > > > log
> > > > > > > >> > > > >> > >> > > > > > > > > >>> along
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> with the
> > > > currentTxnFirstOffset
> > > > > > > >> > > information?
> > > > > > > >> > > > I
> > > > > > > >> > > > >> > would
> > > > > > > >> > > > >> > >> > like
> > > > > > > >> > > > >> > >> > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > understand
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> when the field is
> > written to
> > > > > > disk.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Thanks,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Jeff
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> On Tue, Nov 22, 2022 at
> > 4:44
> > > > PM
> > > > > > Artem
> > > > > > > >> > > > Livshits
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> <
> alivshits@confluent.io
> > > > .invalid>
> > > > > > > >> wrote:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> Hi Justine,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> Thank you for the KIP.
> > I
> > > > have
> > > > > > one
> > > > > > > >> > > > question.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> 5) For new clients, we
> > can
> > > > once
> > > > > > > >> again
> > > > > > > >> > > > return
> > > > > > > >> > > > >> an
> > > > > > > >> > > > >> > >> error
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> I believe we had
> > problems
> > > > in the
> > > > > > > >> past
> > > > > > > >> > > with
> > > > > > > >> > > > >> > >> returning
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> because it was
> > considered
> > > > fatal
> > > > > > and
> > > > > > > >> > > > required
> > > > > > > >> > > > >> > >> client
> > > > > > > >> > > > >> > >> > > > > restart.
> > > > > > > >> > > > >> > >> > > > > > > It
> > > > > > > >> > > > >> > >> > > > > > > > > >>> would
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> be
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> good to spell out the
> > new
> > > > client
> > > > > > > >> > behavior
> > > > > > > >> > > > >> when
> > > > > > > >> > > > >> > it
> > > > > > > >> > > > >> > >> > > > receives
> > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > error.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> -Artem
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> On Tue, Nov 22, 2022
> at
> > > > 10:00 AM
> > > > > > > >> > Justine
> > > > > > > >> > > > >> Olshan
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > <jo...@confluent.io.invalid>
> > > > > > > >> wrote:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks for taking a
> > look
> > > > > > Matthias.
> > > > > > > >> > I've
> > > > > > > >> > > > >> tried
> > > > > > > >> > > > >> > to
> > > > > > > >> > > > >> > >> > > answer
> > > > > > > >> > > > >> > >> > > > > your
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> questions
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> below:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 10)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Right — so the
> hanging
> > > > > > transaction
> > > > > > > >> > only
> > > > > > > >> > > > >> occurs
> > > > > > > >> > > > >> > >> when
> > > > > > > >> > > > >> > >> > we
> > > > > > > >> > > > >> > >> > > > > have
> > > > > > > >> > > > >> > >> > > > > > a
> > > > > > > >> > > > >> > >> > > > > > > > late
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> message
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> come in and the
> > partition
> > > > is
> > > > > > never
> > > > > > > >> > added
> > > > > > > >> > > > to
> > > > > > > >> > > > >> a
> > > > > > > >> > > > >> > >> > > > transaction
> > > > > > > >> > > > >> > >> > > > > > > again.
> > > > > > > >> > > > >> > >> > > > > > > > > If
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> we
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> never add the
> > partition to
> > > > a
> > > > > > > >> > > transaction,
> > > > > > > >> > > > we
> > > > > > > >> > > > >> > will
> > > > > > > >> > > > >> > >> > > never
> > > > > > > >> > > > >> > >> > > > > > write
> > > > > > > >> > > > >> > >> > > > > > > a
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> marker
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> and
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> never advance the
> LSO.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> If we do end up
> adding
> > the
> > > > > > > >> partition
> > > > > > > >> > to
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > >> > > transaction
> > > > > > > >> > > > >> > >> > > > (I
> > > > > > > >> > > > >> > >> > > > > > > > suppose
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> this
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> can
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> happen before or
> after
> > the
> > > > late
> > > > > > > >> > message
> > > > > > > >> > > > >> comes
> > > > > > > >> > > > >> > in)
> > > > > > > >> > > > >> > >> > then
> > > > > > > >> > > > >> > >> > > > we
> > > > > > > >> > > > >> > >> > > > > > will
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> include
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> late message in the
> > next
> > > > > > > >> (incorrect)
> > > > > > > >> > > > >> > transaction.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> So perhaps it is
> > clearer to
> > > > > > make
> > > > > > > >> the
> > > > > > > >> > > > >> > distinction
> > > > > > > >> > > > >> > >> > > between
> > > > > > > >> > > > >> > >> > > > > > > > messages
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> that
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> eventually get added
> > to the
> > > > > > > >> > transaction
> > > > > > > >> > > > (but
> > > > > > > >> > > > >> > the
> > > > > > > >> > > > >> > >> > wrong
> > > > > > > >> > > > >> > >> > > > > one)
> > > > > > > >> > > > >> > >> > > > > > or
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> messages
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> that never get added
> > and
> > > > become
> > > > > > > >> > hanging.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 20)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> The client side
> change
> > for
> > > > 2 is
> > > > > > > >> > removing
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > >> > > > addPartitions
> > > > > > > >> > > > >> > >> > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> transaction
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> call. We don't need
> to
> > make
> > > > > > this
> > > > > > > >> from
> > > > > > > >> > > the
> > > > > > > >> > > > >> > >> producer
> > > > > > > >> > > > >> > >> > to
> > > > > > > >> > > > >> > >> > > > the
> > > > > > > >> > > > >> > >> > > > > > txn
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> coordinator,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> only server side.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> In my opinion, the
> > issue
> > > > with
> > > > > > the
> > > > > > > >> > > > >> > >> addPartitionsToTxn
> > > > > > > >> > > > >> > >> > > > call
> > > > > > > >> > > > >> > >> > > > > > for
> > > > > > > >> > > > >> > >> > > > > > > > > older
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> clients
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> is that we don't have
> > the
> > > > epoch
> > > > > > > >> bump,
> > > > > > > >> > so
> > > > > > > >> > > > we
> > > > > > > >> > > > >> > don't
> > > > > > > >> > > > >> > >> > know
> > > > > > > >> > > > >> > >> > > > if
> > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>> message
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> belongs to the
> previous
> > > > > > > >> transaction or
> > > > > > > >> > > > this
> > > > > > > >> > > > >> > one.
> > > > > > > >> > > > >> > >> We
> > > > > > > >> > > > >> > >> > > need
> > > > > > > >> > > > >> > >> > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > check
> > > > > > > >> > > > >> > >> > > > > > > > > if
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> partition has been
> > added to
> > > > > > this
> > > > > > > >> > > > >> transaction.
> > > > > > > >> > > > >> > Of
> > > > > > > >> > > > >> > >> > > course,
> > > > > > > >> > > > >> > >> > > > > > this
> > > > > > > >> > > > >> > >> > > > > > > > > means
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> we
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> won't completely
> cover
> > the
> > > > case
> > > > > > > >> where
> > > > > > > >> > we
> > > > > > > >> > > > >> have a
> > > > > > > >> > > > >> > >> > really
> > > > > > > >> > > > >> > >> > > > > late
> > > > > > > >> > > > >> > >> > > > > > > > > message
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> and
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> we
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> have added the
> > partition to
> > > > > > the new
> > > > > > > >> > > > >> > transaction,
> > > > > > > >> > > > >> > >> but
> > > > > > > >> > > > >> > >> > > > > that's
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> unfortunately
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> something we will
> need
> > the
> > > > new
> > > > > > > >> clients
> > > > > > > >> > > to
> > > > > > > >> > > > >> > cover.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 30)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Transaction is
> ongoing
> > =
> > > > > > partition
> > > > > > > >> was
> > > > > > > >> > > > >> added to
> > > > > > > >> > > > >> > >> > > > > transaction
> > > > > > > >> > > > >> > >> > > > > > > via
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> addPartitionsToTxn.
> We
> > > > check
> > > > > > this
> > > > > > > >> with
> > > > > > > >> > > the
> > > > > > > >> > > > >> > >> > > > > > > DescribeTransactions
> > > > > > > >> > > > >> > >> > > > > > > > > >>> call.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Let
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> me know if this
> wasn't
> > > > > > sufficiently
> > > > > > > >> > > > >> explained
> > > > > > > >> > > > >> > >> here:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> > > > >> > >> > > > >
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > > >> > >> > >
> > > > > > > >> > > > >> > >> >
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >>
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >>
> > > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 40)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> The idea here is that
> > if
> > > > any
> > > > > > > >> messages
> > > > > > > >> > > > >> somehow
> > > > > > > >> > > > >> > >> come
> > > > > > > >> > > > >> > >> > in
> > > > > > > >> > > > >> > >> > > > > before
> > > > > > > >> > > > >> > >> > > > > > > we
> > > > > > > >> > > > >> > >> > > > > > > > > get
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> new
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> epoch to the
> producer,
> > they
> > > > > > will be
> > > > > > > >> > > > fenced.
> > > > > > > >> > > > >> > >> However,
> > > > > > > >> > > > >> > >> > > if
> > > > > > > >> > > > >> > >> > > > we
> > > > > > > >> > > > >> > >> > > > > > > don't
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> think
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> this
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> is necessary, it can
> be
> > > > > > discussed
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 50)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> It should be
> > synchronous
> > > > > > because
> > > > > > > >> if we
> > > > > > > >> > > > have
> > > > > > > >> > > > >> an
> > > > > > > >> > > > >> > >> event
> > > > > > > >> > > > >> > >> > > > (ie,
> > > > > > > >> > > > >> > >> > > > > an
> > > > > > > >> > > > >> > >> > > > > > > > > error)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> that
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> causes us to need to
> > abort
> > > > the
> > > > > > > >> > > > transaction,
> > > > > > > >> > > > >> we
> > > > > > > >> > > > >> > >> need
> > > > > > > >> > > > >> > >> > to
> > > > > > > >> > > > >> > >> > > > > know
> > > > > > > >> > > > >> > >> > > > > > > > which
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> partitions to send
> > > > transaction
> > > > > > > >> markers
> > > > > > > >> > > to.
> > > > > > > >> > > > >> We
> > > > > > > >> > > > >> > >> know
> > > > > > > >> > > > >> > >> > the
> > > > > > > >> > > > >> > >> > > > > > > > partitions
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> because
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> we added them to the
> > > > > > coordinator
> > > > > > > >> via
> > > > > > > >> > the
> > > > > > > >> > > > >> > >> > > > > addPartitionsToTxn
> > > > > > > >> > > > >> > >> > > > > > > > call.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Previously we have
> had
> > > > > > asynchronous
> > > > > > > >> > > calls
> > > > > > > >> > > > in
> > > > > > > >> > > > >> > the
> > > > > > > >> > > > >> > >> > past
> > > > > > > >> > > > >> > >> > > > (ie,
> > > > > > > >> > > > >> > >> > > > > > > > writing
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> commit markers when
> the
> > > > > > > >> transaction is
> > > > > > > >> > > > >> > completed)
> > > > > > > >> > > > >> > >> > but
> > > > > > > >> > > > >> > >> > > > > often
> > > > > > > >> > > > >> > >> > > > > > > this
> > > > > > > >> > > > >> > >> > > > > > > > > >>> just
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> causes confusion as
> we
> > > > need to
> > > > > > wait
> > > > > > > >> > for
> > > > > > > >> > > > some
> > > > > > > >> > > > >> > >> > > operations
> > > > > > > >> > > > >> > >> > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > complete.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> In
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> writing commit
> markers
> > > > case,
> > > > > > > >> clients
> > > > > > > >> > > often
> > > > > > > >> > > > >> see
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> CONCURRENT_TRANSACTIONs
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> error messages and
> that
> > > > can be
> > > > > > > >> > > confusing.
> > > > > > > >> > > > >> For
> > > > > > > >> > > > >> > >> that
> > > > > > > >> > > > >> > >> > > > reason,
> > > > > > > >> > > > >> > >> > > > > > it
> > > > > > > >> > > > >> > >> > > > > > > > may
> > > > > > > >> > > > >> > >> > > > > > > > > be
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> simpler to just have
> > > > > > synchronous
> > > > > > > >> > calls —
> > > > > > > >> > > > >> > >> especially
> > > > > > > >> > > > >> > >> > if
> > > > > > > >> > > > >> > >> > > > we
> > > > > > > >> > > > >> > >> > > > > > need
> > > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> block
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> on
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> some operation's
> > completion
> > > > > > anyway
> > > > > > > >> > > before
> > > > > > > >> > > > we
> > > > > > > >> > > > >> > can
> > > > > > > >> > > > >> > >> > start
> > > > > > > >> > > > >> > >> > > > the
> > > > > > > >> > > > >> > >> > > > > > > next
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> transaction. And
> yes, I
> > > > meant
> > > > > > > >> > > > coordinator. I
> > > > > > > >> > > > >> > will
> > > > > > > >> > > > >> > >> > fix
> > > > > > > >> > > > >> > >> > > > > that.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 60)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> When we are checking
> > if the
> > > > > > > >> > transaction
> > > > > > > >> > > is
> > > > > > > >> > > > >> > >> ongoing,
> > > > > > > >> > > > >> > >> > we
> > > > > > > >> > > > >> > >> > > > > need
> > > > > > > >> > > > >> > >> > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > make
> > > > > > > >> > > > >> > >> > > > > > > > > >>> a
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> round
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> trip from the leader
> > > > partition
> > > > > > to
> > > > > > > >> the
> > > > > > > >> > > > >> > transaction
> > > > > > > >> > > > >> > >> > > > > > coordinator.
> > > > > > > >> > > > >> > >> > > > > > > > In
> > > > > > > >> > > > >> > >> > > > > > > > > >>> the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> time
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> we are waiting for
> this
> > > > > > message to
> > > > > > > >> > come
> > > > > > > >> > > > >> back,
> > > > > > > >> > > > >> > in
> > > > > > > >> > > > >> > >> > > theory
> > > > > > > >> > > > >> > >> > > > we
> > > > > > > >> > > > >> > >> > > > > > > could
> > > > > > > >> > > > >> > >> > > > > > > > > >>> have
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> sent
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> a commit/abort call
> > that
> > > > would
> > > > > > make
> > > > > > > >> > the
> > > > > > > >> > > > >> > original
> > > > > > > >> > > > >> > >> > > result
> > > > > > > >> > > > >> > >> > > > of
> > > > > > > >> > > > >> > >> > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > check
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> out
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> of
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> date. That is why we
> > can
> > > > check
> > > > > > the
> > > > > > > >> > > leader
> > > > > > > >> > > > >> state
> > > > > > > >> > > > >> > >> > before
> > > > > > > >> > > > >> > >> > > > we
> > > > > > > >> > > > >> > >> > > > > > > write
> > > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> log.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> I'm happy to update
> the
> > > > KIP if
> > > > > > > >> some of
> > > > > > > >> > > > these
> > > > > > > >> > > > >> > >> things
> > > > > > > >> > > > >> > >> > > were
> > > > > > > >> > > > >> > >> > > > > not
> > > > > > > >> > > > >> > >> > > > > > > > > clear.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Justine
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> On Mon, Nov 21, 2022
> at
> > > > 7:11 PM
> > > > > > > >> > Matthias
> > > > > > > >> > > > J.
> > > > > > > >> > > > >> > Sax <
> > > > > > > >> > > > >> > >> > > > > > > > mjsax@apache.org
> > > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> wrote:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Thanks for the KIP.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Couple of
> > clarification
> > > > > > questions
> > > > > > > >> (I
> > > > > > > >> > am
> > > > > > > >> > > > >> not a
> > > > > > > >> > > > >> > >> > broker
> > > > > > > >> > > > >> > >> > > > > expert
> > > > > > > >> > > > >> > >> > > > > > > do
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> maybe
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> some question are
> > obvious
> > > > for
> > > > > > > >> others,
> > > > > > > >> > > but
> > > > > > > >> > > > >> not
> > > > > > > >> > > > >> > >> for
> > > > > > > >> > > > >> > >> > me
> > > > > > > >> > > > >> > >> > > > with
> > > > > > > >> > > > >> > >> > > > > > my
> > > > > > > >> > > > >> > >> > > > > > > > lack
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> of
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> broker knowledge).
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (10)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> 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.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> What happens if the
> > > > message
> > > > > > come
> > > > > > > >> in
> > > > > > > >> > > > before
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> next
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> addPartitionsToTxn
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> request? It seems
> the
> > > > broker
> > > > > > > >> hosting
> > > > > > > >> > > the
> > > > > > > >> > > > >> data
> > > > > > > >> > > > >> > >> > > > partitions
> > > > > > > >> > > > >> > >> > > > > > > won't
> > > > > > > >> > > > >> > >> > > > > > > > > know
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> anything about it
> and
> > > > append
> > > > > > it to
> > > > > > > >> > the
> > > > > > > >> > > > >> > >> partition,
> > > > > > > >> > > > >> > >> > > too?
> > > > > > > >> > > > >> > >> > > > > What
> > > > > > > >> > > > >> > >> > > > > > > is
> > > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> difference between
> > both
> > > > cases?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Also, it seems a TX
> > would
> > > > only
> > > > > > > >> hang,
> > > > > > > >> > if
> > > > > > > >> > > > >> there
> > > > > > > >> > > > >> > >> is no
> > > > > > > >> > > > >> > >> > > > > > following
> > > > > > > >> > > > >> > >> > > > > > > > TX
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> that
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> is
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> either committer or
> > > > aborted?
> > > > > > Thus,
> > > > > > > >> > for
> > > > > > > >> > > > the
> > > > > > > >> > > > >> > case
> > > > > > > >> > > > >> > >> > > above,
> > > > > > > >> > > > >> > >> > > > > the
> > > > > > > >> > > > >> > >> > > > > > TX
> > > > > > > >> > > > >> > >> > > > > > > > > might
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> actually not hang
> (of
> > > > course,
> > > > > > we
> > > > > > > >> > might
> > > > > > > >> > > > get
> > > > > > > >> > > > >> an
> > > > > > > >> > > > >> > >> EOS
> > > > > > > >> > > > >> > >> > > > > violation
> > > > > > > >> > > > >> > >> > > > > > > if
> > > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> first
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> TX was aborted and
> the
> > > > second
> > > > > > > >> > > committed,
> > > > > > > >> > > > or
> > > > > > > >> > > > >> > the
> > > > > > > >> > > > >> > >> > other
> > > > > > > >> > > > >> > >> > > > way
> > > > > > > >> > > > >> > >> > > > > > > > > around).
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (20)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Of course, 1 and 2
> > > > require
> > > > > > > >> > client-side
> > > > > > > >> > > > >> > >> changes, so
> > > > > > > >> > > > >> > >> > > for
> > > > > > > >> > > > >> > >> > > > > > older
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> clients,
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> those approaches
> won’t
> > > > apply.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> For (1) I understand
> > why a
> > > > > > client
> > > > > > > >> > > change
> > > > > > > >> > > > is
> > > > > > > >> > > > >> > >> > > necessary,
> > > > > > > >> > > > >> > >> > > > > but
> > > > > > > >> > > > >> > >> > > > > > > not
> > > > > > > >> > > > >> > >> > > > > > > > > sure
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> why
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> we need a client
> > change
> > > > for
> > > > > > (2).
> > > > > > > >> Can
> > > > > > > >> > > you
> > > > > > > >> > > > >> > >> elaborate?
> > > > > > > >> > > > >> > >> > > --
> > > > > > > >> > > > >> > >> > > > > > Later
> > > > > > > >> > > > >> > >> > > > > > > > you
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> explain
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> that we should send
> a
> > > > > > > >> > > > >> > >> DescribeTransactionRequest,
> > > > > > > >> > > > >> > >> > > but I
> > > > > > > >> > > > >> > >> > > > > am
> > > > > > > >> > > > >> > >> > > > > > > not
> > > > > > > >> > > > >> > >> > > > > > > > > sure
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> why?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Can't we not just do
> > an
> > > > > > implicit
> > > > > > > >> > > > >> > >> AddPartiitonToTx,
> > > > > > > >> > > > >> > >> > > too?
> > > > > > > >> > > > >> > >> > > > > If
> > > > > > > >> > > > >> > >> > > > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > old
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> producer correctly
> > > > registered
> > > > > > the
> > > > > > > >> > > > partition
> > > > > > > >> > > > >> > >> > already,
> > > > > > > >> > > > >> > >> > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> TX-coordinator
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> can just ignore it
> as
> > > > it's an
> > > > > > > >> > > idempotent
> > > > > > > >> > > > >> > >> operation?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (30)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> To cover older
> > clients,
> > > > we
> > > > > > will
> > > > > > > >> > > ensure a
> > > > > > > >> > > > >> > >> > transaction
> > > > > > > >> > > > >> > >> > > > is
> > > > > > > >> > > > >> > >> > > > > > > > ongoing
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> before
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> we write to a
> > transaction
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Not sure what you
> > mean by
> > > > > > this?
> > > > > > > >> Can
> > > > > > > >> > you
> > > > > > > >> > > > >> > >> elaborate?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (40)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> [the
> TX-coordinator]
> > will
> > > > > > write
> > > > > > > >> the
> > > > > > > >> > > > >> prepare
> > > > > > > >> > > > >> > >> commit
> > > > > > > >> > > > >> > >> > > > > message
> > > > > > > >> > > > >> > >> > > > > > > > with
> > > > > > > >> > > > >> > >> > > > > > > > > a
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> bumped
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> epoch and send
> > > > > > > >> WriteTxnMarkerRequests
> > > > > > > >> > > > with
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > >> > bumped
> > > > > > > >> > > > >> > >> > > > > > epoch.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Why do we use the
> > bumped
> > > > > > epoch for
> > > > > > > >> > > both?
> > > > > > > >> > > > It
> > > > > > > >> > > > >> > >> seems
> > > > > > > >> > > > >> > >> > > more
> > > > > > > >> > > > >> > >> > > > > > > > intuitive
> > > > > > > >> > > > >> > >> > > > > > > > > to
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> use
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> the current epoch,
> and
> > > > only
> > > > > > return
> > > > > > > >> > the
> > > > > > > >> > > > >> bumped
> > > > > > > >> > > > >> > >> epoch
> > > > > > > >> > > > >> > >> > > to
> > > > > > > >> > > > >> > >> > > > > the
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> producer?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (50) "Implicit
> > > > > > > >> > > AddPartitionToTransaction"
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Why does the
> > implicitly
> > > > sent
> > > > > > > >> request
> > > > > > > >> > > need
> > > > > > > >> > > > >> to
> > > > > > > >> > > > >> > be
> > > > > > > >> > > > >> > >> > > > > > synchronous?
> > > > > > > >> > > > >> > >> > > > > > > > The
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> KIP
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> also says
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> in case we need to
> > abort
> > > > and
> > > > > > > >> need to
> > > > > > > >> > > > know
> > > > > > > >> > > > >> > which
> > > > > > > >> > > > >> > >> > > > > partitions
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> What do you mean by
> > this?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> we don’t want to
> > write
> > > > to it
> > > > > > > >> before
> > > > > > > >> > we
> > > > > > > >> > > > >> store
> > > > > > > >> > > > >> > in
> > > > > > > >> > > > >> > >> > the
> > > > > > > >> > > > >> > >> > > > > > > > transaction
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> manager
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Do you mean
> > TX-coordinator
> > > > > > > >> instead of
> > > > > > > >> > > > >> > "manager"?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (60)
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> For older clients
> and
> > > > ensuring
> > > > > > > >> that
> > > > > > > >> > the
> > > > > > > >> > > > TX
> > > > > > > >> > > > >> is
> > > > > > > >> > > > >> > >> > > ongoing,
> > > > > > > >> > > > >> > >> > > > > you
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>> describe a
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> race condition. I am
> > not
> > > > sure
> > > > > > if I
> > > > > > > >> > can
> > > > > > > >> > > > >> follow
> > > > > > > >> > > > >> > >> here.
> > > > > > > >> > > > >> > >> > > Can
> > > > > > > >> > > > >> > >> > > > > you
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> elaborate?
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> -Matthias
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> On 11/18/22 1:21 PM,
> > > > Justine
> > > > > > > >> Olshan
> > > > > > > >> > > > wrote:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Hey all!
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> I'd like to start a
> > > > > > discussion
> > > > > > > >> on my
> > > > > > > >> > > > >> proposal
> > > > > > > >> > > > >> > >> to
> > > > > > > >> > > > >> > >> > add
> > > > > > > >> > > > >> > >> > > > > some
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> server-side
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> checks on
> > transactions to
> > > > > > avoid
> > > > > > > >> > > hanging
> > > > > > > >> > > > >> > >> > > transactions.
> > > > > > > >> > > > >> > >> > > > I
> > > > > > > >> > > > >> > >> > > > > > know
> > > > > > > >> > > > >> > >> > > > > > > > > this
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> has
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> been
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> an issue for some
> > time,
> > > > so I
> > > > > > > >> really
> > > > > > > >> > > hope
> > > > > > > >> > > > >> this
> > > > > > > >> > > > >> > >> KIP
> > > > > > > >> > > > >> > >> > > will
> > > > > > > >> > > > >> > >> > > > > be
> > > > > > > >> > > > >> > >> > > > > > > > > helpful
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> for
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> many
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> users of EOS.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> The KIP includes
> > changes
> > > > that
> > > > > > > >> will
> > > > > > > >> > be
> > > > > > > >> > > > >> > >> compatible
> > > > > > > >> > > > >> > >> > > with
> > > > > > > >> > > > >> > >> > > > > old
> > > > > > > >> > > > >> > >> > > > > > > > > clients
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>> and
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> changes to improve
> > > > > > performance
> > > > > > > >> and
> > > > > > > >> > > > >> > correctness
> > > > > > > >> > > > >> > >> on
> > > > > > > >> > > > >> > >> > > new
> > > > > > > >> > > > >> > >> > > > > > > clients.
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Please take a look
> > and
> > > > leave
> > > > > > any
> > > > > > > >> > > > comments
> > > > > > > >> > > > >> you
> > > > > > > >> > > > >> > >> may
> > > > > > > >> > > > >> > >> > > > have!
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> KIP:
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> > > > >> > >> > > > >
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > > >> > >> > >
> > > > > > > >> > > > >> > >> >
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >>
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >>
> > > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> JIRA:
> > > > > > > >> > > > >> > >> > > >
> > > > https://issues.apache.org/jira/browse/KAFKA-14402
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Thanks!
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Justine
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > > >
> > > > > > > >> > > > >> > >> > > > > > >
> > > > > > > >> > > > >> > >> > > > > >
> > > > > > > >> > > > >> > >> > > > >
> > > > > > > >> > > > >> > >> > > >
> > > > > > > >> > > > >> > >> > >
> > > > > > > >> > > > >> > >> >
> > > > > > > >> > > > >> > >>
> > > > > > > >> > > > >> > >
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >>
> > > > > > > >> > > > >
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >>
> > > > > > > >
> > > > > >
> > > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
Artem -- I'm a bit confused because the epoch bump is done from the client
side. The code that references Short.MAX_VALUE is on the client. Are you
concerned about non-Java clients here or something else?
I can add some explicit text in the KIP about this. But my understanding is
that it could work something like the following:
1. server returns max value after transaction completes
2. client will check epoch and if it is max value, allocate new producer
ID.

The only difference here is instead of client bumping epoch itself and
doing this check, it must do the check when handling the response from
committing the transaction.

Guozhang -- yup that makes sense. I see the same. I just wanted to confirm
your suggestion was to shut down the producer totally and not just fail the
batch. I think we are aligned here.

Thanks again for the discussion all!
Justine

On Fri, Jan 20, 2023 at 2:57 PM Guozhang Wang <gu...@gmail.com>
wrote:

> For InvalidRecord -- when I did the research on KIP-691 I saw from the
> code we would treat them as abortableError, not fatalError inside
> Producer's TxnManager (again sorry for raising such implementation
> details in KIP discussions), so I think, from client's perspective,
> they will not cause the client to be shutdown automatically?
>
> On Fri, Jan 20, 2023 at 2:52 PM Justine Olshan
> <jo...@confluent.io.invalid> wrote:
> >
> > That's a fair point about other clients.
> >
> > I think the abortable error case is interesting because I'm curious how
> > other clients would handle this. I assume they would need to implement
> > handling for the error code unless they did something like "any unknown
> > error codes/any codes that aren't x,y,z are retriable." I would hope that
> > unknown error codes were fatal, and if the code was implemented it would
> > abort the transaction. But I will think on this too.
> >
> > As for InvalidRecord -- you mentioned it was not fatal, but I'm taking a
> > look through the code. We would see this on handling the produce
> response.
> > If I recall correctly, we check if errors are retriable. I think this
> error
> > would not be retriable. But I guess the concern here is that it is not
> > enough for just that batch to fail. I guess I hadn't considered fully
> > fencing the old producer but there are valid arguments here why we would
> > want to.
> >
> > Thanks,
> > Justine
> >
> > On Fri, Jan 20, 2023 at 2:35 PM Guozhang Wang <
> guozhang.wang.us@gmail.com>
> > wrote:
> >
> > > Thanks Justine for the replies! I agree with most of your thoughts.
> > >
> > > Just for 3/7), though I agree for our own AK producer, since we do
> > > "nextRequest(boolean hasIncompleteBatches)", we guarantee the end-txn
> > > would not be sent until we've effectively flushed, but I was referring
> > > to any future bugs or other buggy clients that the same client may get
> > > into this situation, in which case we should give the client a clear
> > > msg that "you did something wrong, and hence now you should fatally
> > > close yourself". What I'm concerned about is that, by seeing an
> > > "abortable error" or in some rare cases an "invalid record", the
> > > client could not realize "something that's really bad happened". So
> > > it's not about adding a new error, it's mainly about those real buggy
> > > situations causing such "should never happen" cases, the errors return
> > > would not be informative enough.
> > >
> > > Thinking in other ways, if we believe that for most cases such error
> > > codes would not reach the original clients since they would be
> > > disconnected or even gone by that time, and only in some rare cases
> > > they would still be seen by the sending clients, then why not make
> > > them more fatal and more specific than generic.
> > >
> > > Guozhang
> > >
> > > On Fri, Jan 20, 2023 at 1:59 PM Justine Olshan
> > > <jo...@confluent.io.invalid> wrote:
> > > >
> > > > Hey Guozhang. Thanks for taking a look and for the detailed comments!
> > > I'll
> > > > do my best to address below.
> > > >
> > > > 1. I see what you are saying here, but I think I need to look
> through the
> > > > sequence of events you mention. Typically we've seen this issue in a
> few
> > > > cases.
> > > >
> > > >  One is when we have a producer disconnect when trying to produce.
> > > > Typically in these cases, we abort the transaction. We've seen that
> after
> > > > the markers are written, the disconnection can sometimes cause the
> > > request
> > > > to get flushed to the broker. In this case, we don't need client
> handling
> > > > because the producer we are responding to is gone. We just needed to
> make
> > > > sure we didn't write to the log on the broker side. I'm trying to
> think
> > > of
> > > > a case where we do have the client to return to. I'd think the same
> > > client
> > > > couldn't progress to committing the transaction unless the produce
> > > request
> > > > returned right? Of course, there is the incorrectly written clients
> case.
> > > > I'll think on this a bit more and let you know if I come up with
> another
> > > > scenario when we would return to an active client when the
> transaction is
> > > > no longer ongoing.
> > > >
> > > > I was not aware that we checked the result of a send after we commit
> > > > though. I'll need to look into that a bit more.
> > > >
> > > > 2. There were some questions about this in the discussion. The plan
> is to
> > > > handle overflow with the mechanism we currently have in the
> producer. If
> > > we
> > > > try to bump and the epoch will overflow, we actually allocate a new
> > > > producer ID. I need to confirm the fencing logic on the last epoch
> (ie,
> > > we
> > > > probably shouldn't allow any records to be produced with the final
> epoch
> > > > since we can never properly fence that one).
> > > >
> > > > 3. I can agree with you that the current error handling is messy. I
> > > recall
> > > > taking a look at your KIP a while back, but I think I mostly saw the
> > > > section about how the errors were wrapped. Maybe I need to take
> another
> > > > look. As for abortable error, the idea was that the handling would be
> > > > simple -- if this error is seen, the transaction should be aborted
> -- no
> > > > other logic about previous state or requests necessary. Is your
> concern
> > > > simply about adding new errors? We were hoping to have an error that
> > > would
> > > > have one meaning and many of the current errors have a history of
> meaning
> > > > different things on different client versions. That was the main
> > > motivation
> > > > for adding a new error.
> > > >
> > > > 4. This is a good point about record timestamp reordering. Timestamps
> > > don't
> > > > affect compaction, but they do affect retention deletion. For that,
> kafka
> > > > considers the largest timestamp in the segment, so I think a small
> amount
> > > > of reordering (hopefully on the order of milliseconds or even
> seconds)
> > > will
> > > > be ok. We take timestamps from clients so there is already a
> possibility
> > > > for some drift and non-monotonically increasing timestamps.
> > > >
> > > > 5. Thanks for catching. The error is there, but it's actually that
> those
> > > > fields should be 4+! Due to how the message generator works, I
> actually
> > > > have to redefine those fields inside the
> `"AddPartitionsToTxnTransaction`
> > > > block for it to build correctly. I'll fix it to be correct.
> > > >
> > > > 6. Correct -- we will only add the request to purgatory if the cache
> has
> > > no
> > > > ongoing transaction. I can change the wording to make that clearer
> that
> > > we
> > > > only place the request in purgatory if we need to contact the
> transaction
> > > > coordinator.
> > > >
> > > > 7. We did take a look at some of the errors and it was hard to come
> up
> > > with
> > > > a good one. I agree that InvalidTxnStateException is ideal except
> for the
> > > > fact that it hasn't been returned on Produce requests before. The
> error
> > > > handling for clients is a bit vague (which is why I opened
> KAFKA-14439
> > > > <https://issues.apache.org/jira/browse/KAFKA-14439>), but the
> decision
> > > we
> > > > made here was to only return errors that have been previously
> returned to
> > > > producers. As for not being fatal, I think part of the theory was
> that in
> > > > many cases, the producer would be disconnected. (See point 1) and
> this
> > > > would just be an error to return from the server. I did plan to think
> > > about
> > > > other cases, so let me know if you think of any as well!
> > > >
> > > > Lots to say! Let me know if you have further thoughts!
> > > > Justine
> > > >
> > > > On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang <
> > > guozhang.wang.us@gmail.com>
> > > > wrote:
> > > >
> > > > > Hello Justine,
> > > > >
> > > > > Thanks for the great write-up! I made a quick pass through it and
> here
> > > > > are some thoughts (I have not been able to read through this
> thread so
> > > > > pardon me if they have overlapped or subsumed by previous
> comments):
> > > > >
> > > > > First are some meta ones:
> > > > >
> > > > > 1. I think we need to also improve the client's experience once we
> > > > > have this defence in place. More concretely, say a user's producer
> > > > > code is like following:
> > > > >
> > > > > future = producer.send();
> > > > > // producer.flush();
> > > > > producer.commitTransaction();
> > > > > future.get();
> > > > >
> > > > > Which resulted in the order of a) produce-request sent by
> producer, b)
> > > > > end-txn-request sent by producer, c) end-txn-response sent back, d)
> > > > > txn-marker-request sent from coordinator to partition leader, e)
> > > > > produce-request finally received by the partition leader, before
> this
> > > > > KIP e) step would be accepted causing a dangling txn; now it would
> be
> > > > > rejected in step e) which is good. But from the client's point of
> view
> > > > > now it becomes confusing since the `commitTransaction()` returns
> > > > > successfully, but the "future" throws an invalid-epoch error, and
> they
> > > > > are not sure if the transaction did succeed or not. In fact, it
> > > > > "partially succeeded" with some msgs being rejected but others
> > > > > committed successfully.
> > > > >
> > > > > Of course the easy way to avoid this is, always call
> > > > > "producer.flush()" before commitTxn and that's what we do
> ourselves,
> > > > > and what we recommend users do. But I suspect not everyone does
> it. In
> > > > > fact I just checked the javadoc in KafkaProducer and our code
> snippet
> > > > > does not include a `flush()` call. So I'm thinking maybe we can in
> > > > > side the `commitTxn` code to enforce flushing before sending the
> > > > > end-txn request.
> > > > >
> > > > > 2. I'd like to clarify a bit details on "just add partitions to the
> > > > > transaction on the first produce request during a transaction". My
> > > > > understanding is that the partition leader's cache has the
> producer id
> > > > > / sequence / epoch for the latest txn, either on-going or is
> completed
> > > > > (upon receiving the marker request from coordinator). When a
> produce
> > > > > request is received, if
> > > > >
> > > > > * producer's epoch < cached epoch, or producer's epoch == cached
> epoch
> > > > > but the latest txn is completed, leader directly reject with
> > > > > invalid-epoch.
> > > > > * producer's epoch > cached epoch, park the the request and send
> > > > > add-partitions request to coordinator.
> > > > >
> > > > > In order to do it, does the coordinator need to bump the sequence
> and
> > > > > reset epoch to 0 when the next epoch is going to overflow? If no
> need
> > > > > to do so, then how we handle the (admittedly rare, but still may
> > > > > happen) epoch overflow situation?
> > > > >
> > > > > 3. I'm a bit concerned about adding a generic "ABORTABLE_ERROR"
> given
> > > > > we already have a pretty messy error classification and error
> handling
> > > > > on the producer clients side --- I have a summary about the issues
> and
> > > > > a proposal to address this in
> > > > >
> > > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
> > > > > -- I understand we do not want to use "UNKNOWN_PRODUCER_ID" anymore
> > > > > and in fact we intend to deprecate it in KIP-360 and eventually
> remove
> > > > > it; but I'm wondering can we still use specific error codes. E.g.
> what
> > > > > about "InvalidProducerEpochException" since for new clients, the
> > > > > actual reason this would actually be rejected is indeed because the
> > > > > epoch on the coordinator caused the add-partitions-request from the
> > > > > brokers to be rejected anyways?
> > > > >
> > > > > 4. It seems we put the producer request into purgatory before we
> ever
> > > > > append the records, while other producer's records may still be
> > > > > appended during the time; and that potentially may result in some
> > > > > re-ordering compared with reception order. I'm not super concerned
> > > > > about it since Kafka does not guarantee reception ordering across
> > > > > producers anyways, but it may make the timestamps of records
> inside a
> > > > > partition to be more out-of-ordered. Are we aware of any scenarios
> > > > > such as future enhancements on log compactions that may be
> affected by
> > > > > this effect?
> > > > >
> > > > > Below are just minor comments:
> > > > >
> > > > > 5. In "AddPartitionsToTxnTransaction" field of
> > > > > "AddPartitionsToTxnRequest" RPC, the versions of those inner fields
> > > > > are "0-3" while I thought they should be "0+" still?
> > > > >
> > > > > 6. Regarding "we can place the request in a purgatory of sorts and
> > > > > check if there is any state for the transaction on the broker": i
> > > > > think at this time when we just do the checks against the cached
> > > > > state, we do not need to put the request to purgatory yet?
> > > > >
> > > > > 7. This is related to 3) above. I feel using
> "InvalidRecordException"
> > > > > for older clients may also be a bit confusing, and also it is not
> > > > > fatal -- for old clients, it better to be fatal since this
> indicates
> > > > > the clients is doing something wrong and hence it should be closed.
> > > > > And in general I'd prefer to use slightly more specific meaning
> error
> > > > > codes for clients. That being said, I also feel
> > > > > "InvalidProducerEpochException" is not suitable for old versioned
> > > > > clients, and we'd have to pick one that old clients recognize. I'd
> > > > > prefer "InvalidTxnStateException" but that one is supposed to be
> > > > > returned from txn coordinators only today. I'd suggest we do a
> quick
> > > > > check in the current client's code path and see if that one would
> be
> > > > > handled if it's from a produce-response, and if yes, use this one;
> > > > > otherwise, use "ProducerFencedException" which is much less
> meaningful
> > > > > but it's still a fatal error.
> > > > >
> > > > >
> > > > > Thanks,
> > > > > Guozhang
> > > > >
> > > > >
> > > > >
> > > > > On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
> > > > > <jo...@confluent.io.invalid> wrote:
> > > > > >
> > > > > > Yeah -- looks like we already have code to handle bumping the
> epoch
> > > and
> > > > > > when the epoch is Short.MAX_VALUE, we get a new producer ID.
> Since
> > > this
> > > > > is
> > > > > > already the behavior, do we want to change it further?
> > > > > >
> > > > > > Justine
> > > > > >
> > > > > > On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <
> jolshan@confluent.io
> > > >
> > > > > wrote:
> > > > > >
> > > > > > > Hey all, just wanted to quickly update and say I've modified
> the
> > > KIP to
> > > > > > > explicitly mention that AddOffsetCommitsToTxnRequest will be
> > > replaced
> > > > > by
> > > > > > > a coordinator-side (inter-broker) AddPartitionsToTxn implicit
> > > request.
> > > > > This
> > > > > > > mirrors the user partitions and will implicitly add offset
> > > partitions
> > > > > to
> > > > > > > transactions when we commit offsets on them. We will deprecate
> > > > > AddOffsetCommitsToTxnRequest
> > > > > > > for new clients.
> > > > > > >
> > > > > > > Also to address Artem's comments --
> > > > > > > I'm a bit unsure if the changes here will change the previous
> > > behavior
> > > > > for
> > > > > > > fencing producers. In the case you mention in the first
> paragraph,
> > > are
> > > > > you
> > > > > > > saying we bump the epoch before we try to abort the
> transaction? I
> > > > > think I
> > > > > > > need to understand the scenarios you mention a bit better.
> > > > > > >
> > > > > > > As for the second part -- I think it makes sense to have some
> sort
> > > of
> > > > > > > "sentinel" epoch to signal epoch is about to overflow (I think
> we
> > > sort
> > > > > of
> > > > > > > have this value in place in some ways) so we can codify it in
> the
> > > KIP.
> > > > > I'll
> > > > > > > look into that and try to update soon.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Justine.
> > > > > > >
> > > > > > > On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
> > > > > > > <al...@confluent.io.invalid> wrote:
> > > > > > >
> > > > > > >> It's good to know that KIP-588 addressed some of the issues.
> > > Looking
> > > > > at
> > > > > > >> the code, it still looks like there are some cases that would
> > > result
> > > > > in
> > > > > > >> fatal error, e.g. PRODUCER_FENCED is issued by the transaction
> > > > > coordinator
> > > > > > >> if epoch doesn't match, and the client treats it as a fatal
> error
> > > > > (code in
> > > > > > >> TransactionManager request handling).  If we consider, for
> > > example,
> > > > > > >> committing a transaction that returns a timeout, but actually
> > > > > succeeds,
> > > > > > >> trying to abort it or re-commit may result in PRODUCER_FENCED
> > > error
> > > > > > >> (because of epoch bump).
> > > > > > >>
> > > > > > >> For failed commits, specifically, we need to know the actual
> > > outcome,
> > > > > > >> because if we return an error the application may think that
> the
> > > > > > >> transaction is aborted and redo the work, leading to
> duplicates.
> > > > > > >>
> > > > > > >> Re: overflowing epoch.  We could either do it on the TC and
> return
> > > > > both
> > > > > > >> producer id and epoch (e.g. change the protocol), or signal
> the
> > > client
> > > > > > >> that
> > > > > > >> it needs to get a new producer id.  Checking for max epoch
> could
> > > be a
> > > > > > >> reasonable signal, the value to check should probably be
> present
> > > in
> > > > > the
> > > > > > >> KIP
> > > > > > >> as this is effectively a part of the contract.  Also, the TC
> > > should
> > > > > > >> probably return an error if the client didn't change producer
> id
> > > after
> > > > > > >> hitting max epoch.
> > > > > > >>
> > > > > > >> -Artem
> > > > > > >>
> > > > > > >>
> > > > > > >> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
> > > > > > >> <jo...@confluent.io.invalid> wrote:
> > > > > > >>
> > > > > > >> > Thanks for the discussion Artem.
> > > > > > >> >
> > > > > > >> > With respect to the handling of fenced producers, we have
> some
> > > > > behavior
> > > > > > >> > already in place. As of KIP-588:
> > > > > > >> >
> > > > > > >> >
> > > > > > >>
> > > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> > > > > > >> > ,
> > > > > > >> > we handle timeouts more gracefully. The producer can
> recover.
> > > > > > >> >
> > > > > > >> > Produce requests can also recover from epoch fencing by
> > > aborting the
> > > > > > >> > transaction and starting over.
> > > > > > >> >
> > > > > > >> > What other cases were you considering that would cause us to
> > > have a
> > > > > > >> fenced
> > > > > > >> > epoch but we'd want to recover?
> > > > > > >> >
> > > > > > >> > The first point about handling epoch overflows is fair. I
> think
> > > > > there is
> > > > > > >> > some logic we'd need to consider. (ie, if we are one away
> from
> > > the
> > > > > max
> > > > > > >> > epoch, we need to reset the producer ID.) I'm still
> wondering if
> > > > > there
> > > > > > >> is a
> > > > > > >> > way to direct this from the response, or if everything
> should be
> > > > > done on
> > > > > > >> > the client side. Let me know if you have any thoughts here.
> > > > > > >> >
> > > > > > >> > Thanks,
> > > > > > >> > Justine
> > > > > > >> >
> > > > > > >> > On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
> > > > > > >> > <al...@confluent.io.invalid> wrote:
> > > > > > >> >
> > > > > > >> > > There are some workflows in the client that are implied by
> > > > > protocol
> > > > > > >> > > changes, e.g.:
> > > > > > >> > >
> > > > > > >> > > - for new clients, epoch changes with every transaction
> and
> > > can
> > > > > > >> overflow,
> > > > > > >> > > in old clients this condition was handled transparently,
> > > because
> > > > > epoch
> > > > > > >> > was
> > > > > > >> > > bumped in InitProducerId and it would return a new
> producer
> > > id if
> > > > > > >> epoch
> > > > > > >> > > overflows, the new clients would need to implement some
> > > workflow
> > > > > to
> > > > > > >> > refresh
> > > > > > >> > > producer id
> > > > > > >> > > - how to handle fenced producers, for new clients epoch
> > > changes
> > > > > with
> > > > > > >> > every
> > > > > > >> > > transaction, so in presence of failures during commits /
> > > aborts,
> > > > > the
> > > > > > >> > > producer could get easily fenced, old clients would pretty
> > > much
> > > > > would
> > > > > > >> get
> > > > > > >> > > fenced when a new incarnation of the producer was
> initialized
> > > with
> > > > > > >> > > InitProducerId so it's ok to treat as a fatal error, the
> new
> > > > > clients
> > > > > > >> > would
> > > > > > >> > > need to implement some workflow to handle that error,
> > > otherwise
> > > > > they
> > > > > > >> > could
> > > > > > >> > > get fenced by themselves
> > > > > > >> > > - in particular (as a subset of the previous issue), what
> > > would
> > > > > the
> > > > > > >> > client
> > > > > > >> > > do if it got a timeout during commit?  commit could've
> > > succeeded
> > > > > or
> > > > > > >> > failed
> > > > > > >> > >
> > > > > > >> > > Not sure if this has to be defined in the KIP as
> implementing
> > > > > those
> > > > > > >> > > probably wouldn't require protocol changes, but we have
> > > multiple
> > > > > > >> > > implementations of Kafka clients, so probably would be
> good to
> > > > > have
> > > > > > >> some
> > > > > > >> > > client implementation guidance.  Could also be done as a
> > > separate
> > > > > doc.
> > > > > > >> > >
> > > > > > >> > > -Artem
> > > > > > >> > >
> > > > > > >> > > On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
> > > > > > >> > <jolshan@confluent.io.invalid
> > > > > > >> > > >
> > > > > > >> > > wrote:
> > > > > > >> > >
> > > > > > >> > > > Hey all, I've updated the KIP to incorporate Jason's
> > > > > suggestions.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > > 1. Use AddPartitionsToTxn + verify flag to check on old
> > > clients
> > > > > > >> > > > 2. Updated AddPartitionsToTxn API to support transaction
> > > > > batching
> > > > > > >> > > > 3. Mention IBP bump
> > > > > > >> > > > 4. Mention auth change on new AddPartitionsToTxn
> version.
> > > > > > >> > > >
> > > > > > >> > > > I'm planning on opening a vote soon.
> > > > > > >> > > > Thanks,
> > > > > > >> > > > Justine
> > > > > > >> > > >
> > > > > > >> > > > On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <
> > > > > jolshan@confluent.io
> > > > > > >> >
> > > > > > >> > > > wrote:
> > > > > > >> > > >
> > > > > > >> > > > > Thanks Jason. Those changes make sense to me. I will
> > > update
> > > > > the
> > > > > > >> KIP.
> > > > > > >> > > > >
> > > > > > >> > > > >
> > > > > > >> > > > >
> > > > > > >> > > > > On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
> > > > > > >> > > > <ja...@confluent.io.invalid>
> > > > > > >> > > > > wrote:
> > > > > > >> > > > >
> > > > > > >> > > > >> Hey Justine,
> > > > > > >> > > > >>
> > > > > > >> > > > >> > I was wondering about compatibility here. When we
> send
> > > > > requests
> > > > > > >> > > > >> between brokers, we want to ensure that the receiving
> > > broker
> > > > > > >> > > understands
> > > > > > >> > > > >> the request (specifically the new fields). Typically
> > > this is
> > > > > done
> > > > > > >> > via
> > > > > > >> > > > >> IBP/metadata version.
> > > > > > >> > > > >> I'm trying to think if there is a way around it but
> I'm
> > > not
> > > > > sure
> > > > > > >> > there
> > > > > > >> > > > is.
> > > > > > >> > > > >>
> > > > > > >> > > > >> Yes. I think we would gate usage of this behind an
> IBP
> > > bump.
> > > > > Does
> > > > > > >> > that
> > > > > > >> > > > >> seem
> > > > > > >> > > > >> reasonable?
> > > > > > >> > > > >>
> > > > > > >> > > > >> > As for the improvements -- can you clarify how the
> > > multiple
> > > > > > >> > > > >> transactional
> > > > > > >> > > > >> IDs would help here? Were you thinking of a case
> where we
> > > > > > >> wait/batch
> > > > > > >> > > > >> multiple produce requests together? My understanding
> for
> > > now
> > > > > was
> > > > > > >> 1
> > > > > > >> > > > >> transactional ID and one validation per 1 produce
> > > request.
> > > > > > >> > > > >>
> > > > > > >> > > > >> Each call to `AddPartitionsToTxn` is essentially a
> write
> > > to
> > > > > the
> > > > > > >> > > > >> transaction
> > > > > > >> > > > >> log and must block on replication. The more we can
> fit
> > > into a
> > > > > > >> single
> > > > > > >> > > > >> request, the more writes we can do in parallel. The
> > > > > alternative
> > > > > > >> is
> > > > > > >> > to
> > > > > > >> > > > make
> > > > > > >> > > > >> use of more connections, but usually we prefer
> batching
> > > > > since the
> > > > > > >> > > > network
> > > > > > >> > > > >> stack is not really optimized for high
> connection/request
> > > > > loads.
> > > > > > >> > > > >>
> > > > > > >> > > > >> > Finally with respect to the authorizations, I
> think it
> > > > > makes
> > > > > > >> sense
> > > > > > >> > > to
> > > > > > >> > > > >> skip
> > > > > > >> > > > >> topic authorizations, but I'm a bit confused by the
> > > "leader
> > > > > ID"
> > > > > > >> > field.
> > > > > > >> > > > >> Wouldn't we just want to flag the request as from a
> > > broker
> > > > > (does
> > > > > > >> it
> > > > > > >> > > > matter
> > > > > > >> > > > >> which one?).
> > > > > > >> > > > >>
> > > > > > >> > > > >> We could also make it version-based. For the next
> > > version, we
> > > > > > >> could
> > > > > > >> > > > >> require
> > > > > > >> > > > >> CLUSTER auth. So clients would not be able to use
> the API
> > > > > > >> anymore,
> > > > > > >> > > which
> > > > > > >> > > > >> is
> > > > > > >> > > > >> probably what we want.
> > > > > > >> > > > >>
> > > > > > >> > > > >> -Jason
> > > > > > >> > > > >>
> > > > > > >> > > > >> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> > > > > > >> > > > >> <jo...@confluent.io.invalid>
> > > > > > >> > > > >> wrote:
> > > > > > >> > > > >>
> > > > > > >> > > > >> > As a follow up, I was just thinking about the
> batching
> > > a
> > > > > bit
> > > > > > >> more.
> > > > > > >> > > > >> > I suppose if we have one request in flight and we
> > > queue up
> > > > > the
> > > > > > >> > other
> > > > > > >> > > > >> > produce requests in some sort of purgatory, we
> could
> > > send
> > > > > > >> > > information
> > > > > > >> > > > >> out
> > > > > > >> > > > >> > for all of them rather than one by one. So that
> would
> > > be a
> > > > > > >> benefit
> > > > > > >> > > of
> > > > > > >> > > > >> > batching partitions to add per transaction.
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > I'll need to think a bit more on the design of this
> > > part
> > > > > of the
> > > > > > >> > KIP,
> > > > > > >> > > > and
> > > > > > >> > > > >> > will update the KIP in the next few days.
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > Thanks,
> > > > > > >> > > > >> > Justine
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan <
> > > > > > >> > > jolshan@confluent.io>
> > > > > > >> > > > >> > wrote:
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > > Hey Jason -- thanks for the input -- I was just
> > > digging
> > > > > a bit
> > > > > > >> > > deeper
> > > > > > >> > > > >> into
> > > > > > >> > > > >> > > the design + implementation of the validation
> calls
> > > here
> > > > > and
> > > > > > >> > what
> > > > > > >> > > > you
> > > > > > >> > > > >> say
> > > > > > >> > > > >> > > makes sense.
> > > > > > >> > > > >> > >
> > > > > > >> > > > >> > > I was wondering about compatibility here. When we
> > > send
> > > > > > >> requests
> > > > > > >> > > > >> > > between brokers, we want to ensure that the
> receiving
> > > > > broker
> > > > > > >> > > > >> understands
> > > > > > >> > > > >> > > the request (specifically the new fields).
> Typically
> > > > > this is
> > > > > > >> > done
> > > > > > >> > > > via
> > > > > > >> > > > >> > > IBP/metadata version.
> > > > > > >> > > > >> > > I'm trying to think if there is a way around it
> but
> > > I'm
> > > > > not
> > > > > > >> sure
> > > > > > >> > > > there
> > > > > > >> > > > >> > is.
> > > > > > >> > > > >> > >
> > > > > > >> > > > >> > > As for the improvements -- can you clarify how
> the
> > > > > multiple
> > > > > > >> > > > >> transactional
> > > > > > >> > > > >> > > IDs would help here? Were you thinking of a case
> > > where we
> > > > > > >> > > wait/batch
> > > > > > >> > > > >> > > multiple produce requests together? My
> understanding
> > > for
> > > > > now
> > > > > > >> > was 1
> > > > > > >> > > > >> > > transactional ID and one validation per 1 produce
> > > > > request.
> > > > > > >> > > > >> > >
> > > > > > >> > > > >> > > Finally with respect to the authorizations, I
> think
> > > it
> > > > > makes
> > > > > > >> > sense
> > > > > > >> > > > to
> > > > > > >> > > > >> > skip
> > > > > > >> > > > >> > > topic authorizations, but I'm a bit confused by
> the
> > > > > "leader
> > > > > > >> ID"
> > > > > > >> > > > field.
> > > > > > >> > > > >> > > Wouldn't we just want to flag the request as
> from a
> > > > > broker
> > > > > > >> (does
> > > > > > >> > > it
> > > > > > >> > > > >> > matter
> > > > > > >> > > > >> > > which one?).
> > > > > > >> > > > >> > >
> > > > > > >> > > > >> > > I think I want to adopt these suggestions, just
> had
> > > a few
> > > > > > >> > > questions
> > > > > > >> > > > on
> > > > > > >> > > > >> > the
> > > > > > >> > > > >> > > details.
> > > > > > >> > > > >> > >
> > > > > > >> > > > >> > > Thanks,
> > > > > > >> > > > >> > > Justine
> > > > > > >> > > > >> > >
> > > > > > >> > > > >> > > On Thu, Jan 5, 2023 at 5:05 PM Jason Gustafson
> > > > > > >> > > > >> > <ja...@confluent.io.invalid>
> > > > > > >> > > > >> > > wrote:
> > > > > > >> > > > >> > >
> > > > > > >> > > > >> > >> Hi Justine,
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> > >> Thanks for the proposal.
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> > >> I was thinking about the implementation a little
> > > bit.
> > > > > In the
> > > > > > >> > > > current
> > > > > > >> > > > >> > >> proposal, the behavior depends on whether we
> have an
> > > > > old or
> > > > > > >> new
> > > > > > >> > > > >> client.
> > > > > > >> > > > >> > >> For
> > > > > > >> > > > >> > >> old clients, we send `DescribeTransactions` and
> > > verify
> > > > > the
> > > > > > >> > result
> > > > > > >> > > > and
> > > > > > >> > > > >> > for
> > > > > > >> > > > >> > >> new clients, we send `AddPartitionsToTxn`. We
> might
> > > be
> > > > > able
> > > > > > >> to
> > > > > > >> > > > >> simplify
> > > > > > >> > > > >> > >> the
> > > > > > >> > > > >> > >> implementation if we can use the same request
> type.
> > > For
> > > > > > >> > example,
> > > > > > >> > > > >> what if
> > > > > > >> > > > >> > >> we
> > > > > > >> > > > >> > >> bump the protocol version for
> `AddPartitionsToTxn`
> > > and
> > > > > add a
> > > > > > >> > > > >> > >> `validateOnly`
> > > > > > >> > > > >> > >> flag? For older versions, we can set
> > > > > `validateOnly=true` so
> > > > > > >> > that
> > > > > > >> > > > the
> > > > > > >> > > > >> > >> request only returns successfully if the
> partition
> > > had
> > > > > > >> already
> > > > > > >> > > been
> > > > > > >> > > > >> > added.
> > > > > > >> > > > >> > >> For new versions, we can set
> `validateOnly=false`
> > > and
> > > > > the
> > > > > > >> > > partition
> > > > > > >> > > > >> will
> > > > > > >> > > > >> > >> be
> > > > > > >> > > > >> > >> added to the transaction. The other slightly
> > > annoying
> > > > > thing
> > > > > > >> > that
> > > > > > >> > > > this
> > > > > > >> > > > >> > >> would
> > > > > > >> > > > >> > >> get around is the need to collect the
> transaction
> > > state
> > > > > for
> > > > > > >> all
> > > > > > >> > > > >> > partitions
> > > > > > >> > > > >> > >> even when we only care about a subset.
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> > >> Some additional improvements to consider:
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> > >> - We can give `AddPartitionsToTxn` better batch
> > > support
> > > > > for
> > > > > > >> > > > >> inter-broker
> > > > > > >> > > > >> > >> usage. Currently we only allow one
> > > `TransactionalId` to
> > > > > be
> > > > > > >> > > > specified,
> > > > > > >> > > > >> > but
> > > > > > >> > > > >> > >> the broker may get some benefit being able to
> batch
> > > > > across
> > > > > > >> > > multiple
> > > > > > >> > > > >> > >> transactions.
> > > > > > >> > > > >> > >> - Another small improvement is skipping topic
> > > > > authorization
> > > > > > >> > > checks
> > > > > > >> > > > >> for
> > > > > > >> > > > >> > >> `AddPartitionsToTxn` when the request is from a
> > > broker.
> > > > > > >> Perhaps
> > > > > > >> > > we
> > > > > > >> > > > >> can
> > > > > > >> > > > >> > add
> > > > > > >> > > > >> > >> a field for the `LeaderId` or something like
> that
> > > and
> > > > > > >> require
> > > > > > >> > > > CLUSTER
> > > > > > >> > > > >> > >> permission when set.
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> > >> Best,
> > > > > > >> > > > >> > >> Jason
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> > >> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
> > > > > > >> > <jun@confluent.io.invalid
> > > > > > >> > > >
> > > > > > >> > > > >> > wrote:
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> > >> > Hi, Justine,
> > > > > > >> > > > >> > >> >
> > > > > > >> > > > >> > >> > Thanks for the explanation. It makes sense to
> me
> > > now.
> > > > > > >> > > > >> > >> >
> > > > > > >> > > > >> > >> > Jun
> > > > > > >> > > > >> > >> >
> > > > > > >> > > > >> > >> > On Mon, Dec 19, 2022 at 1:42 PM Justine Olshan
> > > > > > >> > > > >> > >> > <jo...@confluent.io.invalid>
> > > > > > >> > > > >> > >> > wrote:
> > > > > > >> > > > >> > >> >
> > > > > > >> > > > >> > >> > > Hi Jun,
> > > > > > >> > > > >> > >> > >
> > > > > > >> > > > >> > >> > > My understanding of the mechanism is that
> when
> > > we
> > > > > get to
> > > > > > >> > the
> > > > > > >> > > > last
> > > > > > >> > > > >> > >> epoch,
> > > > > > >> > > > >> > >> > we
> > > > > > >> > > > >> > >> > > increment to the fencing/last epoch and if
> any
> > > > > further
> > > > > > >> > > requests
> > > > > > >> > > > >> come
> > > > > > >> > > > >> > >> in
> > > > > > >> > > > >> > >> > for
> > > > > > >> > > > >> > >> > > this producer ID they are fenced. Then the
> > > producer
> > > > > > >> gets a
> > > > > > >> > > new
> > > > > > >> > > > ID
> > > > > > >> > > > >> > and
> > > > > > >> > > > >> > >> > > restarts with epoch/sequence 0. The fenced
> epoch
> > > > > sticks
> > > > > > >> > > around
> > > > > > >> > > > >> for
> > > > > > >> > > > >> > the
> > > > > > >> > > > >> > >> > > duration of producer.id.expiration.ms and
> > > blocks
> > > > > any
> > > > > > >> late
> > > > > > >> > > > >> messages
> > > > > > >> > > > >> > >> > there.
> > > > > > >> > > > >> > >> > > The new ID will get to take advantage of the
> > > > > improved
> > > > > > >> > > semantics
> > > > > > >> > > > >> > around
> > > > > > >> > > > >> > >> > > non-zero start sequences. So I think we are
> > > covered.
> > > > > > >> > > > >> > >> > >
> > > > > > >> > > > >> > >> > > The only potential issue is overloading the
> > > cache,
> > > > > but
> > > > > > >> > > > hopefully
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> > > improvements (lowered
> producer.id.expiration.ms
> > > )
> > > > > will
> > > > > > >> help
> > > > > > >> > > > with
> > > > > > >> > > > >> > that.
> > > > > > >> > > > >> > >> > Let
> > > > > > >> > > > >> > >> > > me know if you still have concerns.
> > > > > > >> > > > >> > >> > >
> > > > > > >> > > > >> > >> > > Thanks,
> > > > > > >> > > > >> > >> > > Justine
> > > > > > >> > > > >> > >> > >
> > > > > > >> > > > >> > >> > > On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
> > > > > > >> > > > >> <ju...@confluent.io.invalid>
> > > > > > >> > > > >> > >> > wrote:
> > > > > > >> > > > >> > >> > >
> > > > > > >> > > > >> > >> > > > Hi, Justine,
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > > >> > >> > > > Thanks for the explanation.
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > > >> > >> > > > 70. The proposed fencing logic doesn't
> apply
> > > when
> > > > > pid
> > > > > > >> > > > changes,
> > > > > > >> > > > >> is
> > > > > > >> > > > >> > >> that
> > > > > > >> > > > >> > >> > > > right? If so, I am not sure how complete
> we
> > > are
> > > > > > >> > addressing
> > > > > > >> > > > this
> > > > > > >> > > > >> > >> issue
> > > > > > >> > > > >> > >> > if
> > > > > > >> > > > >> > >> > > > the pid changes more frequently.
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > > >> > >> > > > Thanks,
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > > >> > >> > > > Jun
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > > >> > >> > > > On Fri, Dec 16, 2022 at 9:16 AM Justine
> Olshan
> > > > > > >> > > > >> > >> > > > <jo...@confluent.io.invalid>
> > > > > > >> > > > >> > >> > > > wrote:
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > > >> > >> > > > > Hi Jun,
> > > > > > >> > > > >> > >> > > > >
> > > > > > >> > > > >> > >> > > > > Thanks for replying!
> > > > > > >> > > > >> > >> > > > >
> > > > > > >> > > > >> > >> > > > > 70.We already do the overflow
> mechanism, so
> > > my
> > > > > > >> change
> > > > > > >> > > would
> > > > > > >> > > > >> just
> > > > > > >> > > > >> > >> make
> > > > > > >> > > > >> > >> > > it
> > > > > > >> > > > >> > >> > > > > happen more often.
> > > > > > >> > > > >> > >> > > > > I was also not suggesting a new field
> in the
> > > > > log,
> > > > > > >> but
> > > > > > >> > in
> > > > > > >> > > > the
> > > > > > >> > > > >> > >> > response,
> > > > > > >> > > > >> > >> > > > > which would be gated by the client
> version.
> > > > > Sorry if
> > > > > > >> > > > >> something
> > > > > > >> > > > >> > >> there
> > > > > > >> > > > >> > >> > is
> > > > > > >> > > > >> > >> > > > > unclear. I think we are starting to
> diverge.
> > > > > > >> > > > >> > >> > > > > The goal of this KIP is to not change
> to the
> > > > > marker
> > > > > > >> > > format
> > > > > > >> > > > at
> > > > > > >> > > > >> > all.
> > > > > > >> > > > >> > >> > > > >
> > > > > > >> > > > >> > >> > > > > 71. Yes, I guess I was going under the
> > > > > assumption
> > > > > > >> that
> > > > > > >> > > the
> > > > > > >> > > > >> log
> > > > > > >> > > > >> > >> would
> > > > > > >> > > > >> > >> > > just
> > > > > > >> > > > >> > >> > > > > look at its last epoch and treat it as
> the
> > > > > current
> > > > > > >> > > epoch. I
> > > > > > >> > > > >> > >> suppose
> > > > > > >> > > > >> > >> > we
> > > > > > >> > > > >> > >> > > > can
> > > > > > >> > > > >> > >> > > > > have some special logic that if the last
> > > epoch
> > > > > was
> > > > > > >> on a
> > > > > > >> > > > >> marker
> > > > > > >> > > > >> > we
> > > > > > >> > > > >> > >> > > > actually
> > > > > > >> > > > >> > >> > > > > expect the next epoch or something like
> > > that. We
> > > > > > >> just
> > > > > > >> > > need
> > > > > > >> > > > to
> > > > > > >> > > > >> > >> > > distinguish
> > > > > > >> > > > >> > >> > > > > based on whether we had a commit/abort
> > > marker.
> > > > > > >> > > > >> > >> > > > >
> > > > > > >> > > > >> > >> > > > > 72.
> > > > > > >> > > > >> > >> > > > > > if the producer epoch hasn't been
> bumped
> > > on
> > > > > the
> > > > > > >> > > > >> > >> > > > > broker, it seems that the stucked
> message
> > > will
> > > > > fail
> > > > > > >> the
> > > > > > >> > > > >> sequence
> > > > > > >> > > > >> > >> > > > validation
> > > > > > >> > > > >> > >> > > > > and will be ignored. If the producer
> epoch
> > > has
> > > > > been
> > > > > > >> > > bumped,
> > > > > > >> > > > >> we
> > > > > > >> > > > >> > >> ignore
> > > > > > >> > > > >> > >> > > the
> > > > > > >> > > > >> > >> > > > > sequence check and the stuck message
> could
> > > be
> > > > > > >> appended
> > > > > > >> > to
> > > > > > >> > > > the
> > > > > > >> > > > >> > log.
> > > > > > >> > > > >> > >> > So,
> > > > > > >> > > > >> > >> > > is
> > > > > > >> > > > >> > >> > > > > the latter case that we want to guard?
> > > > > > >> > > > >> > >> > > > >
> > > > > > >> > > > >> > >> > > > > I'm not sure I follow that "the message
> will
> > > > > fail
> > > > > > >> the
> > > > > > >> > > > >> sequence
> > > > > > >> > > > >> > >> > > > validation".
> > > > > > >> > > > >> > >> > > > > In some of these cases, we had an abort
> > > marker
> > > > > (due
> > > > > > >> to
> > > > > > >> > an
> > > > > > >> > > > >> error)
> > > > > > >> > > > >> > >> and
> > > > > > >> > > > >> > >> > > then
> > > > > > >> > > > >> > >> > > > > the late message comes in with the
> correct
> > > > > sequence
> > > > > > >> > > number.
> > > > > > >> > > > >> This
> > > > > > >> > > > >> > >> is a
> > > > > > >> > > > >> > >> > > > case
> > > > > > >> > > > >> > >> > > > > covered by the KIP.
> > > > > > >> > > > >> > >> > > > > The latter case is actually not
> something
> > > we've
> > > > > > >> > > considered
> > > > > > >> > > > >> > here. I
> > > > > > >> > > > >> > >> > > think
> > > > > > >> > > > >> > >> > > > > generally when we bump the epoch, we are
> > > > > accepting
> > > > > > >> that
> > > > > > >> > > the
> > > > > > >> > > > >> > >> sequence
> > > > > > >> > > > >> > >> > > does
> > > > > > >> > > > >> > >> > > > > not need to be checked anymore. My
> > > > > understanding is
> > > > > > >> > also
> > > > > > >> > > > >> that we
> > > > > > >> > > > >> > >> > don't
> > > > > > >> > > > >> > >> > > > > typically bump epoch mid transaction
> (based
> > > on a
> > > > > > >> quick
> > > > > > >> > > look
> > > > > > >> > > > >> at
> > > > > > >> > > > >> > the
> > > > > > >> > > > >> > >> > > code)
> > > > > > >> > > > >> > >> > > > > but let me know if that is the case.
> > > > > > >> > > > >> > >> > > > >
> > > > > > >> > > > >> > >> > > > > Thanks,
> > > > > > >> > > > >> > >> > > > > Justine
> > > > > > >> > > > >> > >> > > > >
> > > > > > >> > > > >> > >> > > > > On Thu, Dec 15, 2022 at 12:23 PM Jun Rao
> > > > > > >> > > > >> > <jun@confluent.io.invalid
> > > > > > >> > > > >> > >> >
> > > > > > >> > > > >> > >> > > > wrote:
> > > > > > >> > > > >> > >> > > > >
> > > > > > >> > > > >> > >> > > > > > Hi, Justine,
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> > > > >> > >> > > > > > Thanks for the reply.
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> > > > >> > >> > > > > > 70. Assigning a new pid on int
> overflow
> > > seems
> > > > > a
> > > > > > >> bit
> > > > > > >> > > > hacky.
> > > > > > >> > > > >> If
> > > > > > >> > > > >> > we
> > > > > > >> > > > >> > >> > > need a
> > > > > > >> > > > >> > >> > > > > txn
> > > > > > >> > > > >> > >> > > > > > level id, it will be better to model
> this
> > > > > > >> explicitly.
> > > > > > >> > > > >> Adding a
> > > > > > >> > > > >> > >> new
> > > > > > >> > > > >> > >> > > > field
> > > > > > >> > > > >> > >> > > > > > would require a bit more work since it
> > > > > requires a
> > > > > > >> new
> > > > > > >> > > txn
> > > > > > >> > > > >> > marker
> > > > > > >> > > > >> > >> > > format
> > > > > > >> > > > >> > >> > > > > in
> > > > > > >> > > > >> > >> > > > > > the log. So, we probably need to
> guard it
> > > > > with an
> > > > > > >> IBP
> > > > > > >> > > or
> > > > > > >> > > > >> > >> metadata
> > > > > > >> > > > >> > >> > > > version
> > > > > > >> > > > >> > >> > > > > > and document the impact on downgrade
> once
> > > the
> > > > > new
> > > > > > >> > > format
> > > > > > >> > > > is
> > > > > > >> > > > >> > >> written
> > > > > > >> > > > >> > >> > > to
> > > > > > >> > > > >> > >> > > > > the
> > > > > > >> > > > >> > >> > > > > > log.
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> > > > >> > >> > > > > > 71. Hmm, once the marker is written,
> the
> > > > > partition
> > > > > > >> > will
> > > > > > >> > > > >> expect
> > > > > > >> > > > >> > >> the
> > > > > > >> > > > >> > >> > > next
> > > > > > >> > > > >> > >> > > > > > append to be on the next epoch. Does
> that
> > > > > cover
> > > > > > >> the
> > > > > > >> > > case
> > > > > > >> > > > >> you
> > > > > > >> > > > >> > >> > > mentioned?
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> > > > >> > >> > > > > > 72. Also, just to be clear on the
> stucked
> > > > > message
> > > > > > >> > issue
> > > > > > >> > > > >> > >> described
> > > > > > >> > > > >> > >> > in
> > > > > > >> > > > >> > >> > > > the
> > > > > > >> > > > >> > >> > > > > > motivation. With EoS, we also
> validate the
> > > > > > >> sequence
> > > > > > >> > id
> > > > > > >> > > > for
> > > > > > >> > > > >> > >> > > idempotency.
> > > > > > >> > > > >> > >> > > > > So,
> > > > > > >> > > > >> > >> > > > > > with the current logic, if the
> producer
> > > epoch
> > > > > > >> hasn't
> > > > > > >> > > been
> > > > > > >> > > > >> > >> bumped on
> > > > > > >> > > > >> > >> > > the
> > > > > > >> > > > >> > >> > > > > > broker, it seems that the stucked
> message
> > > will
> > > > > > >> fail
> > > > > > >> > the
> > > > > > >> > > > >> > sequence
> > > > > > >> > > > >> > >> > > > > validation
> > > > > > >> > > > >> > >> > > > > > and will be ignored. If the producer
> > > epoch has
> > > > > > >> been
> > > > > > >> > > > >> bumped, we
> > > > > > >> > > > >> > >> > ignore
> > > > > > >> > > > >> > >> > > > the
> > > > > > >> > > > >> > >> > > > > > sequence check and the stuck message
> > > could be
> > > > > > >> > appended
> > > > > > >> > > to
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> log.
> > > > > > >> > > > >> > >> > > So,
> > > > > > >> > > > >> > >> > > > is
> > > > > > >> > > > >> > >> > > > > > the latter case that we want to guard?
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> > > > >> > >> > > > > > Thanks,
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> > > > >> > >> > > > > > Jun
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> > > > >> > >> > > > > > On Wed, Dec 14, 2022 at 10:44 AM
> Justine
> > > > > Olshan
> > > > > > >> > > > >> > >> > > > > > <jo...@confluent.io.invalid> wrote:
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> > > > >> > >> > > > > > > Matthias — thanks again for taking
> time
> > > to
> > > > > look
> > > > > > >> a
> > > > > > >> > > this.
> > > > > > >> > > > >> You
> > > > > > >> > > > >> > >> said:
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > My proposal was only focusing to
> avoid
> > > > > > >> dangling
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > transactions if records are added
> > > without
> > > > > > >> > registered
> > > > > > >> > > > >> > >> partition.
> > > > > > >> > > > >> > >> > --
> > > > > > >> > > > >> > >> > > > > Maybe
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > you can add a few more details to
> the
> > > KIP
> > > > > about
> > > > > > >> > this
> > > > > > >> > > > >> > scenario
> > > > > > >> > > > >> > >> for
> > > > > > >> > > > >> > >> > > > > better
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > documentation purpose?
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > I'm not sure I understand what you
> mean
> > > > > here.
> > > > > > >> The
> > > > > > >> > > > >> motivation
> > > > > > >> > > > >> > >> > > section
> > > > > > >> > > > >> > >> > > > > > > describes two scenarios about how
> the
> > > record
> > > > > > >> can be
> > > > > > >> > > > added
> > > > > > >> > > > >> > >> > without a
> > > > > > >> > > > >> > >> > > > > > > registered partition:
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > 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.
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > Another way hanging transactions
> can
> > > > > occur is
> > > > > > >> > that
> > > > > > >> > > a
> > > > > > >> > > > >> > client
> > > > > > >> > > > >> > >> is
> > > > > > >> > > > >> > >> > > > buggy
> > > > > > >> > > > >> > >> > > > > > and
> > > > > > >> > > > >> > >> > > > > > > may somehow try to write to a
> partition
> > > > > before
> > > > > > >> it
> > > > > > >> > > adds
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> > > partition
> > > > > > >> > > > >> > >> > > > to
> > > > > > >> > > > >> > >> > > > > > the
> > > > > > >> > > > >> > >> > > > > > > transaction.
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > For the first example of this would
> it
> > > be
> > > > > > >> helpful
> > > > > > >> > to
> > > > > > >> > > > say
> > > > > > >> > > > >> > that
> > > > > > >> > > > >> > >> > this
> > > > > > >> > > > >> > >> > > > > > message
> > > > > > >> > > > >> > >> > > > > > > comes in after the abort, but
> before the
> > > > > > >> partition
> > > > > > >> > is
> > > > > > >> > > > >> added
> > > > > > >> > > > >> > to
> > > > > > >> > > > >> > >> > the
> > > > > > >> > > > >> > >> > > > next
> > > > > > >> > > > >> > >> > > > > > > transaction so it becomes "hanging."
> > > > > Perhaps the
> > > > > > >> > next
> > > > > > >> > > > >> > sentence
> > > > > > >> > > > >> > >> > > > > describing
> > > > > > >> > > > >> > >> > > > > > > the message becoming part of the
> next
> > > > > > >> transaction
> > > > > > >> > (a
> > > > > > >> > > > >> > different
> > > > > > >> > > > >> > >> > > case)
> > > > > > >> > > > >> > >> > > > > was
> > > > > > >> > > > >> > >> > > > > > > not properly differentiated.
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > Jun — thanks for reading the KIP.
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > 70. The int typing was a concern.
> > > Currently
> > > > > we
> > > > > > >> > have a
> > > > > > >> > > > >> > >> mechanism
> > > > > > >> > > > >> > >> > in
> > > > > > >> > > > >> > >> > > > > place
> > > > > > >> > > > >> > >> > > > > > to
> > > > > > >> > > > >> > >> > > > > > > fence the final epoch when the
> epoch is
> > > > > about to
> > > > > > >> > > > overflow
> > > > > > >> > > > >> > and
> > > > > > >> > > > >> > >> > > assign
> > > > > > >> > > > >> > >> > > > a
> > > > > > >> > > > >> > >> > > > > > new
> > > > > > >> > > > >> > >> > > > > > > producer ID with epoch 0. Of course,
> > > this
> > > > > is a
> > > > > > >> bit
> > > > > > >> > > > tricky
> > > > > > >> > > > >> > >> when it
> > > > > > >> > > > >> > >> > > > comes
> > > > > > >> > > > >> > >> > > > > > to
> > > > > > >> > > > >> > >> > > > > > > the response back to the client.
> > > > > > >> > > > >> > >> > > > > > > Making this a long could be another
> > > option,
> > > > > but
> > > > > > >> I
> > > > > > >> > > > wonder
> > > > > > >> > > > >> are
> > > > > > >> > > > >> > >> > there
> > > > > > >> > > > >> > >> > > > any
> > > > > > >> > > > >> > >> > > > > > > implications on changing this field
> if
> > > the
> > > > > > >> epoch is
> > > > > > >> > > > >> > persisted
> > > > > > >> > > > >> > >> to
> > > > > > >> > > > >> > >> > > > disk?
> > > > > > >> > > > >> > >> > > > > > I'd
> > > > > > >> > > > >> > >> > > > > > > need to check the usages.
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > 71.This was something Matthias asked
> > > about
> > > > > as
> > > > > > >> > well. I
> > > > > > >> > > > was
> > > > > > >> > > > >> > >> > > > considering a
> > > > > > >> > > > >> > >> > > > > > > possible edge case where a produce
> > > request
> > > > > from
> > > > > > >> a
> > > > > > >> > new
> > > > > > >> > > > >> > >> transaction
> > > > > > >> > > > >> > >> > > > > somehow
> > > > > > >> > > > >> > >> > > > > > > gets sent right after the marker is
> > > > > written, but
> > > > > > >> > > before
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> > > producer
> > > > > > >> > > > >> > >> > > > is
> > > > > > >> > > > >> > >> > > > > > > alerted of the newly bumped epoch.
> In
> > > this
> > > > > > >> case, we
> > > > > > >> > > may
> > > > > > >> > > > >> > >> include
> > > > > > >> > > > >> > >> > > this
> > > > > > >> > > > >> > >> > > > > > record
> > > > > > >> > > > >> > >> > > > > > > when we don't want to. I suppose we
> > > could
> > > > > try
> > > > > > >> to do
> > > > > > >> > > > >> > something
> > > > > > >> > > > >> > >> > > client
> > > > > > >> > > > >> > >> > > > > side
> > > > > > >> > > > >> > >> > > > > > > to bump the epoch after sending an
> > > endTxn as
> > > > > > >> well
> > > > > > >> > in
> > > > > > >> > > > this
> > > > > > >> > > > >> > >> > scenario
> > > > > > >> > > > >> > >> > > —
> > > > > > >> > > > >> > >> > > > > but
> > > > > > >> > > > >> > >> > > > > > I
> > > > > > >> > > > >> > >> > > > > > > wonder how it would work when the
> > > server is
> > > > > > >> > aborting
> > > > > > >> > > > >> based
> > > > > > >> > > > >> > on
> > > > > > >> > > > >> > >> a
> > > > > > >> > > > >> > >> > > > > > server-side
> > > > > > >> > > > >> > >> > > > > > > error. I could also be missing
> > > something and
> > > > > > >> this
> > > > > > >> > > > >> scenario
> > > > > > >> > > > >> > is
> > > > > > >> > > > >> > >> > > > actually
> > > > > > >> > > > >> > >> > > > > > not
> > > > > > >> > > > >> > >> > > > > > > possible.
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > Thanks again to everyone reading and
> > > > > commenting.
> > > > > > >> > Let
> > > > > > >> > > me
> > > > > > >> > > > >> know
> > > > > > >> > > > >> > >> > about
> > > > > > >> > > > >> > >> > > > any
> > > > > > >> > > > >> > >> > > > > > > further questions or comments.
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > Justine
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > On Wed, Dec 14, 2022 at 9:41 AM Jun
> Rao
> > > > > > >> > > > >> > >> <jun@confluent.io.invalid
> > > > > > >> > > > >> > >> > >
> > > > > > >> > > > >> > >> > > > > > wrote:
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > Hi, Justine,
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > Thanks for the KIP. A couple of
> > > comments.
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > 70. Currently, the producer epoch
> is
> > > an
> > > > > int.
> > > > > > >> I am
> > > > > > >> > > not
> > > > > > >> > > > >> sure
> > > > > > >> > > > >> > >> if
> > > > > > >> > > > >> > >> > > it's
> > > > > > >> > > > >> > >> > > > > > enough
> > > > > > >> > > > >> > >> > > > > > > > to accommodate all transactions
> in the
> > > > > > >> lifetime
> > > > > > >> > of
> > > > > > >> > > a
> > > > > > >> > > > >> > >> producer.
> > > > > > >> > > > >> > >> > > > Should
> > > > > > >> > > > >> > >> > > > > > we
> > > > > > >> > > > >> > >> > > > > > > > change that to a long or add a new
> > > long
> > > > > field
> > > > > > >> > like
> > > > > > >> > > > >> txnId?
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > 71. "it will write the prepare
> commit
> > > > > message
> > > > > > >> > with
> > > > > > >> > > a
> > > > > > >> > > > >> > bumped
> > > > > > >> > > > >> > >> > epoch
> > > > > > >> > > > >> > >> > > > and
> > > > > > >> > > > >> > >> > > > > > > send
> > > > > > >> > > > >> > >> > > > > > > > WriteTxnMarkerRequests with the
> bumped
> > > > > epoch."
> > > > > > >> > Hmm,
> > > > > > >> > > > the
> > > > > > >> > > > >> > >> epoch
> > > > > > >> > > > >> > >> > is
> > > > > > >> > > > >> > >> > > > > > > associated
> > > > > > >> > > > >> > >> > > > > > > > with the current txn right? So, it
> > > seems
> > > > > > >> weird to
> > > > > > >> > > > >> write a
> > > > > > >> > > > >> > >> > commit
> > > > > > >> > > > >> > >> > > > > > message
> > > > > > >> > > > >> > >> > > > > > > > with a bumped epoch. Should we
> only
> > > bump
> > > > > up
> > > > > > >> the
> > > > > > >> > > epoch
> > > > > > >> > > > >> in
> > > > > > >> > > > >> > >> > > > > EndTxnResponse
> > > > > > >> > > > >> > >> > > > > > > and
> > > > > > >> > > > >> > >> > > > > > > > rename the field to sth like
> > > > > > >> nextProducerEpoch?
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > Thanks,
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > Jun
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > On Mon, Dec 12, 2022 at 8:54 PM
> > > Matthias
> > > > > J.
> > > > > > >> Sax <
> > > > > > >> > > > >> > >> > > mjsax@apache.org>
> > > > > > >> > > > >> > >> > > > > > > wrote:
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > Thanks for the background.
> > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > 20/30: SGTM. My proposal was
> only
> > > > > focusing
> > > > > > >> to
> > > > > > >> > > avoid
> > > > > > >> > > > >> > >> dangling
> > > > > > >> > > > >> > >> > > > > > > > > transactions if records are
> added
> > > > > without
> > > > > > >> > > > registered
> > > > > > >> > > > >> > >> > partition.
> > > > > > >> > > > >> > >> > > > --
> > > > > > >> > > > >> > >> > > > > > > Maybe
> > > > > > >> > > > >> > >> > > > > > > > > you can add a few more details
> to
> > > the
> > > > > KIP
> > > > > > >> about
> > > > > > >> > > > this
> > > > > > >> > > > >> > >> scenario
> > > > > > >> > > > >> > >> > > for
> > > > > > >> > > > >> > >> > > > > > > better
> > > > > > >> > > > >> > >> > > > > > > > > documentation purpose?
> > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > 40: I think you hit a fair point
> > > about
> > > > > race
> > > > > > >> > > > >> conditions
> > > > > > >> > > > >> > or
> > > > > > >> > > > >> > >> > > client
> > > > > > >> > > > >> > >> > > > > bugs
> > > > > > >> > > > >> > >> > > > > > > > > (incorrectly not bumping the
> > > epoch). The
> > > > > > >> > > > >> > >> complexity/confusion
> > > > > > >> > > > >> > >> > > for
> > > > > > >> > > > >> > >> > > > > > using
> > > > > > >> > > > >> > >> > > > > > > > > the bumped epoch I see, is
> mainly
> > > for
> > > > > > >> internal
> > > > > > >> > > > >> > debugging,
> > > > > > >> > > > >> > >> ie,
> > > > > > >> > > > >> > >> > > > > > > inspecting
> > > > > > >> > > > >> > >> > > > > > > > > log segment dumps -- it seems
> > > harder to
> > > > > > >> reason
> > > > > > >> > > > about
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> > system
> > > > > > >> > > > >> > >> > > > for
> > > > > > >> > > > >> > >> > > > > > us
> > > > > > >> > > > >> > >> > > > > > > > > humans. But if we get better
> > > > > guarantees, it
> > > > > > >> > would
> > > > > > >> > > > be
> > > > > > >> > > > >> > >> worth to
> > > > > > >> > > > >> > >> > > use
> > > > > > >> > > > >> > >> > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > bumped epoch.
> > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > 60: as I mentioned already, I
> don't
> > > > > know the
> > > > > > >> > > broker
> > > > > > >> > > > >> > >> internals
> > > > > > >> > > > >> > >> > > to
> > > > > > >> > > > >> > >> > > > > > > provide
> > > > > > >> > > > >> > >> > > > > > > > > more input. So if nobody else
> chimes
> > > > > in, we
> > > > > > >> > > should
> > > > > > >> > > > >> just
> > > > > > >> > > > >> > >> move
> > > > > > >> > > > >> > >> > > > > forward
> > > > > > >> > > > >> > >> > > > > > > > > with your proposal.
> > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > -Matthias
> > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > On 12/6/22 4:22 PM, Justine
> Olshan
> > > > > wrote:
> > > > > > >> > > > >> > >> > > > > > > > > > Hi all,
> > > > > > >> > > > >> > >> > > > > > > > > > After Artem's questions about
> > > error
> > > > > > >> behavior,
> > > > > > >> > > > I've
> > > > > > >> > > > >> > >> > > re-evaluated
> > > > > > >> > > > >> > >> > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > > unknown producer ID exception
> and
> > > had
> > > > > some
> > > > > > >> > > > >> discussions
> > > > > > >> > > > >> > >> > > offline.
> > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > > I think generally it makes
> sense
> > > to
> > > > > > >> simplify
> > > > > > >> > > > error
> > > > > > >> > > > >> > >> handling
> > > > > > >> > > > >> > >> > > in
> > > > > > >> > > > >> > >> > > > > > cases
> > > > > > >> > > > >> > >> > > > > > > > like
> > > > > > >> > > > >> > >> > > > > > > > > > this and the
> UNKNOWN_PRODUCER_ID
> > > error
> > > > > > >> has a
> > > > > > >> > > > pretty
> > > > > > >> > > > >> > long
> > > > > > >> > > > >> > >> > and
> > > > > > >> > > > >> > >> > > > > > > > complicated
> > > > > > >> > > > >> > >> > > > > > > > > > history. Because of this, I
> > > propose
> > > > > > >> adding a
> > > > > > >> > > new
> > > > > > >> > > > >> error
> > > > > > >> > > > >> > >> code
> > > > > > >> > > > >> > >> > > > > > > > > ABORTABLE_ERROR
> > > > > > >> > > > >> > >> > > > > > > > > > that when encountered by new
> > > clients
> > > > > > >> (gated
> > > > > > >> > by
> > > > > > >> > > > the
> > > > > > >> > > > >> > >> produce
> > > > > > >> > > > >> > >> > > > > request
> > > > > > >> > > > >> > >> > > > > > > > > version)
> > > > > > >> > > > >> > >> > > > > > > > > > will simply abort the
> transaction.
> > > > > This
> > > > > > >> > allows
> > > > > > >> > > > the
> > > > > > >> > > > >> > >> server
> > > > > > >> > > > >> > >> > to
> > > > > > >> > > > >> > >> > > > have
> > > > > > >> > > > >> > >> > > > > > > some
> > > > > > >> > > > >> > >> > > > > > > > > say
> > > > > > >> > > > >> > >> > > > > > > > > > in whether the client aborts
> and
> > > makes
> > > > > > >> > handling
> > > > > > >> > > > >> much
> > > > > > >> > > > >> > >> > simpler.
> > > > > > >> > > > >> > >> > > > In
> > > > > > >> > > > >> > >> > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > > future, we can also use this
> > > error in
> > > > > > >> other
> > > > > > >> > > > >> situations
> > > > > > >> > > > >> > >> > where
> > > > > > >> > > > >> > >> > > we
> > > > > > >> > > > >> > >> > > > > > want
> > > > > > >> > > > >> > >> > > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > > > abort the transactions. We can
> > > even
> > > > > use on
> > > > > > >> > > other
> > > > > > >> > > > >> apis.
> > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > > I've added this to the KIP.
> Let me
> > > > > know if
> > > > > > >> > > there
> > > > > > >> > > > >> are
> > > > > > >> > > > >> > any
> > > > > > >> > > > >> > >> > > > > questions
> > > > > > >> > > > >> > >> > > > > > or
> > > > > > >> > > > >> > >> > > > > > > > > > issues.
> > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > > Justine
> > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > > On Fri, Dec 2, 2022 at 10:22
> AM
> > > > > Justine
> > > > > > >> > Olshan
> > > > > > >> > > <
> > > > > > >> > > > >> > >> > > > > > jolshan@confluent.io
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > wrote:
> > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > >> Hey Matthias,
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >> 20/30 — Maybe I also didn't
> > > express
> > > > > > >> myself
> > > > > > >> > > > >> clearly.
> > > > > > >> > > > >> > For
> > > > > > >> > > > >> > >> > > older
> > > > > > >> > > > >> > >> > > > > > > clients
> > > > > > >> > > > >> > >> > > > > > > > we
> > > > > > >> > > > >> > >> > > > > > > > > >> don't have a way to
> distinguish
> > > > > between a
> > > > > > >> > > > previous
> > > > > > >> > > > >> > and
> > > > > > >> > > > >> > >> the
> > > > > > >> > > > >> > >> > > > > current
> > > > > > >> > > > >> > >> > > > > > > > > >> transaction since we don't
> have
> > > the
> > > > > epoch
> > > > > > >> > > bump.
> > > > > > >> > > > >> This
> > > > > > >> > > > >> > >> means
> > > > > > >> > > > >> > >> > > > that
> > > > > > >> > > > >> > >> > > > > a
> > > > > > >> > > > >> > >> > > > > > > late
> > > > > > >> > > > >> > >> > > > > > > > > >> message from the previous
> > > transaction
> > > > > > >> may be
> > > > > > >> > > > >> added to
> > > > > > >> > > > >> > >> the
> > > > > > >> > > > >> > >> > > new
> > > > > > >> > > > >> > >> > > > > one.
> > > > > > >> > > > >> > >> > > > > > > > With
> > > > > > >> > > > >> > >> > > > > > > > > >> older clients — we can't
> > > guarantee
> > > > > this
> > > > > > >> > won't
> > > > > > >> > > > >> happen
> > > > > > >> > > > >> > >> if we
> > > > > > >> > > > >> > >> > > > > already
> > > > > > >> > > > >> > >> > > > > > > > sent
> > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call (why
> we
> > > make
> > > > > > >> changes
> > > > > > >> > > for
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> newer
> > > > > > >> > > > >> > >> > > > > client)
> > > > > > >> > > > >> > >> > > > > > > but
> > > > > > >> > > > >> > >> > > > > > > > > we
> > > > > > >> > > > >> > >> > > > > > > > > >> can at least gate some by
> > > ensuring
> > > > > that
> > > > > > >> the
> > > > > > >> > > > >> partition
> > > > > > >> > > > >> > >> has
> > > > > > >> > > > >> > >> > > been
> > > > > > >> > > > >> > >> > > > > > added
> > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >> transaction. The rationale
> here
> > > is
> > > > > that
> > > > > > >> > there
> > > > > > >> > > > are
> > > > > > >> > > > >> > >> likely
> > > > > > >> > > > >> > >> > > LESS
> > > > > > >> > > > >> > >> > > > > late
> > > > > > >> > > > >> > >> > > > > > > > > arrivals
> > > > > > >> > > > >> > >> > > > > > > > > >> as time goes on, so hopefully
> > > most
> > > > > late
> > > > > > >> > > arrivals
> > > > > > >> > > > >> will
> > > > > > >> > > > >> > >> come
> > > > > > >> > > > >> > >> > > in
> > > > > > >> > > > >> > >> > > > > > BEFORE
> > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call.
> Those
> > > that
> > > > > > >> arrive
> > > > > > >> > > > before
> > > > > > >> > > > >> > will
> > > > > > >> > > > >> > >> be
> > > > > > >> > > > >> > >> > > > > properly
> > > > > > >> > > > >> > >> > > > > > > > gated
> > > > > > >> > > > >> > >> > > > > > > > > >> with the describeTransactions
> > > > > approach.
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >> If we take the approach you
> > > > > suggested,
> > > > > > >> ANY
> > > > > > >> > > late
> > > > > > >> > > > >> > arrival
> > > > > > >> > > > >> > >> > > from a
> > > > > > >> > > > >> > >> > > > > > > > previous
> > > > > > >> > > > >> > >> > > > > > > > > >> transaction will be added.
> And we
> > > > > don't
> > > > > > >> want
> > > > > > >> > > > >> that. I
> > > > > > >> > > > >> > >> also
> > > > > > >> > > > >> > >> > > > don't
> > > > > > >> > > > >> > >> > > > > > see
> > > > > > >> > > > >> > >> > > > > > > > any
> > > > > > >> > > > >> > >> > > > > > > > > >> benefit in sending
> > > addPartitionsToTxn
> > > > > > >> over
> > > > > > >> > the
> > > > > > >> > > > >> > >> > describeTxns
> > > > > > >> > > > >> > >> > > > > call.
> > > > > > >> > > > >> > >> > > > > > > They
> > > > > > >> > > > >> > >> > > > > > > > > will
> > > > > > >> > > > >> > >> > > > > > > > > >> both be one extra RPC to the
> Txn
> > > > > > >> > coordinator.
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >> To be clear — newer clients
> will
> > > use
> > > > > > >> > > > >> > addPartitionsToTxn
> > > > > > >> > > > >> > >> > > > instead
> > > > > > >> > > > >> > >> > > > > of
> > > > > > >> > > > >> > >> > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >> DescribeTxns.
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >> 40)
> > > > > > >> > > > >> > >> > > > > > > > > >> My concern is that if we have
> > > some
> > > > > delay
> > > > > > >> in
> > > > > > >> > > the
> > > > > > >> > > > >> > client
> > > > > > >> > > > >> > >> to
> > > > > > >> > > > >> > >> > > bump
> > > > > > >> > > > >> > >> > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > epoch,
> > > > > > >> > > > >> > >> > > > > > > > > >> it could continue to send
> epoch
> > > 73
> > > > > and
> > > > > > >> those
> > > > > > >> > > > >> records
> > > > > > >> > > > >> > >> would
> > > > > > >> > > > >> > >> > > not
> > > > > > >> > > > >> > >> > > > > be
> > > > > > >> > > > >> > >> > > > > > > > > fenced.
> > > > > > >> > > > >> > >> > > > > > > > > >> Perhaps this is not an issue
> if
> > > we
> > > > > don't
> > > > > > >> > allow
> > > > > > >> > > > the
> > > > > > >> > > > >> > next
> > > > > > >> > > > >> > >> > > > produce
> > > > > > >> > > > >> > >> > > > > to
> > > > > > >> > > > >> > >> > > > > > > go
> > > > > > >> > > > >> > >> > > > > > > > > >> through before the EndTxn
> request
> > > > > > >> returns.
> > > > > > >> > I'm
> > > > > > >> > > > >> also
> > > > > > >> > > > >> > >> > thinking
> > > > > > >> > > > >> > >> > > > > about
> > > > > > >> > > > >> > >> > > > > > > > > cases of
> > > > > > >> > > > >> > >> > > > > > > > > >> failure. I will need to
> think on
> > > > > this a
> > > > > > >> bit.
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >> I wasn't sure if it was that
> > > > > confusing.
> > > > > > >> But
> > > > > > >> > if
> > > > > > >> > > > we
> > > > > > >> > > > >> > >> think it
> > > > > > >> > > > >> > >> > > is,
> > > > > > >> > > > >> > >> > > > > we
> > > > > > >> > > > >> > >> > > > > > > can
> > > > > > >> > > > >> > >> > > > > > > > > >> investigate other ways.
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >> 60)
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >> I'm not sure these are the
> same
> > > > > > >> purgatories
> > > > > > >> > > > since
> > > > > > >> > > > >> one
> > > > > > >> > > > >> > >> is a
> > > > > > >> > > > >> > >> > > > > produce
> > > > > > >> > > > >> > >> > > > > > > > > >> purgatory (I was planning on
> > > using a
> > > > > > >> > callback
> > > > > > >> > > > >> rather
> > > > > > >> > > > >> > >> than
> > > > > > >> > > > >> > >> > > > > > purgatory)
> > > > > > >> > > > >> > >> > > > > > > > and
> > > > > > >> > > > >> > >> > > > > > > > > >> the other is simply a
> request to
> > > > > append
> > > > > > >> to
> > > > > > >> > the
> > > > > > >> > > > >> log.
> > > > > > >> > > > >> > Not
> > > > > > >> > > > >> > >> > sure
> > > > > > >> > > > >> > >> > > > we
> > > > > > >> > > > >> > >> > > > > > have
> > > > > > >> > > > >> > >> > > > > > > > any
> > > > > > >> > > > >> > >> > > > > > > > > >> structure here for ordering,
> but
> > > my
> > > > > > >> > > > understanding
> > > > > > >> > > > >> is
> > > > > > >> > > > >> > >> that
> > > > > > >> > > > >> > >> > > the
> > > > > > >> > > > >> > >> > > > > > broker
> > > > > > >> > > > >> > >> > > > > > > > > could
> > > > > > >> > > > >> > >> > > > > > > > > >> handle the write request
> before
> > > it
> > > > > hears
> > > > > > >> > back
> > > > > > >> > > > from
> > > > > > >> > > > >> > the
> > > > > > >> > > > >> > >> Txn
> > > > > > >> > > > >> > >> > > > > > > > Coordinator.
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >> Let me know if I
> misunderstood
> > > > > something
> > > > > > >> or
> > > > > > >> > > > >> something
> > > > > > >> > > > >> > >> was
> > > > > > >> > > > >> > >> > > > > unclear.
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >> Justine
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >> On Thu, Dec 1, 2022 at 12:15
> PM
> > > > > Matthias
> > > > > > >> J.
> > > > > > >> > > Sax
> > > > > > >> > > > <
> > > > > > >> > > > >> > >> > > > > mjsax@apache.org
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > wrote:
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >>> Thanks for the details
> Justine!
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>> 20)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>> The client side change for
> 2 is
> > > > > > >> removing
> > > > > > >> > the
> > > > > > >> > > > >> > >> > addPartitions
> > > > > > >> > > > >> > >> > > > to
> > > > > > >> > > > >> > >> > > > > > > > > >>> transaction
> > > > > > >> > > > >> > >> > > > > > > > > >>>> call. We don't need to make
> > > this
> > > > > from
> > > > > > >> the
> > > > > > >> > > > >> producer
> > > > > > >> > > > >> > to
> > > > > > >> > > > >> > >> > the
> > > > > > >> > > > >> > >> > > > txn
> > > > > > >> > > > >> > >> > > > > > > > > >>> coordinator,
> > > > > > >> > > > >> > >> > > > > > > > > >>>> only server side.
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>> I think I did not express
> myself
> > > > > > >> clearly. I
> > > > > > >> > > > >> > understand
> > > > > > >> > > > >> > >> > that
> > > > > > >> > > > >> > >> > > > we
> > > > > > >> > > > >> > >> > > > > > can
> > > > > > >> > > > >> > >> > > > > > > > (and
> > > > > > >> > > > >> > >> > > > > > > > > >>> should) change the producer
> to
> > > not
> > > > > send
> > > > > > >> the
> > > > > > >> > > > >> > >> > `addPartitions`
> > > > > > >> > > > >> > >> > > > > > request
> > > > > > >> > > > >> > >> > > > > > > > any
> > > > > > >> > > > >> > >> > > > > > > > > >>> longer. But I don't thinks
> it's
> > > > > > >> requirement
> > > > > > >> > > to
> > > > > > >> > > > >> > change
> > > > > > >> > > > >> > >> the
> > > > > > >> > > > >> > >> > > > > broker?
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>> What I am trying to say is:
> as a
> > > > > > >> safe-guard
> > > > > > >> > > and
> > > > > > >> > > > >> > >> > improvement
> > > > > > >> > > > >> > >> > > > for
> > > > > > >> > > > >> > >> > > > > > > older
> > > > > > >> > > > >> > >> > > > > > > > > >>> producers, the partition
> leader
> > > can
> > > > > just
> > > > > > >> > send
> > > > > > >> > > > the
> > > > > > >> > > > >> > >> > > > > `addPartitions`
> > > > > > >> > > > >> > >> > > > > > > > > >>> request to the
> TX-coordinator
> > > in any
> > > > > > >> case
> > > > > > >> > --
> > > > > > >> > > if
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> old
> > > > > > >> > > > >> > >> > > > > producer
> > > > > > >> > > > >> > >> > > > > > > > > >>> correctly did send the
> > > > > `addPartition`
> > > > > > >> > request
> > > > > > >> > > > to
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> > > > > > TX-coordinator
> > > > > > >> > > > >> > >> > > > > > > > > >>> already, the TX-coordinator
> can
> > > just
> > > > > > >> > "ignore"
> > > > > > >> > > > is
> > > > > > >> > > > >> as
> > > > > > >> > > > >> > >> > > > idempotent.
> > > > > > >> > > > >> > >> > > > > > > > > However,
> > > > > > >> > > > >> > >> > > > > > > > > >>> if the old producer has a
> bug
> > > and
> > > > > did
> > > > > > >> > forget
> > > > > > >> > > to
> > > > > > >> > > > >> sent
> > > > > > >> > > > >> > >> the
> > > > > > >> > > > >> > >> > > > > > > > `addPartition`
> > > > > > >> > > > >> > >> > > > > > > > > >>> request, we would now ensure
> > > that
> > > > > the
> > > > > > >> > > partition
> > > > > > >> > > > >> is
> > > > > > >> > > > >> > >> indeed
> > > > > > >> > > > >> > >> > > > added
> > > > > > >> > > > >> > >> > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> TX and thus fix a potential
> > > > > producer bug
> > > > > > >> > > (even
> > > > > > >> > > > >> if we
> > > > > > >> > > > >> > >> > don't
> > > > > > >> > > > >> > >> > > > get
> > > > > > >> > > > >> > >> > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> fencing via the bump
> epoch). --
> > > It
> > > > > > >> seems to
> > > > > > >> > > be
> > > > > > >> > > > a
> > > > > > >> > > > >> > good
> > > > > > >> > > > >> > >> > > > > > improvement?
> > > > > > >> > > > >> > >> > > > > > > Or
> > > > > > >> > > > >> > >> > > > > > > > > is
> > > > > > >> > > > >> > >> > > > > > > > > >>> there a reason to not do
> this?
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>> 30)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>> Transaction is ongoing =
> > > partition
> > > > > was
> > > > > > >> > added
> > > > > > >> > > > to
> > > > > > >> > > > >> > >> > > transaction
> > > > > > >> > > > >> > >> > > > > via
> > > > > > >> > > > >> > >> > > > > > > > > >>>> addPartitionsToTxn. We
> check
> > > this
> > > > > with
> > > > > > >> the
> > > > > > >> > > > >> > >> > > > > DescribeTransactions
> > > > > > >> > > > >> > >> > > > > > > > call.
> > > > > > >> > > > >> > >> > > > > > > > > >>> Let
> > > > > > >> > > > >> > >> > > > > > > > > >>>> me know if this wasn't
> > > sufficiently
> > > > > > >> > > explained
> > > > > > >> > > > >> here:
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>> If we do what I propose in
> > > (20), we
> > > > > > >> don't
> > > > > > >> > > > really
> > > > > > >> > > > >> > need
> > > > > > >> > > > >> > >> to
> > > > > > >> > > > >> > >> > > make
> > > > > > >> > > > >> > >> > > > > > this
> > > > > > >> > > > >> > >> > > > > > > > > >>> `DescribeTransaction` call,
> as
> > > the
> > > > > > >> > partition
> > > > > > >> > > > >> leader
> > > > > > >> > > > >> > >> adds
> > > > > > >> > > > >> > >> > > the
> > > > > > >> > > > >> > >> > > > > > > > partition
> > > > > > >> > > > >> > >> > > > > > > > > >>> for older clients and we get
> > > this
> > > > > check
> > > > > > >> for
> > > > > > >> > > > free.
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>> 40)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>> The idea here is that if
> any
> > > > > messages
> > > > > > >> > > somehow
> > > > > > >> > > > >> come
> > > > > > >> > > > >> > in
> > > > > > >> > > > >> > >> > > before
> > > > > > >> > > > >> > >> > > > > we
> > > > > > >> > > > >> > >> > > > > > > get
> > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> new
> > > > > > >> > > > >> > >> > > > > > > > > >>>> epoch to the producer, they
> > > will be
> > > > > > >> > fenced.
> > > > > > >> > > > >> > However,
> > > > > > >> > > > >> > >> if
> > > > > > >> > > > >> > >> > we
> > > > > > >> > > > >> > >> > > > > don't
> > > > > > >> > > > >> > >> > > > > > > > think
> > > > > > >> > > > >> > >> > > > > > > > > >>> this
> > > > > > >> > > > >> > >> > > > > > > > > >>>> is necessary, it can be
> > > discussed
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>> I agree that we should have
> > > epoch
> > > > > > >> fencing.
> > > > > > >> > My
> > > > > > >> > > > >> > >> question is
> > > > > > >> > > > >> > >> > > > > > > different:
> > > > > > >> > > > >> > >> > > > > > > > > >>> Assume we are at epoch 73,
> and
> > > we
> > > > > have
> > > > > > >> an
> > > > > > >> > > > ongoing
> > > > > > >> > > > >> > >> > > > transaction,
> > > > > > >> > > > >> > >> > > > > > that
> > > > > > >> > > > >> > >> > > > > > > > is
> > > > > > >> > > > >> > >> > > > > > > > > >>> committed. It seems natural
> to
> > > > > write the
> > > > > > >> > > > "prepare
> > > > > > >> > > > >> > >> commit"
> > > > > > >> > > > >> > >> > > > > marker
> > > > > > >> > > > >> > >> > > > > > > and
> > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` both
> with
> > > > > epoch
> > > > > > >> 73,
> > > > > > >> > > too,
> > > > > > >> > > > >> as
> > > > > > >> > > > >> > it
> > > > > > >> > > > >> > >> > > belongs
> > > > > > >> > > > >> > >> > > > > to
> > > > > > >> > > > >> > >> > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> current transaction. Of
> course,
> > > we
> > > > > now
> > > > > > >> also
> > > > > > >> > > > bump
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> > epoch
> > > > > > >> > > > >> > >> > > > and
> > > > > > >> > > > >> > >> > > > > > > expect
> > > > > > >> > > > >> > >> > > > > > > > > >>> the next requests to have
> epoch
> > > 74,
> > > > > and
> > > > > > >> > would
> > > > > > >> > > > >> reject
> > > > > > >> > > > >> > >> an
> > > > > > >> > > > >> > >> > > > request
> > > > > > >> > > > >> > >> > > > > > > with
> > > > > > >> > > > >> > >> > > > > > > > > >>> epoch 73, as the
> corresponding
> > > TX
> > > > > for
> > > > > > >> epoch
> > > > > > >> > > 73
> > > > > > >> > > > >> was
> > > > > > >> > > > >> > >> > already
> > > > > > >> > > > >> > >> > > > > > > committed.
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>> It seems you propose to
> write
> > > the
> > > > > > >> "prepare
> > > > > > >> > > > commit
> > > > > > >> > > > >> > >> marker"
> > > > > > >> > > > >> > >> > > and
> > > > > > >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` with
> > > epoch 74
> > > > > > >> > though,
> > > > > > >> > > > what
> > > > > > >> > > > >> > >> would
> > > > > > >> > > > >> > >> > > work,
> > > > > > >> > > > >> > >> > > > > but
> > > > > > >> > > > >> > >> > > > > > > it
> > > > > > >> > > > >> > >> > > > > > > > > >>> seems confusing. Is there a
> > > reason
> > > > > why
> > > > > > >> we
> > > > > > >> > > would
> > > > > > >> > > > >> use
> > > > > > >> > > > >> > >> the
> > > > > > >> > > > >> > >> > > > bumped
> > > > > > >> > > > >> > >> > > > > > > epoch
> > > > > > >> > > > >> > >> > > > > > > > 74
> > > > > > >> > > > >> > >> > > > > > > > > >>> instead of the current
> epoch 73?
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>> 60)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>> When we are checking if the
> > > > > > >> transaction is
> > > > > > >> > > > >> ongoing,
> > > > > > >> > > > >> > >> we
> > > > > > >> > > > >> > >> > > need
> > > > > > >> > > > >> > >> > > > to
> > > > > > >> > > > >> > >> > > > > > > make
> > > > > > >> > > > >> > >> > > > > > > > a
> > > > > > >> > > > >> > >> > > > > > > > > >>> round
> > > > > > >> > > > >> > >> > > > > > > > > >>>> trip from the leader
> partition
> > > to
> > > > > the
> > > > > > >> > > > >> transaction
> > > > > > >> > > > >> > >> > > > coordinator.
> > > > > > >> > > > >> > >> > > > > > In
> > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> time
> > > > > > >> > > > >> > >> > > > > > > > > >>>> we are waiting for this
> > > message to
> > > > > come
> > > > > > >> > > back,
> > > > > > >> > > > in
> > > > > > >> > > > >> > >> theory
> > > > > > >> > > > >> > >> > we
> > > > > > >> > > > >> > >> > > > > could
> > > > > > >> > > > >> > >> > > > > > > > have
> > > > > > >> > > > >> > >> > > > > > > > > >>> sent
> > > > > > >> > > > >> > >> > > > > > > > > >>>> a commit/abort call that
> would
> > > > > make the
> > > > > > >> > > > original
> > > > > > >> > > > >> > >> result
> > > > > > >> > > > >> > >> > of
> > > > > > >> > > > >> > >> > > > the
> > > > > > >> > > > >> > >> > > > > > > check
> > > > > > >> > > > >> > >> > > > > > > > > >>> out of
> > > > > > >> > > > >> > >> > > > > > > > > >>>> date. That is why we can
> check
> > > the
> > > > > > >> leader
> > > > > > >> > > > state
> > > > > > >> > > > >> > >> before
> > > > > > >> > > > >> > >> > we
> > > > > > >> > > > >> > >> > > > > write
> > > > > > >> > > > >> > >> > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> log.
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>> Thanks. Got it.
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>> However, is this really an
> > > issue?
> > > > > We put
> > > > > > >> > the
> > > > > > >> > > > >> produce
> > > > > > >> > > > >> > >> > > request
> > > > > > >> > > > >> > >> > > > in
> > > > > > >> > > > >> > >> > > > > > > > > >>> purgatory, so how could we
> > > process
> > > > > the
> > > > > > >> > > > >> > >> > > > `WriteTxnMarkerRequest`
> > > > > > >> > > > >> > >> > > > > > > first?
> > > > > > >> > > > >> > >> > > > > > > > > >>> Don't we need to put the
> > > > > > >> > > > `WriteTxnMarkerRequest`
> > > > > > >> > > > >> > into
> > > > > > >> > > > >> > >> > > > > purgatory,
> > > > > > >> > > > >> > >> > > > > > > too,
> > > > > > >> > > > >> > >> > > > > > > > > >>> for this case, and process
> both
> > > > > request
> > > > > > >> > > > in-order?
> > > > > > >> > > > >> > >> (Again,
> > > > > > >> > > > >> > >> > > my
> > > > > > >> > > > >> > >> > > > > > broker
> > > > > > >> > > > >> > >> > > > > > > > > >>> knowledge is limited and
> maybe
> > > we
> > > > > don't
> > > > > > >> > > > maintain
> > > > > > >> > > > >> > >> request
> > > > > > >> > > > >> > >> > > > order
> > > > > > >> > > > >> > >> > > > > > for
> > > > > > >> > > > >> > >> > > > > > > > this
> > > > > > >> > > > >> > >> > > > > > > > > >>> case, what seems to be an
> issue
> > > > > IMHO,
> > > > > > >> and I
> > > > > > >> > > am
> > > > > > >> > > > >> > >> wondering
> > > > > > >> > > > >> > >> > if
> > > > > > >> > > > >> > >> > > > > > > changing
> > > > > > >> > > > >> > >> > > > > > > > > >>> request handling to preserve
> > > order
> > > > > for
> > > > > > >> this
> > > > > > >> > > > case
> > > > > > >> > > > >> > >> might be
> > > > > > >> > > > >> > >> > > the
> > > > > > >> > > > >> > >> > > > > > > cleaner
> > > > > > >> > > > >> > >> > > > > > > > > >>> solution?)
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>> -Matthias
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>> On 11/30/22 3:28 PM, Artem
> > > Livshits
> > > > > > >> wrote:
> > > > > > >> > > > >> > >> > > > > > > > > >>>> Hi Justine,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>> I think the interesting
> part is
> > > > > not in
> > > > > > >> > this
> > > > > > >> > > > >> logic
> > > > > > >> > > > >> > >> > (because
> > > > > > >> > > > >> > >> > > > it
> > > > > > >> > > > >> > >> > > > > > > tries
> > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > > >>>> figure out when
> > > > > UNKNOWN_PRODUCER_ID is
> > > > > > >> > > > retriable
> > > > > > >> > > > >> > and
> > > > > > >> > > > >> > >> if
> > > > > > >> > > > >> > >> > > it's
> > > > > > >> > > > >> > >> > > > > > > > > retryable,
> > > > > > >> > > > >> > >> > > > > > > > > >>>> it's definitely not
> fatal), but
> > > > > what
> > > > > > >> > happens
> > > > > > >> > > > >> when
> > > > > > >> > > > >> > >> this
> > > > > > >> > > > >> > >> > > logic
> > > > > > >> > > > >> > >> > > > > > > doesn't
> > > > > > >> > > > >> > >> > > > > > > > > >>> return
> > > > > > >> > > > >> > >> > > > > > > > > >>>> 'true' and falls through.
> In
> > > the
> > > > > old
> > > > > > >> > > clients
> > > > > > >> > > > it
> > > > > > >> > > > >> > >> seems
> > > > > > >> > > > >> > >> > to
> > > > > > >> > > > >> > >> > > be
> > > > > > >> > > > >> > >> > > > > > > fatal,
> > > > > > >> > > > >> > >> > > > > > > > if
> > > > > > >> > > > >> > >> > > > > > > > > >>> we
> > > > > > >> > > > >> > >> > > > > > > > > >>>> keep the behavior in the
> new
> > > > > clients,
> > > > > > >> I'd
> > > > > > >> > > > >> expect it
> > > > > > >> > > > >> > >> > would
> > > > > > >> > > > >> > >> > > be
> > > > > > >> > > > >> > >> > > > > > fatal
> > > > > > >> > > > >> > >> > > > > > > > as
> > > > > > >> > > > >> > >> > > > > > > > > >>> well.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>> -Artem
> > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>> On Tue, Nov 29, 2022 at
> 11:57
> > > AM
> > > > > > >> Justine
> > > > > > >> > > > Olshan
> > > > > > >> > > > >> > >> > > > > > > > > >>>>
> <jo...@confluent.io.invalid>
> > > > > wrote:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> Hi Artem and Jeff,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> Thanks for taking a look
> and
> > > > > sorry for
> > > > > > >> > the
> > > > > > >> > > > slow
> > > > > > >> > > > >> > >> > response.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> You both mentioned the
> change
> > > to
> > > > > > >> handle
> > > > > > >> > > > >> > >> > > UNKNOWN_PRODUCER_ID
> > > > > > >> > > > >> > >> > > > > > > errors.
> > > > > > >> > > > >> > >> > > > > > > > > To
> > > > > > >> > > > >> > >> > > > > > > > > >>> be
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> clear — this error code
> will
> > > only
> > > > > be
> > > > > > >> sent
> > > > > > >> > > > again
> > > > > > >> > > > >> > when
> > > > > > >> > > > >> > >> > the
> > > > > > >> > > > >> > >> > > > > > client's
> > > > > > >> > > > >> > >> > > > > > > > > >>> request
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> version is high enough to
> > > ensure
> > > > > we
> > > > > > >> > handle
> > > > > > >> > > it
> > > > > > >> > > > >> > >> > correctly.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> The current (Java) client
> > > handles
> > > > > > >> this by
> > > > > > >> > > the
> > > > > > >> > > > >> > >> following
> > > > > > >> > > > >> > >> > > > > > (somewhat
> > > > > > >> > > > >> > >> > > > > > > > > long)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> code snippet:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> // An UNKNOWN_PRODUCER_ID
> > > means
> > > > > that
> > > > > > >> we
> > > > > > >> > > have
> > > > > > >> > > > >> lost
> > > > > > >> > > > >> > >> the
> > > > > > >> > > > >> > >> > > > > producer
> > > > > > >> > > > >> > >> > > > > > > > state
> > > > > > >> > > > >> > >> > > > > > > > > >>> on the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> broker. Depending on the
> log
> > > start
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> // offset, we may want to
> > > retry
> > > > > > >> these, as
> > > > > > >> > > > >> > described
> > > > > > >> > > > >> > >> for
> > > > > > >> > > > >> > >> > > > each
> > > > > > >> > > > >> > >> > > > > > case
> > > > > > >> > > > >> > >> > > > > > > > > >>> below. If
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> none of those apply, then
> for
> > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> // idempotent producer, we
> > > will
> > > > > > >> locally
> > > > > > >> > > bump
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> epoch
> > > > > > >> > > > >> > >> > > and
> > > > > > >> > > > >> > >> > > > > > reset
> > > > > > >> > > > >> > >> > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> sequence numbers of
> in-flight
> > > > > batches
> > > > > > >> > from
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> // sequence 0, then retry
> the
> > > > > failed
> > > > > > >> > batch,
> > > > > > >> > > > >> which
> > > > > > >> > > > >> > >> > should
> > > > > > >> > > > >> > >> > > > now
> > > > > > >> > > > >> > >> > > > > > > > succeed.
> > > > > > >> > > > >> > >> > > > > > > > > >>> For
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> the transactional
> producer,
> > > allow
> > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> // batch to fail. When
> > > processing
> > > > > the
> > > > > > >> > > failed
> > > > > > >> > > > >> > batch,
> > > > > > >> > > > >> > >> we
> > > > > > >> > > > >> > >> > > will
> > > > > > >> > > > >> > >> > > > > > > > > transition
> > > > > > >> > > > >> > >> > > > > > > > > >>> to
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> an abortable error and
> set a
> > > flag
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> // indicating that we
> need to
> > > > > bump the
> > > > > > >> > > epoch
> > > > > > >> > > > >> (if
> > > > > > >> > > > >> > >> > > supported
> > > > > > >> > > > >> > >> > > > by
> > > > > > >> > > > >> > >> > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> broker).
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> if (error ==
> > > > > > >> > Errors.*UNKNOWN_PRODUCER_ID*)
> > > > > > >> > > {
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>       if
> > > (response.logStartOffset
> > > > > ==
> > > > > > >> -1)
> > > > > > >> > {
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // We don't know
> > > the log
> > > > > > >> start
> > > > > > >> > > > offset
> > > > > > >> > > > >> > with
> > > > > > >> > > > >> > >> > this
> > > > > > >> > > > >> > >> > > > > > > response.
> > > > > > >> > > > >> > >> > > > > > > > > We
> > > > > > >> > > > >> > >> > > > > > > > > >>> should
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> just retry the request
> until
> > > we
> > > > > get
> > > > > > >> it.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // The
> > > > > UNKNOWN_PRODUCER_ID
> > > > > > >> > error
> > > > > > >> > > > code
> > > > > > >> > > > >> > was
> > > > > > >> > > > >> > >> > added
> > > > > > >> > > > >> > >> > > > > along
> > > > > > >> > > > >> > >> > > > > > > > with
> > > > > > >> > > > >> > >> > > > > > > > > >>> the new
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> ProduceResponse which
> > > includes the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           //
> logStartOffset.
> > > So
> > > > > the
> > > > > > >> '-1'
> > > > > > >> > > > >> sentinel
> > > > > > >> > > > >> > is
> > > > > > >> > > > >> > >> > not
> > > > > > >> > > > >> > >> > > > for
> > > > > > >> > > > >> > >> > > > > > > > backward
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> compatibility. Instead,
> it is
> > > > > possible
> > > > > > >> > for
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // a broker to
> not
> > > know
> > > > > the
> > > > > > >> > > > >> > >> logStartOffset at
> > > > > > >> > > > >> > >> > > > when
> > > > > > >> > > > >> > >> > > > > it
> > > > > > >> > > > >> > >> > > > > > > is
> > > > > > >> > > > >> > >> > > > > > > > > >>> returning
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> the response because the
> > > partition
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // may have
> moved
> > > away
> > > > > from
> > > > > > >> the
> > > > > > >> > > > >> broker
> > > > > > >> > > > >> > >> from
> > > > > > >> > > > >> > >> > the
> > > > > > >> > > > >> > >> > > > > time
> > > > > > >> > > > >> > >> > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> error was
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> initially raised to the
> time
> > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // response was
> > > being
> > > > > > >> > > constructed.
> > > > > > >> > > > In
> > > > > > >> > > > >> > >> these
> > > > > > >> > > > >> > >> > > > cases,
> > > > > > >> > > > >> > >> > > > > we
> > > > > > >> > > > >> > >> > > > > > > > > should
> > > > > > >> > > > >> > >> > > > > > > > > >>> just
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> retry the request: we are
> > > > > guaranteed
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // to eventually
> > > get a
> > > > > > >> > > > logStartOffset
> > > > > > >> > > > >> > once
> > > > > > >> > > > >> > >> > > things
> > > > > > >> > > > >> > >> > > > > > > settle
> > > > > > >> > > > >> > >> > > > > > > > > down.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>       if
> > > > > > >> (batch.sequenceHasBeenReset()) {
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // When the
> first
> > > > > inflight
> > > > > > >> > batch
> > > > > > >> > > > >> fails
> > > > > > >> > > > >> > >> due to
> > > > > > >> > > > >> > >> > > the
> > > > > > >> > > > >> > >> > > > > > > > > truncation
> > > > > > >> > > > >> > >> > > > > > > > > >>> case,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> then the sequences of all
> the
> > > > > other
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // in flight
> batches
> > > > > would
> > > > > > >> have
> > > > > > >> > > > been
> > > > > > >> > > > >> > >> > restarted
> > > > > > >> > > > >> > >> > > > from
> > > > > > >> > > > >> > >> > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> beginning.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> However, when those
> responses
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // come back
> from
> > > the
> > > > > > >> broker,
> > > > > > >> > > they
> > > > > > >> > > > >> would
> > > > > > >> > > > >> > >> also
> > > > > > >> > > > >> > >> > > > come
> > > > > > >> > > > >> > >> > > > > > with
> > > > > > >> > > > >> > >> > > > > > > > an
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> error. In
> > > this
> > > > > > >> case,
> > > > > > >> > we
> > > > > > >> > > > >> should
> > > > > > >> > > > >> > >> not
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // reset the
> > > sequence
> > > > > > >> numbers
> > > > > > >> > to
> > > > > > >> > > > the
> > > > > > >> > > > >> > >> > beginning.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>       } else if
> > > > > > >> > > > >> > >> > > > >
> > > (lastAckedOffset(batch.topicPartition).orElse(
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > > > > > >> > > > >> > >> > > > response.logStartOffset) {
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // The head of
> the
> > > log
> > > > > has
> > > > > > >> been
> > > > > > >> > > > >> removed,
> > > > > > >> > > > >> > >> > > probably
> > > > > > >> > > > >> > >> > > > > due
> > > > > > >> > > > >> > >> > > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> retention time elapsing.
> In
> > > this
> > > > > case,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // we expect to
> > > lose the
> > > > > > >> > producer
> > > > > > >> > > > >> state.
> > > > > > >> > > > >> > >> For
> > > > > > >> > > > >> > >> > > the
> > > > > > >> > > > >> > >> > > > > > > > > transactional
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> producer, reset the
> sequences
> > > of
> > > > > all
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // inflight
> batches
> > > to
> > > > > be
> > > > > > >> from
> > > > > > >> > > the
> > > > > > >> > > > >> > >> beginning
> > > > > > >> > > > >> > >> > > and
> > > > > > >> > > > >> > >> > > > > > retry
> > > > > > >> > > > >> > >> > > > > > > > > them,
> > > > > > >> > > > >> > >> > > > > > > > > >>> so
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> that the transaction does
> not
> > > > > need to
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // be aborted.
> For
> > > the
> > > > > > >> > idempotent
> > > > > > >> > > > >> > >> producer,
> > > > > > >> > > > >> > >> > > bump
> > > > > > >> > > > >> > >> > > > > the
> > > > > > >> > > > >> > >> > > > > > > > epoch
> > > > > > >> > > > >> > >> > > > > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > > >>> avoid
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> reusing (sequence, epoch)
> > > pairs
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           if
> > > (isTransactional()) {
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > >
> > > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> this.producerIdAndEpoch);
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           } else {
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> requestEpochBumpForPartition(batch.topicPartition);
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           }
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>       if
> (!isTransactional())
> > > {
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // For the
> > > idempotent
> > > > > > >> producer,
> > > > > > >> > > > >> always
> > > > > > >> > > > >> > >> retry
> > > > > > >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> errors. If the batch has
> the
> > > > > current
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           // producer ID
> and
> > > > > epoch,
> > > > > > >> > > request a
> > > > > > >> > > > >> bump
> > > > > > >> > > > >> > >> of
> > > > > > >> > > > >> > >> > the
> > > > > > >> > > > >> > >> > > > > > epoch.
> > > > > > >> > > > >> > >> > > > > > > > > >>> Otherwise
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> just retry the produce.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > >
> > > > > requestEpochBumpForPartition(batch.topicPartition);
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> }
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> I was considering keeping
> this
> > > > > > >> behavior —
> > > > > > >> > > but
> > > > > > >> > > > >> am
> > > > > > >> > > > >> > >> open
> > > > > > >> > > > >> > >> > to
> > > > > > >> > > > >> > >> > > > > > > > simplifying
> > > > > > >> > > > >> > >> > > > > > > > > >>> it.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> We are leaving changes to
> > > older
> > > > > > >> clients
> > > > > > >> > off
> > > > > > >> > > > the
> > > > > > >> > > > >> > >> table
> > > > > > >> > > > >> > >> > > here
> > > > > > >> > > > >> > >> > > > > > since
> > > > > > >> > > > >> > >> > > > > > > it
> > > > > > >> > > > >> > >> > > > > > > > > >>> caused
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> many issues for clients
> in the
> > > > > past.
> > > > > > >> > > > Previously
> > > > > > >> > > > >> > this
> > > > > > >> > > > >> > >> > was
> > > > > > >> > > > >> > >> > > a
> > > > > > >> > > > >> > >> > > > > > fatal
> > > > > > >> > > > >> > >> > > > > > > > > error
> > > > > > >> > > > >> > >> > > > > > > > > >>> and
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> we didn't have the
> mechanisms
> > > in
> > > > > > >> place to
> > > > > > >> > > > >> detect
> > > > > > >> > > > >> > >> when
> > > > > > >> > > > >> > >> > > this
> > > > > > >> > > > >> > >> > > > > was
> > > > > > >> > > > >> > >> > > > > > a
> > > > > > >> > > > >> > >> > > > > > > > > >>> legitimate
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> case vs some bug or gap
> in the
> > > > > > >> protocol.
> > > > > > >> > > > >> Ensuring
> > > > > > >> > > > >> > >> each
> > > > > > >> > > > >> > >> > > > > > > transaction
> > > > > > >> > > > >> > >> > > > > > > > > has
> > > > > > >> > > > >> > >> > > > > > > > > >>> its
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> own epoch should close
> this
> > > gap.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> And to address Jeff's
> second
> > > > > point:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> *does the typical produce
> > > request
> > > > > path
> > > > > > >> > > append
> > > > > > >> > > > >> > >> records
> > > > > > >> > > > >> > >> > to
> > > > > > >> > > > >> > >> > > > > local
> > > > > > >> > > > >> > >> > > > > > > log
> > > > > > >> > > > >> > >> > > > > > > > > >>> along*
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> *with the
> > > currentTxnFirstOffset
> > > > > > >> > > information?
> > > > > > >> > > > I
> > > > > > >> > > > >> > would
> > > > > > >> > > > >> > >> > like
> > > > > > >> > > > >> > >> > > > to
> > > > > > >> > > > >> > >> > > > > > > > > >>> understand*
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> *when the field is
> written to
> > > > > disk.*
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> Yes, the first produce
> request
> > > > > > >> populates
> > > > > > >> > > this
> > > > > > >> > > > >> > field
> > > > > > >> > > > >> > >> and
> > > > > > >> > > > >> > >> > > > > writes
> > > > > > >> > > > >> > >> > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> offset
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> as part of the record
> batch
> > > and
> > > > > also
> > > > > > >> to
> > > > > > >> > the
> > > > > > >> > > > >> > producer
> > > > > > >> > > > >> > >> > > state
> > > > > > >> > > > >> > >> > > > > > > > snapshot.
> > > > > > >> > > > >> > >> > > > > > > > > >>> When
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> we reload the records on
> > > restart
> > > > > > >> and/or
> > > > > > >> > > > >> > >> reassignment,
> > > > > > >> > > > >> > >> > we
> > > > > > >> > > > >> > >> > > > > > > repopulate
> > > > > > >> > > > >> > >> > > > > > > > > >>> this
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> field with the snapshot
> from
> > > disk
> > > > > > >> along
> > > > > > >> > > with
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> rest
> > > > > > >> > > > >> > >> > of
> > > > > > >> > > > >> > >> > > > the
> > > > > > >> > > > >> > >> > > > > > > > producer
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> state.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> Let me know if there are
> > > further
> > > > > > >> comments
> > > > > > >> > > > >> and/or
> > > > > > >> > > > >> > >> > > questions.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> Thanks,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> Justine
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> On Tue, Nov 22, 2022 at
> 9:00
> > > PM
> > > > > Jeff
> > > > > > >> Kim
> > > > > > >> > > > >> > >> > > > > > > > > <jeff.kim@confluent.io.invalid
> > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> wrote:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Hi Justine,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Thanks for the KIP! I
> have
> > > two
> > > > > > >> > questions:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> 1) For new clients, we
> can
> > > once
> > > > > again
> > > > > > >> > > return
> > > > > > >> > > > >> an
> > > > > > >> > > > >> > >> error
> > > > > > >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> for sequences
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> that are non-zero when
> there
> > > is
> > > > > no
> > > > > > >> > > producer
> > > > > > >> > > > >> state
> > > > > > >> > > > >> > >> > > present
> > > > > > >> > > > >> > >> > > > on
> > > > > > >> > > > >> > >> > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> server.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> This will indicate we
> missed
> > > the
> > > > > 0
> > > > > > >> > > sequence
> > > > > > >> > > > >> and
> > > > > > >> > > > >> > we
> > > > > > >> > > > >> > >> > don't
> > > > > > >> > > > >> > >> > > > yet
> > > > > > >> > > > >> > >> > > > > > > want
> > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> write
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> to the log.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> I would like to
> understand
> > > the
> > > > > > >> current
> > > > > > >> > > > >> behavior
> > > > > > >> > > > >> > to
> > > > > > >> > > > >> > >> > > handle
> > > > > > >> > > > >> > >> > > > > > older
> > > > > > >> > > > >> > >> > > > > > > > > >>> clients,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> and if there are any
> changes
> > > we
> > > > > are
> > > > > > >> > > making.
> > > > > > >> > > > >> Maybe
> > > > > > >> > > > >> > >> I'm
> > > > > > >> > > > >> > >> > > > > missing
> > > > > > >> > > > >> > >> > > > > > > > > >>> something,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> but we would want to
> identify
> > > > > > >> whether we
> > > > > > >> > > > >> missed
> > > > > > >> > > > >> > >> the 0
> > > > > > >> > > > >> > >> > > > > sequence
> > > > > > >> > > > >> > >> > > > > > > for
> > > > > > >> > > > >> > >> > > > > > > > > >>> older
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> clients, no?
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> 2) Upon returning from
> the
> > > > > > >> transaction
> > > > > > >> > > > >> > >> coordinator, we
> > > > > > >> > > > >> > >> > > can
> > > > > > >> > > > >> > >> > > > > set
> > > > > > >> > > > >> > >> > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> transaction
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> as ongoing on the leader
> by
> > > > > > >> populating
> > > > > > >> > > > >> > >> > > > currentTxnFirstOffset
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> through the typical
> produce
> > > > > request
> > > > > > >> > > > handling.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> does the typical produce
> > > request
> > > > > path
> > > > > > >> > > append
> > > > > > >> > > > >> > >> records
> > > > > > >> > > > >> > >> > to
> > > > > > >> > > > >> > >> > > > > local
> > > > > > >> > > > >> > >> > > > > > > log
> > > > > > >> > > > >> > >> > > > > > > > > >>> along
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> with the
> > > currentTxnFirstOffset
> > > > > > >> > > information?
> > > > > > >> > > > I
> > > > > > >> > > > >> > would
> > > > > > >> > > > >> > >> > like
> > > > > > >> > > > >> > >> > > > to
> > > > > > >> > > > >> > >> > > > > > > > > understand
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> when the field is
> written to
> > > > > disk.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Thanks,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Jeff
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> On Tue, Nov 22, 2022 at
> 4:44
> > > PM
> > > > > Artem
> > > > > > >> > > > Livshits
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> <alivshits@confluent.io
> > > .invalid>
> > > > > > >> wrote:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> Hi Justine,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> Thank you for the KIP.
> I
> > > have
> > > > > one
> > > > > > >> > > > question.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> 5) For new clients, we
> can
> > > once
> > > > > > >> again
> > > > > > >> > > > return
> > > > > > >> > > > >> an
> > > > > > >> > > > >> > >> error
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> I believe we had
> problems
> > > in the
> > > > > > >> past
> > > > > > >> > > with
> > > > > > >> > > > >> > >> returning
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> because it was
> considered
> > > fatal
> > > > > and
> > > > > > >> > > > required
> > > > > > >> > > > >> > >> client
> > > > > > >> > > > >> > >> > > > > restart.
> > > > > > >> > > > >> > >> > > > > > > It
> > > > > > >> > > > >> > >> > > > > > > > > >>> would
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> be
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> good to spell out the
> new
> > > client
> > > > > > >> > behavior
> > > > > > >> > > > >> when
> > > > > > >> > > > >> > it
> > > > > > >> > > > >> > >> > > > receives
> > > > > > >> > > > >> > >> > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > error.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> -Artem
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> On Tue, Nov 22, 2022 at
> > > 10:00 AM
> > > > > > >> > Justine
> > > > > > >> > > > >> Olshan
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > <jo...@confluent.io.invalid>
> > > > > > >> wrote:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks for taking a
> look
> > > > > Matthias.
> > > > > > >> > I've
> > > > > > >> > > > >> tried
> > > > > > >> > > > >> > to
> > > > > > >> > > > >> > >> > > answer
> > > > > > >> > > > >> > >> > > > > your
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> questions
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> below:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 10)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Right — so the hanging
> > > > > transaction
> > > > > > >> > only
> > > > > > >> > > > >> occurs
> > > > > > >> > > > >> > >> when
> > > > > > >> > > > >> > >> > we
> > > > > > >> > > > >> > >> > > > > have
> > > > > > >> > > > >> > >> > > > > > a
> > > > > > >> > > > >> > >> > > > > > > > late
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> message
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> come in and the
> partition
> > > is
> > > > > never
> > > > > > >> > added
> > > > > > >> > > > to
> > > > > > >> > > > >> a
> > > > > > >> > > > >> > >> > > > transaction
> > > > > > >> > > > >> > >> > > > > > > again.
> > > > > > >> > > > >> > >> > > > > > > > > If
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> we
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> never add the
> partition to
> > > a
> > > > > > >> > > transaction,
> > > > > > >> > > > we
> > > > > > >> > > > >> > will
> > > > > > >> > > > >> > >> > > never
> > > > > > >> > > > >> > >> > > > > > write
> > > > > > >> > > > >> > >> > > > > > > a
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> marker
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> and
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> never advance the LSO.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> If we do end up adding
> the
> > > > > > >> partition
> > > > > > >> > to
> > > > > > >> > > > the
> > > > > > >> > > > >> > >> > > transaction
> > > > > > >> > > > >> > >> > > > (I
> > > > > > >> > > > >> > >> > > > > > > > suppose
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> this
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> can
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> happen before or after
> the
> > > late
> > > > > > >> > message
> > > > > > >> > > > >> comes
> > > > > > >> > > > >> > in)
> > > > > > >> > > > >> > >> > then
> > > > > > >> > > > >> > >> > > > we
> > > > > > >> > > > >> > >> > > > > > will
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> include
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> late message in the
> next
> > > > > > >> (incorrect)
> > > > > > >> > > > >> > transaction.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> So perhaps it is
> clearer to
> > > > > make
> > > > > > >> the
> > > > > > >> > > > >> > distinction
> > > > > > >> > > > >> > >> > > between
> > > > > > >> > > > >> > >> > > > > > > > messages
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> that
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> eventually get added
> to the
> > > > > > >> > transaction
> > > > > > >> > > > (but
> > > > > > >> > > > >> > the
> > > > > > >> > > > >> > >> > wrong
> > > > > > >> > > > >> > >> > > > > one)
> > > > > > >> > > > >> > >> > > > > > or
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> messages
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> that never get added
> and
> > > become
> > > > > > >> > hanging.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 20)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> The client side change
> for
> > > 2 is
> > > > > > >> > removing
> > > > > > >> > > > the
> > > > > > >> > > > >> > >> > > > addPartitions
> > > > > > >> > > > >> > >> > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> transaction
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> call. We don't need to
> make
> > > > > this
> > > > > > >> from
> > > > > > >> > > the
> > > > > > >> > > > >> > >> producer
> > > > > > >> > > > >> > >> > to
> > > > > > >> > > > >> > >> > > > the
> > > > > > >> > > > >> > >> > > > > > txn
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> coordinator,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> only server side.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> In my opinion, the
> issue
> > > with
> > > > > the
> > > > > > >> > > > >> > >> addPartitionsToTxn
> > > > > > >> > > > >> > >> > > > call
> > > > > > >> > > > >> > >> > > > > > for
> > > > > > >> > > > >> > >> > > > > > > > > older
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> clients
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> is that we don't have
> the
> > > epoch
> > > > > > >> bump,
> > > > > > >> > so
> > > > > > >> > > > we
> > > > > > >> > > > >> > don't
> > > > > > >> > > > >> > >> > know
> > > > > > >> > > > >> > >> > > > if
> > > > > > >> > > > >> > >> > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>> message
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> belongs to the previous
> > > > > > >> transaction or
> > > > > > >> > > > this
> > > > > > >> > > > >> > one.
> > > > > > >> > > > >> > >> We
> > > > > > >> > > > >> > >> > > need
> > > > > > >> > > > >> > >> > > > > to
> > > > > > >> > > > >> > >> > > > > > > > check
> > > > > > >> > > > >> > >> > > > > > > > > if
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> partition has been
> added to
> > > > > this
> > > > > > >> > > > >> transaction.
> > > > > > >> > > > >> > Of
> > > > > > >> > > > >> > >> > > course,
> > > > > > >> > > > >> > >> > > > > > this
> > > > > > >> > > > >> > >> > > > > > > > > means
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> we
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> won't completely cover
> the
> > > case
> > > > > > >> where
> > > > > > >> > we
> > > > > > >> > > > >> have a
> > > > > > >> > > > >> > >> > really
> > > > > > >> > > > >> > >> > > > > late
> > > > > > >> > > > >> > >> > > > > > > > > message
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> and
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> we
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> have added the
> partition to
> > > > > the new
> > > > > > >> > > > >> > transaction,
> > > > > > >> > > > >> > >> but
> > > > > > >> > > > >> > >> > > > > that's
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> unfortunately
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> something we will need
> the
> > > new
> > > > > > >> clients
> > > > > > >> > > to
> > > > > > >> > > > >> > cover.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 30)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Transaction is ongoing
> =
> > > > > partition
> > > > > > >> was
> > > > > > >> > > > >> added to
> > > > > > >> > > > >> > >> > > > > transaction
> > > > > > >> > > > >> > >> > > > > > > via
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> addPartitionsToTxn. We
> > > check
> > > > > this
> > > > > > >> with
> > > > > > >> > > the
> > > > > > >> > > > >> > >> > > > > > > DescribeTransactions
> > > > > > >> > > > >> > >> > > > > > > > > >>> call.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> Let
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> me know if this wasn't
> > > > > sufficiently
> > > > > > >> > > > >> explained
> > > > > > >> > > > >> > >> here:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> > > > >> > >> > > > >
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > > >> > >> > >
> > > > > > >> > > > >> > >> >
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> >
> > > > > > >> > > > >>
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 40)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> The idea here is that
> if
> > > any
> > > > > > >> messages
> > > > > > >> > > > >> somehow
> > > > > > >> > > > >> > >> come
> > > > > > >> > > > >> > >> > in
> > > > > > >> > > > >> > >> > > > > before
> > > > > > >> > > > >> > >> > > > > > > we
> > > > > > >> > > > >> > >> > > > > > > > > get
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> new
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> epoch to the producer,
> they
> > > > > will be
> > > > > > >> > > > fenced.
> > > > > > >> > > > >> > >> However,
> > > > > > >> > > > >> > >> > > if
> > > > > > >> > > > >> > >> > > > we
> > > > > > >> > > > >> > >> > > > > > > don't
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> think
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> this
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> is necessary, it can be
> > > > > discussed
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 50)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> It should be
> synchronous
> > > > > because
> > > > > > >> if we
> > > > > > >> > > > have
> > > > > > >> > > > >> an
> > > > > > >> > > > >> > >> event
> > > > > > >> > > > >> > >> > > > (ie,
> > > > > > >> > > > >> > >> > > > > an
> > > > > > >> > > > >> > >> > > > > > > > > error)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> that
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> causes us to need to
> abort
> > > the
> > > > > > >> > > > transaction,
> > > > > > >> > > > >> we
> > > > > > >> > > > >> > >> need
> > > > > > >> > > > >> > >> > to
> > > > > > >> > > > >> > >> > > > > know
> > > > > > >> > > > >> > >> > > > > > > > which
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> partitions to send
> > > transaction
> > > > > > >> markers
> > > > > > >> > > to.
> > > > > > >> > > > >> We
> > > > > > >> > > > >> > >> know
> > > > > > >> > > > >> > >> > the
> > > > > > >> > > > >> > >> > > > > > > > partitions
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> because
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> we added them to the
> > > > > coordinator
> > > > > > >> via
> > > > > > >> > the
> > > > > > >> > > > >> > >> > > > > addPartitionsToTxn
> > > > > > >> > > > >> > >> > > > > > > > call.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Previously we have had
> > > > > asynchronous
> > > > > > >> > > calls
> > > > > > >> > > > in
> > > > > > >> > > > >> > the
> > > > > > >> > > > >> > >> > past
> > > > > > >> > > > >> > >> > > > (ie,
> > > > > > >> > > > >> > >> > > > > > > > writing
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> commit markers when the
> > > > > > >> transaction is
> > > > > > >> > > > >> > completed)
> > > > > > >> > > > >> > >> > but
> > > > > > >> > > > >> > >> > > > > often
> > > > > > >> > > > >> > >> > > > > > > this
> > > > > > >> > > > >> > >> > > > > > > > > >>> just
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> causes confusion as we
> > > need to
> > > > > wait
> > > > > > >> > for
> > > > > > >> > > > some
> > > > > > >> > > > >> > >> > > operations
> > > > > > >> > > > >> > >> > > > to
> > > > > > >> > > > >> > >> > > > > > > > > complete.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> In
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> writing commit markers
> > > case,
> > > > > > >> clients
> > > > > > >> > > often
> > > > > > >> > > > >> see
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> CONCURRENT_TRANSACTIONs
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> error messages and that
> > > can be
> > > > > > >> > > confusing.
> > > > > > >> > > > >> For
> > > > > > >> > > > >> > >> that
> > > > > > >> > > > >> > >> > > > reason,
> > > > > > >> > > > >> > >> > > > > > it
> > > > > > >> > > > >> > >> > > > > > > > may
> > > > > > >> > > > >> > >> > > > > > > > > be
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> simpler to just have
> > > > > synchronous
> > > > > > >> > calls —
> > > > > > >> > > > >> > >> especially
> > > > > > >> > > > >> > >> > if
> > > > > > >> > > > >> > >> > > > we
> > > > > > >> > > > >> > >> > > > > > need
> > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> block
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> on
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> some operation's
> completion
> > > > > anyway
> > > > > > >> > > before
> > > > > > >> > > > we
> > > > > > >> > > > >> > can
> > > > > > >> > > > >> > >> > start
> > > > > > >> > > > >> > >> > > > the
> > > > > > >> > > > >> > >> > > > > > > next
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> transaction. And yes, I
> > > meant
> > > > > > >> > > > coordinator. I
> > > > > > >> > > > >> > will
> > > > > > >> > > > >> > >> > fix
> > > > > > >> > > > >> > >> > > > > that.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 60)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> When we are checking
> if the
> > > > > > >> > transaction
> > > > > > >> > > is
> > > > > > >> > > > >> > >> ongoing,
> > > > > > >> > > > >> > >> > we
> > > > > > >> > > > >> > >> > > > > need
> > > > > > >> > > > >> > >> > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > > make
> > > > > > >> > > > >> > >> > > > > > > > > >>> a
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> round
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> trip from the leader
> > > partition
> > > > > to
> > > > > > >> the
> > > > > > >> > > > >> > transaction
> > > > > > >> > > > >> > >> > > > > > coordinator.
> > > > > > >> > > > >> > >> > > > > > > > In
> > > > > > >> > > > >> > >> > > > > > > > > >>> the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> time
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> we are waiting for this
> > > > > message to
> > > > > > >> > come
> > > > > > >> > > > >> back,
> > > > > > >> > > > >> > in
> > > > > > >> > > > >> > >> > > theory
> > > > > > >> > > > >> > >> > > > we
> > > > > > >> > > > >> > >> > > > > > > could
> > > > > > >> > > > >> > >> > > > > > > > > >>> have
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> sent
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> a commit/abort call
> that
> > > would
> > > > > make
> > > > > > >> > the
> > > > > > >> > > > >> > original
> > > > > > >> > > > >> > >> > > result
> > > > > > >> > > > >> > >> > > > of
> > > > > > >> > > > >> > >> > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > check
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> out
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> of
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> date. That is why we
> can
> > > check
> > > > > the
> > > > > > >> > > leader
> > > > > > >> > > > >> state
> > > > > > >> > > > >> > >> > before
> > > > > > >> > > > >> > >> > > > we
> > > > > > >> > > > >> > >> > > > > > > write
> > > > > > >> > > > >> > >> > > > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> log.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> I'm happy to update the
> > > KIP if
> > > > > > >> some of
> > > > > > >> > > > these
> > > > > > >> > > > >> > >> things
> > > > > > >> > > > >> > >> > > were
> > > > > > >> > > > >> > >> > > > > not
> > > > > > >> > > > >> > >> > > > > > > > > clear.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Justine
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> On Mon, Nov 21, 2022 at
> > > 7:11 PM
> > > > > > >> > Matthias
> > > > > > >> > > > J.
> > > > > > >> > > > >> > Sax <
> > > > > > >> > > > >> > >> > > > > > > > mjsax@apache.org
> > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> wrote:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Thanks for the KIP.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Couple of
> clarification
> > > > > questions
> > > > > > >> (I
> > > > > > >> > am
> > > > > > >> > > > >> not a
> > > > > > >> > > > >> > >> > broker
> > > > > > >> > > > >> > >> > > > > expert
> > > > > > >> > > > >> > >> > > > > > > do
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> maybe
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> some question are
> obvious
> > > for
> > > > > > >> others,
> > > > > > >> > > but
> > > > > > >> > > > >> not
> > > > > > >> > > > >> > >> for
> > > > > > >> > > > >> > >> > me
> > > > > > >> > > > >> > >> > > > with
> > > > > > >> > > > >> > >> > > > > > my
> > > > > > >> > > > >> > >> > > > > > > > lack
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> of
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> broker knowledge).
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (10)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> 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.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> What happens if the
> > > message
> > > > > come
> > > > > > >> in
> > > > > > >> > > > before
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> next
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> addPartitionsToTxn
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> request? It seems the
> > > broker
> > > > > > >> hosting
> > > > > > >> > > the
> > > > > > >> > > > >> data
> > > > > > >> > > > >> > >> > > > partitions
> > > > > > >> > > > >> > >> > > > > > > won't
> > > > > > >> > > > >> > >> > > > > > > > > know
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> anything about it and
> > > append
> > > > > it to
> > > > > > >> > the
> > > > > > >> > > > >> > >> partition,
> > > > > > >> > > > >> > >> > > too?
> > > > > > >> > > > >> > >> > > > > What
> > > > > > >> > > > >> > >> > > > > > > is
> > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> difference between
> both
> > > cases?
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Also, it seems a TX
> would
> > > only
> > > > > > >> hang,
> > > > > > >> > if
> > > > > > >> > > > >> there
> > > > > > >> > > > >> > >> is no
> > > > > > >> > > > >> > >> > > > > > following
> > > > > > >> > > > >> > >> > > > > > > > TX
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> that
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> is
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> either committer or
> > > aborted?
> > > > > Thus,
> > > > > > >> > for
> > > > > > >> > > > the
> > > > > > >> > > > >> > case
> > > > > > >> > > > >> > >> > > above,
> > > > > > >> > > > >> > >> > > > > the
> > > > > > >> > > > >> > >> > > > > > TX
> > > > > > >> > > > >> > >> > > > > > > > > might
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> actually not hang (of
> > > course,
> > > > > we
> > > > > > >> > might
> > > > > > >> > > > get
> > > > > > >> > > > >> an
> > > > > > >> > > > >> > >> EOS
> > > > > > >> > > > >> > >> > > > > violation
> > > > > > >> > > > >> > >> > > > > > > if
> > > > > > >> > > > >> > >> > > > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> first
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> TX was aborted and the
> > > second
> > > > > > >> > > committed,
> > > > > > >> > > > or
> > > > > > >> > > > >> > the
> > > > > > >> > > > >> > >> > other
> > > > > > >> > > > >> > >> > > > way
> > > > > > >> > > > >> > >> > > > > > > > > around).
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (20)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Of course, 1 and 2
> > > require
> > > > > > >> > client-side
> > > > > > >> > > > >> > >> changes, so
> > > > > > >> > > > >> > >> > > for
> > > > > > >> > > > >> > >> > > > > > older
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> clients,
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> those approaches won’t
> > > apply.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> For (1) I understand
> why a
> > > > > client
> > > > > > >> > > change
> > > > > > >> > > > is
> > > > > > >> > > > >> > >> > > necessary,
> > > > > > >> > > > >> > >> > > > > but
> > > > > > >> > > > >> > >> > > > > > > not
> > > > > > >> > > > >> > >> > > > > > > > > sure
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> why
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> we need a client
> change
> > > for
> > > > > (2).
> > > > > > >> Can
> > > > > > >> > > you
> > > > > > >> > > > >> > >> elaborate?
> > > > > > >> > > > >> > >> > > --
> > > > > > >> > > > >> > >> > > > > > Later
> > > > > > >> > > > >> > >> > > > > > > > you
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> explain
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> that we should send a
> > > > > > >> > > > >> > >> DescribeTransactionRequest,
> > > > > > >> > > > >> > >> > > but I
> > > > > > >> > > > >> > >> > > > > am
> > > > > > >> > > > >> > >> > > > > > > not
> > > > > > >> > > > >> > >> > > > > > > > > sure
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> why?
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Can't we not just do
> an
> > > > > implicit
> > > > > > >> > > > >> > >> AddPartiitonToTx,
> > > > > > >> > > > >> > >> > > too?
> > > > > > >> > > > >> > >> > > > > If
> > > > > > >> > > > >> > >> > > > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > old
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> producer correctly
> > > registered
> > > > > the
> > > > > > >> > > > partition
> > > > > > >> > > > >> > >> > already,
> > > > > > >> > > > >> > >> > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> TX-coordinator
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> can just ignore it as
> > > it's an
> > > > > > >> > > idempotent
> > > > > > >> > > > >> > >> operation?
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (30)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> To cover older
> clients,
> > > we
> > > > > will
> > > > > > >> > > ensure a
> > > > > > >> > > > >> > >> > transaction
> > > > > > >> > > > >> > >> > > > is
> > > > > > >> > > > >> > >> > > > > > > > ongoing
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> before
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> we write to a
> transaction
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Not sure what you
> mean by
> > > > > this?
> > > > > > >> Can
> > > > > > >> > you
> > > > > > >> > > > >> > >> elaborate?
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (40)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> [the TX-coordinator]
> will
> > > > > write
> > > > > > >> the
> > > > > > >> > > > >> prepare
> > > > > > >> > > > >> > >> commit
> > > > > > >> > > > >> > >> > > > > message
> > > > > > >> > > > >> > >> > > > > > > > with
> > > > > > >> > > > >> > >> > > > > > > > > a
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> bumped
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> epoch and send
> > > > > > >> WriteTxnMarkerRequests
> > > > > > >> > > > with
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > >> > bumped
> > > > > > >> > > > >> > >> > > > > > epoch.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Why do we use the
> bumped
> > > > > epoch for
> > > > > > >> > > both?
> > > > > > >> > > > It
> > > > > > >> > > > >> > >> seems
> > > > > > >> > > > >> > >> > > more
> > > > > > >> > > > >> > >> > > > > > > > intuitive
> > > > > > >> > > > >> > >> > > > > > > > > to
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> use
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> the current epoch, and
> > > only
> > > > > return
> > > > > > >> > the
> > > > > > >> > > > >> bumped
> > > > > > >> > > > >> > >> epoch
> > > > > > >> > > > >> > >> > > to
> > > > > > >> > > > >> > >> > > > > the
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> producer?
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (50) "Implicit
> > > > > > >> > > AddPartitionToTransaction"
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Why does the
> implicitly
> > > sent
> > > > > > >> request
> > > > > > >> > > need
> > > > > > >> > > > >> to
> > > > > > >> > > > >> > be
> > > > > > >> > > > >> > >> > > > > > synchronous?
> > > > > > >> > > > >> > >> > > > > > > > The
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> KIP
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> also says
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> in case we need to
> abort
> > > and
> > > > > > >> need to
> > > > > > >> > > > know
> > > > > > >> > > > >> > which
> > > > > > >> > > > >> > >> > > > > partitions
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> What do you mean by
> this?
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> we don’t want to
> write
> > > to it
> > > > > > >> before
> > > > > > >> > we
> > > > > > >> > > > >> store
> > > > > > >> > > > >> > in
> > > > > > >> > > > >> > >> > the
> > > > > > >> > > > >> > >> > > > > > > > transaction
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>> manager
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Do you mean
> TX-coordinator
> > > > > > >> instead of
> > > > > > >> > > > >> > "manager"?
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (60)
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> For older clients and
> > > ensuring
> > > > > > >> that
> > > > > > >> > the
> > > > > > >> > > > TX
> > > > > > >> > > > >> is
> > > > > > >> > > > >> > >> > > ongoing,
> > > > > > >> > > > >> > >> > > > > you
> > > > > > >> > > > >> > >> > > > > > > > > >>>>> describe a
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> race condition. I am
> not
> > > sure
> > > > > if I
> > > > > > >> > can
> > > > > > >> > > > >> follow
> > > > > > >> > > > >> > >> here.
> > > > > > >> > > > >> > >> > > Can
> > > > > > >> > > > >> > >> > > > > you
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> elaborate?
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> -Matthias
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> On 11/18/22 1:21 PM,
> > > Justine
> > > > > > >> Olshan
> > > > > > >> > > > wrote:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Hey all!
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> I'd like to start a
> > > > > discussion
> > > > > > >> on my
> > > > > > >> > > > >> proposal
> > > > > > >> > > > >> > >> to
> > > > > > >> > > > >> > >> > add
> > > > > > >> > > > >> > >> > > > > some
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> server-side
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> checks on
> transactions to
> > > > > avoid
> > > > > > >> > > hanging
> > > > > > >> > > > >> > >> > > transactions.
> > > > > > >> > > > >> > >> > > > I
> > > > > > >> > > > >> > >> > > > > > know
> > > > > > >> > > > >> > >> > > > > > > > > this
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> has
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> been
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> an issue for some
> time,
> > > so I
> > > > > > >> really
> > > > > > >> > > hope
> > > > > > >> > > > >> this
> > > > > > >> > > > >> > >> KIP
> > > > > > >> > > > >> > >> > > will
> > > > > > >> > > > >> > >> > > > > be
> > > > > > >> > > > >> > >> > > > > > > > > helpful
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> for
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> many
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> users of EOS.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> The KIP includes
> changes
> > > that
> > > > > > >> will
> > > > > > >> > be
> > > > > > >> > > > >> > >> compatible
> > > > > > >> > > > >> > >> > > with
> > > > > > >> > > > >> > >> > > > > old
> > > > > > >> > > > >> > >> > > > > > > > > clients
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>> and
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> changes to improve
> > > > > performance
> > > > > > >> and
> > > > > > >> > > > >> > correctness
> > > > > > >> > > > >> > >> on
> > > > > > >> > > > >> > >> > > new
> > > > > > >> > > > >> > >> > > > > > > clients.
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Please take a look
> and
> > > leave
> > > > > any
> > > > > > >> > > > comments
> > > > > > >> > > > >> you
> > > > > > >> > > > >> > >> may
> > > > > > >> > > > >> > >> > > > have!
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> KIP:
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> > > > >> > >> > > > >
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > > >> > >> > >
> > > > > > >> > > > >> > >> >
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> >
> > > > > > >> > > > >>
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> JIRA:
> > > > > > >> > > > >> > >> > > >
> > > https://issues.apache.org/jira/browse/KAFKA-14402
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Thanks!
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Justine
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > > >> > > > >> > >> > > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > > >
> > > > > > >> > > > >> > >> > > > > > > >
> > > > > > >> > > > >> > >> > > > > > >
> > > > > > >> > > > >> > >> > > > > >
> > > > > > >> > > > >> > >> > > > >
> > > > > > >> > > > >> > >> > > >
> > > > > > >> > > > >> > >> > >
> > > > > > >> > > > >> > >> >
> > > > > > >> > > > >> > >>
> > > > > > >> > > > >> > >
> > > > > > >> > > > >> >
> > > > > > >> > > > >>
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > > >
> > > > >
> > >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Guozhang Wang <gu...@gmail.com>.
For InvalidRecord -- when I did the research on KIP-691 I saw from the
code we would treat them as abortableError, not fatalError inside
Producer's TxnManager (again sorry for raising such implementation
details in KIP discussions), so I think, from client's perspective,
they will not cause the client to be shutdown automatically?

On Fri, Jan 20, 2023 at 2:52 PM Justine Olshan
<jo...@confluent.io.invalid> wrote:
>
> That's a fair point about other clients.
>
> I think the abortable error case is interesting because I'm curious how
> other clients would handle this. I assume they would need to implement
> handling for the error code unless they did something like "any unknown
> error codes/any codes that aren't x,y,z are retriable." I would hope that
> unknown error codes were fatal, and if the code was implemented it would
> abort the transaction. But I will think on this too.
>
> As for InvalidRecord -- you mentioned it was not fatal, but I'm taking a
> look through the code. We would see this on handling the produce response.
> If I recall correctly, we check if errors are retriable. I think this error
> would not be retriable. But I guess the concern here is that it is not
> enough for just that batch to fail. I guess I hadn't considered fully
> fencing the old producer but there are valid arguments here why we would
> want to.
>
> Thanks,
> Justine
>
> On Fri, Jan 20, 2023 at 2:35 PM Guozhang Wang <gu...@gmail.com>
> wrote:
>
> > Thanks Justine for the replies! I agree with most of your thoughts.
> >
> > Just for 3/7), though I agree for our own AK producer, since we do
> > "nextRequest(boolean hasIncompleteBatches)", we guarantee the end-txn
> > would not be sent until we've effectively flushed, but I was referring
> > to any future bugs or other buggy clients that the same client may get
> > into this situation, in which case we should give the client a clear
> > msg that "you did something wrong, and hence now you should fatally
> > close yourself". What I'm concerned about is that, by seeing an
> > "abortable error" or in some rare cases an "invalid record", the
> > client could not realize "something that's really bad happened". So
> > it's not about adding a new error, it's mainly about those real buggy
> > situations causing such "should never happen" cases, the errors return
> > would not be informative enough.
> >
> > Thinking in other ways, if we believe that for most cases such error
> > codes would not reach the original clients since they would be
> > disconnected or even gone by that time, and only in some rare cases
> > they would still be seen by the sending clients, then why not make
> > them more fatal and more specific than generic.
> >
> > Guozhang
> >
> > On Fri, Jan 20, 2023 at 1:59 PM Justine Olshan
> > <jo...@confluent.io.invalid> wrote:
> > >
> > > Hey Guozhang. Thanks for taking a look and for the detailed comments!
> > I'll
> > > do my best to address below.
> > >
> > > 1. I see what you are saying here, but I think I need to look through the
> > > sequence of events you mention. Typically we've seen this issue in a few
> > > cases.
> > >
> > >  One is when we have a producer disconnect when trying to produce.
> > > Typically in these cases, we abort the transaction. We've seen that after
> > > the markers are written, the disconnection can sometimes cause the
> > request
> > > to get flushed to the broker. In this case, we don't need client handling
> > > because the producer we are responding to is gone. We just needed to make
> > > sure we didn't write to the log on the broker side. I'm trying to think
> > of
> > > a case where we do have the client to return to. I'd think the same
> > client
> > > couldn't progress to committing the transaction unless the produce
> > request
> > > returned right? Of course, there is the incorrectly written clients case.
> > > I'll think on this a bit more and let you know if I come up with another
> > > scenario when we would return to an active client when the transaction is
> > > no longer ongoing.
> > >
> > > I was not aware that we checked the result of a send after we commit
> > > though. I'll need to look into that a bit more.
> > >
> > > 2. There were some questions about this in the discussion. The plan is to
> > > handle overflow with the mechanism we currently have in the producer. If
> > we
> > > try to bump and the epoch will overflow, we actually allocate a new
> > > producer ID. I need to confirm the fencing logic on the last epoch (ie,
> > we
> > > probably shouldn't allow any records to be produced with the final epoch
> > > since we can never properly fence that one).
> > >
> > > 3. I can agree with you that the current error handling is messy. I
> > recall
> > > taking a look at your KIP a while back, but I think I mostly saw the
> > > section about how the errors were wrapped. Maybe I need to take another
> > > look. As for abortable error, the idea was that the handling would be
> > > simple -- if this error is seen, the transaction should be aborted -- no
> > > other logic about previous state or requests necessary. Is your concern
> > > simply about adding new errors? We were hoping to have an error that
> > would
> > > have one meaning and many of the current errors have a history of meaning
> > > different things on different client versions. That was the main
> > motivation
> > > for adding a new error.
> > >
> > > 4. This is a good point about record timestamp reordering. Timestamps
> > don't
> > > affect compaction, but they do affect retention deletion. For that, kafka
> > > considers the largest timestamp in the segment, so I think a small amount
> > > of reordering (hopefully on the order of milliseconds or even seconds)
> > will
> > > be ok. We take timestamps from clients so there is already a possibility
> > > for some drift and non-monotonically increasing timestamps.
> > >
> > > 5. Thanks for catching. The error is there, but it's actually that those
> > > fields should be 4+! Due to how the message generator works, I actually
> > > have to redefine those fields inside the `"AddPartitionsToTxnTransaction`
> > > block for it to build correctly. I'll fix it to be correct.
> > >
> > > 6. Correct -- we will only add the request to purgatory if the cache has
> > no
> > > ongoing transaction. I can change the wording to make that clearer that
> > we
> > > only place the request in purgatory if we need to contact the transaction
> > > coordinator.
> > >
> > > 7. We did take a look at some of the errors and it was hard to come up
> > with
> > > a good one. I agree that InvalidTxnStateException is ideal except for the
> > > fact that it hasn't been returned on Produce requests before. The error
> > > handling for clients is a bit vague (which is why I opened KAFKA-14439
> > > <https://issues.apache.org/jira/browse/KAFKA-14439>), but the decision
> > we
> > > made here was to only return errors that have been previously returned to
> > > producers. As for not being fatal, I think part of the theory was that in
> > > many cases, the producer would be disconnected. (See point 1) and this
> > > would just be an error to return from the server. I did plan to think
> > about
> > > other cases, so let me know if you think of any as well!
> > >
> > > Lots to say! Let me know if you have further thoughts!
> > > Justine
> > >
> > > On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang <
> > guozhang.wang.us@gmail.com>
> > > wrote:
> > >
> > > > Hello Justine,
> > > >
> > > > Thanks for the great write-up! I made a quick pass through it and here
> > > > are some thoughts (I have not been able to read through this thread so
> > > > pardon me if they have overlapped or subsumed by previous comments):
> > > >
> > > > First are some meta ones:
> > > >
> > > > 1. I think we need to also improve the client's experience once we
> > > > have this defence in place. More concretely, say a user's producer
> > > > code is like following:
> > > >
> > > > future = producer.send();
> > > > // producer.flush();
> > > > producer.commitTransaction();
> > > > future.get();
> > > >
> > > > Which resulted in the order of a) produce-request sent by producer, b)
> > > > end-txn-request sent by producer, c) end-txn-response sent back, d)
> > > > txn-marker-request sent from coordinator to partition leader, e)
> > > > produce-request finally received by the partition leader, before this
> > > > KIP e) step would be accepted causing a dangling txn; now it would be
> > > > rejected in step e) which is good. But from the client's point of view
> > > > now it becomes confusing since the `commitTransaction()` returns
> > > > successfully, but the "future" throws an invalid-epoch error, and they
> > > > are not sure if the transaction did succeed or not. In fact, it
> > > > "partially succeeded" with some msgs being rejected but others
> > > > committed successfully.
> > > >
> > > > Of course the easy way to avoid this is, always call
> > > > "producer.flush()" before commitTxn and that's what we do ourselves,
> > > > and what we recommend users do. But I suspect not everyone does it. In
> > > > fact I just checked the javadoc in KafkaProducer and our code snippet
> > > > does not include a `flush()` call. So I'm thinking maybe we can in
> > > > side the `commitTxn` code to enforce flushing before sending the
> > > > end-txn request.
> > > >
> > > > 2. I'd like to clarify a bit details on "just add partitions to the
> > > > transaction on the first produce request during a transaction". My
> > > > understanding is that the partition leader's cache has the producer id
> > > > / sequence / epoch for the latest txn, either on-going or is completed
> > > > (upon receiving the marker request from coordinator). When a produce
> > > > request is received, if
> > > >
> > > > * producer's epoch < cached epoch, or producer's epoch == cached epoch
> > > > but the latest txn is completed, leader directly reject with
> > > > invalid-epoch.
> > > > * producer's epoch > cached epoch, park the the request and send
> > > > add-partitions request to coordinator.
> > > >
> > > > In order to do it, does the coordinator need to bump the sequence and
> > > > reset epoch to 0 when the next epoch is going to overflow? If no need
> > > > to do so, then how we handle the (admittedly rare, but still may
> > > > happen) epoch overflow situation?
> > > >
> > > > 3. I'm a bit concerned about adding a generic "ABORTABLE_ERROR" given
> > > > we already have a pretty messy error classification and error handling
> > > > on the producer clients side --- I have a summary about the issues and
> > > > a proposal to address this in
> > > >
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
> > > > -- I understand we do not want to use "UNKNOWN_PRODUCER_ID" anymore
> > > > and in fact we intend to deprecate it in KIP-360 and eventually remove
> > > > it; but I'm wondering can we still use specific error codes. E.g. what
> > > > about "InvalidProducerEpochException" since for new clients, the
> > > > actual reason this would actually be rejected is indeed because the
> > > > epoch on the coordinator caused the add-partitions-request from the
> > > > brokers to be rejected anyways?
> > > >
> > > > 4. It seems we put the producer request into purgatory before we ever
> > > > append the records, while other producer's records may still be
> > > > appended during the time; and that potentially may result in some
> > > > re-ordering compared with reception order. I'm not super concerned
> > > > about it since Kafka does not guarantee reception ordering across
> > > > producers anyways, but it may make the timestamps of records inside a
> > > > partition to be more out-of-ordered. Are we aware of any scenarios
> > > > such as future enhancements on log compactions that may be affected by
> > > > this effect?
> > > >
> > > > Below are just minor comments:
> > > >
> > > > 5. In "AddPartitionsToTxnTransaction" field of
> > > > "AddPartitionsToTxnRequest" RPC, the versions of those inner fields
> > > > are "0-3" while I thought they should be "0+" still?
> > > >
> > > > 6. Regarding "we can place the request in a purgatory of sorts and
> > > > check if there is any state for the transaction on the broker": i
> > > > think at this time when we just do the checks against the cached
> > > > state, we do not need to put the request to purgatory yet?
> > > >
> > > > 7. This is related to 3) above. I feel using "InvalidRecordException"
> > > > for older clients may also be a bit confusing, and also it is not
> > > > fatal -- for old clients, it better to be fatal since this indicates
> > > > the clients is doing something wrong and hence it should be closed.
> > > > And in general I'd prefer to use slightly more specific meaning error
> > > > codes for clients. That being said, I also feel
> > > > "InvalidProducerEpochException" is not suitable for old versioned
> > > > clients, and we'd have to pick one that old clients recognize. I'd
> > > > prefer "InvalidTxnStateException" but that one is supposed to be
> > > > returned from txn coordinators only today. I'd suggest we do a quick
> > > > check in the current client's code path and see if that one would be
> > > > handled if it's from a produce-response, and if yes, use this one;
> > > > otherwise, use "ProducerFencedException" which is much less meaningful
> > > > but it's still a fatal error.
> > > >
> > > >
> > > > Thanks,
> > > > Guozhang
> > > >
> > > >
> > > >
> > > > On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
> > > > <jo...@confluent.io.invalid> wrote:
> > > > >
> > > > > Yeah -- looks like we already have code to handle bumping the epoch
> > and
> > > > > when the epoch is Short.MAX_VALUE, we get a new producer ID. Since
> > this
> > > > is
> > > > > already the behavior, do we want to change it further?
> > > > >
> > > > > Justine
> > > > >
> > > > > On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <jolshan@confluent.io
> > >
> > > > wrote:
> > > > >
> > > > > > Hey all, just wanted to quickly update and say I've modified the
> > KIP to
> > > > > > explicitly mention that AddOffsetCommitsToTxnRequest will be
> > replaced
> > > > by
> > > > > > a coordinator-side (inter-broker) AddPartitionsToTxn implicit
> > request.
> > > > This
> > > > > > mirrors the user partitions and will implicitly add offset
> > partitions
> > > > to
> > > > > > transactions when we commit offsets on them. We will deprecate
> > > > AddOffsetCommitsToTxnRequest
> > > > > > for new clients.
> > > > > >
> > > > > > Also to address Artem's comments --
> > > > > > I'm a bit unsure if the changes here will change the previous
> > behavior
> > > > for
> > > > > > fencing producers. In the case you mention in the first paragraph,
> > are
> > > > you
> > > > > > saying we bump the epoch before we try to abort the transaction? I
> > > > think I
> > > > > > need to understand the scenarios you mention a bit better.
> > > > > >
> > > > > > As for the second part -- I think it makes sense to have some sort
> > of
> > > > > > "sentinel" epoch to signal epoch is about to overflow (I think we
> > sort
> > > > of
> > > > > > have this value in place in some ways) so we can codify it in the
> > KIP.
> > > > I'll
> > > > > > look into that and try to update soon.
> > > > > >
> > > > > > Thanks,
> > > > > > Justine.
> > > > > >
> > > > > > On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
> > > > > > <al...@confluent.io.invalid> wrote:
> > > > > >
> > > > > >> It's good to know that KIP-588 addressed some of the issues.
> > Looking
> > > > at
> > > > > >> the code, it still looks like there are some cases that would
> > result
> > > > in
> > > > > >> fatal error, e.g. PRODUCER_FENCED is issued by the transaction
> > > > coordinator
> > > > > >> if epoch doesn't match, and the client treats it as a fatal error
> > > > (code in
> > > > > >> TransactionManager request handling).  If we consider, for
> > example,
> > > > > >> committing a transaction that returns a timeout, but actually
> > > > succeeds,
> > > > > >> trying to abort it or re-commit may result in PRODUCER_FENCED
> > error
> > > > > >> (because of epoch bump).
> > > > > >>
> > > > > >> For failed commits, specifically, we need to know the actual
> > outcome,
> > > > > >> because if we return an error the application may think that the
> > > > > >> transaction is aborted and redo the work, leading to duplicates.
> > > > > >>
> > > > > >> Re: overflowing epoch.  We could either do it on the TC and return
> > > > both
> > > > > >> producer id and epoch (e.g. change the protocol), or signal the
> > client
> > > > > >> that
> > > > > >> it needs to get a new producer id.  Checking for max epoch could
> > be a
> > > > > >> reasonable signal, the value to check should probably be present
> > in
> > > > the
> > > > > >> KIP
> > > > > >> as this is effectively a part of the contract.  Also, the TC
> > should
> > > > > >> probably return an error if the client didn't change producer id
> > after
> > > > > >> hitting max epoch.
> > > > > >>
> > > > > >> -Artem
> > > > > >>
> > > > > >>
> > > > > >> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
> > > > > >> <jo...@confluent.io.invalid> wrote:
> > > > > >>
> > > > > >> > Thanks for the discussion Artem.
> > > > > >> >
> > > > > >> > With respect to the handling of fenced producers, we have some
> > > > behavior
> > > > > >> > already in place. As of KIP-588:
> > > > > >> >
> > > > > >> >
> > > > > >>
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> > > > > >> > ,
> > > > > >> > we handle timeouts more gracefully. The producer can recover.
> > > > > >> >
> > > > > >> > Produce requests can also recover from epoch fencing by
> > aborting the
> > > > > >> > transaction and starting over.
> > > > > >> >
> > > > > >> > What other cases were you considering that would cause us to
> > have a
> > > > > >> fenced
> > > > > >> > epoch but we'd want to recover?
> > > > > >> >
> > > > > >> > The first point about handling epoch overflows is fair. I think
> > > > there is
> > > > > >> > some logic we'd need to consider. (ie, if we are one away from
> > the
> > > > max
> > > > > >> > epoch, we need to reset the producer ID.) I'm still wondering if
> > > > there
> > > > > >> is a
> > > > > >> > way to direct this from the response, or if everything should be
> > > > done on
> > > > > >> > the client side. Let me know if you have any thoughts here.
> > > > > >> >
> > > > > >> > Thanks,
> > > > > >> > Justine
> > > > > >> >
> > > > > >> > On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
> > > > > >> > <al...@confluent.io.invalid> wrote:
> > > > > >> >
> > > > > >> > > There are some workflows in the client that are implied by
> > > > protocol
> > > > > >> > > changes, e.g.:
> > > > > >> > >
> > > > > >> > > - for new clients, epoch changes with every transaction and
> > can
> > > > > >> overflow,
> > > > > >> > > in old clients this condition was handled transparently,
> > because
> > > > epoch
> > > > > >> > was
> > > > > >> > > bumped in InitProducerId and it would return a new producer
> > id if
> > > > > >> epoch
> > > > > >> > > overflows, the new clients would need to implement some
> > workflow
> > > > to
> > > > > >> > refresh
> > > > > >> > > producer id
> > > > > >> > > - how to handle fenced producers, for new clients epoch
> > changes
> > > > with
> > > > > >> > every
> > > > > >> > > transaction, so in presence of failures during commits /
> > aborts,
> > > > the
> > > > > >> > > producer could get easily fenced, old clients would pretty
> > much
> > > > would
> > > > > >> get
> > > > > >> > > fenced when a new incarnation of the producer was initialized
> > with
> > > > > >> > > InitProducerId so it's ok to treat as a fatal error, the new
> > > > clients
> > > > > >> > would
> > > > > >> > > need to implement some workflow to handle that error,
> > otherwise
> > > > they
> > > > > >> > could
> > > > > >> > > get fenced by themselves
> > > > > >> > > - in particular (as a subset of the previous issue), what
> > would
> > > > the
> > > > > >> > client
> > > > > >> > > do if it got a timeout during commit?  commit could've
> > succeeded
> > > > or
> > > > > >> > failed
> > > > > >> > >
> > > > > >> > > Not sure if this has to be defined in the KIP as implementing
> > > > those
> > > > > >> > > probably wouldn't require protocol changes, but we have
> > multiple
> > > > > >> > > implementations of Kafka clients, so probably would be good to
> > > > have
> > > > > >> some
> > > > > >> > > client implementation guidance.  Could also be done as a
> > separate
> > > > doc.
> > > > > >> > >
> > > > > >> > > -Artem
> > > > > >> > >
> > > > > >> > > On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
> > > > > >> > <jolshan@confluent.io.invalid
> > > > > >> > > >
> > > > > >> > > wrote:
> > > > > >> > >
> > > > > >> > > > Hey all, I've updated the KIP to incorporate Jason's
> > > > suggestions.
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > > 1. Use AddPartitionsToTxn + verify flag to check on old
> > clients
> > > > > >> > > > 2. Updated AddPartitionsToTxn API to support transaction
> > > > batching
> > > > > >> > > > 3. Mention IBP bump
> > > > > >> > > > 4. Mention auth change on new AddPartitionsToTxn version.
> > > > > >> > > >
> > > > > >> > > > I'm planning on opening a vote soon.
> > > > > >> > > > Thanks,
> > > > > >> > > > Justine
> > > > > >> > > >
> > > > > >> > > > On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <
> > > > jolshan@confluent.io
> > > > > >> >
> > > > > >> > > > wrote:
> > > > > >> > > >
> > > > > >> > > > > Thanks Jason. Those changes make sense to me. I will
> > update
> > > > the
> > > > > >> KIP.
> > > > > >> > > > >
> > > > > >> > > > >
> > > > > >> > > > >
> > > > > >> > > > > On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
> > > > > >> > > > <ja...@confluent.io.invalid>
> > > > > >> > > > > wrote:
> > > > > >> > > > >
> > > > > >> > > > >> Hey Justine,
> > > > > >> > > > >>
> > > > > >> > > > >> > I was wondering about compatibility here. When we send
> > > > requests
> > > > > >> > > > >> between brokers, we want to ensure that the receiving
> > broker
> > > > > >> > > understands
> > > > > >> > > > >> the request (specifically the new fields). Typically
> > this is
> > > > done
> > > > > >> > via
> > > > > >> > > > >> IBP/metadata version.
> > > > > >> > > > >> I'm trying to think if there is a way around it but I'm
> > not
> > > > sure
> > > > > >> > there
> > > > > >> > > > is.
> > > > > >> > > > >>
> > > > > >> > > > >> Yes. I think we would gate usage of this behind an IBP
> > bump.
> > > > Does
> > > > > >> > that
> > > > > >> > > > >> seem
> > > > > >> > > > >> reasonable?
> > > > > >> > > > >>
> > > > > >> > > > >> > As for the improvements -- can you clarify how the
> > multiple
> > > > > >> > > > >> transactional
> > > > > >> > > > >> IDs would help here? Were you thinking of a case where we
> > > > > >> wait/batch
> > > > > >> > > > >> multiple produce requests together? My understanding for
> > now
> > > > was
> > > > > >> 1
> > > > > >> > > > >> transactional ID and one validation per 1 produce
> > request.
> > > > > >> > > > >>
> > > > > >> > > > >> Each call to `AddPartitionsToTxn` is essentially a write
> > to
> > > > the
> > > > > >> > > > >> transaction
> > > > > >> > > > >> log and must block on replication. The more we can fit
> > into a
> > > > > >> single
> > > > > >> > > > >> request, the more writes we can do in parallel. The
> > > > alternative
> > > > > >> is
> > > > > >> > to
> > > > > >> > > > make
> > > > > >> > > > >> use of more connections, but usually we prefer batching
> > > > since the
> > > > > >> > > > network
> > > > > >> > > > >> stack is not really optimized for high connection/request
> > > > loads.
> > > > > >> > > > >>
> > > > > >> > > > >> > Finally with respect to the authorizations, I think it
> > > > makes
> > > > > >> sense
> > > > > >> > > to
> > > > > >> > > > >> skip
> > > > > >> > > > >> topic authorizations, but I'm a bit confused by the
> > "leader
> > > > ID"
> > > > > >> > field.
> > > > > >> > > > >> Wouldn't we just want to flag the request as from a
> > broker
> > > > (does
> > > > > >> it
> > > > > >> > > > matter
> > > > > >> > > > >> which one?).
> > > > > >> > > > >>
> > > > > >> > > > >> We could also make it version-based. For the next
> > version, we
> > > > > >> could
> > > > > >> > > > >> require
> > > > > >> > > > >> CLUSTER auth. So clients would not be able to use the API
> > > > > >> anymore,
> > > > > >> > > which
> > > > > >> > > > >> is
> > > > > >> > > > >> probably what we want.
> > > > > >> > > > >>
> > > > > >> > > > >> -Jason
> > > > > >> > > > >>
> > > > > >> > > > >> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> > > > > >> > > > >> <jo...@confluent.io.invalid>
> > > > > >> > > > >> wrote:
> > > > > >> > > > >>
> > > > > >> > > > >> > As a follow up, I was just thinking about the batching
> > a
> > > > bit
> > > > > >> more.
> > > > > >> > > > >> > I suppose if we have one request in flight and we
> > queue up
> > > > the
> > > > > >> > other
> > > > > >> > > > >> > produce requests in some sort of purgatory, we could
> > send
> > > > > >> > > information
> > > > > >> > > > >> out
> > > > > >> > > > >> > for all of them rather than one by one. So that would
> > be a
> > > > > >> benefit
> > > > > >> > > of
> > > > > >> > > > >> > batching partitions to add per transaction.
> > > > > >> > > > >> >
> > > > > >> > > > >> > I'll need to think a bit more on the design of this
> > part
> > > > of the
> > > > > >> > KIP,
> > > > > >> > > > and
> > > > > >> > > > >> > will update the KIP in the next few days.
> > > > > >> > > > >> >
> > > > > >> > > > >> > Thanks,
> > > > > >> > > > >> > Justine
> > > > > >> > > > >> >
> > > > > >> > > > >> > On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan <
> > > > > >> > > jolshan@confluent.io>
> > > > > >> > > > >> > wrote:
> > > > > >> > > > >> >
> > > > > >> > > > >> > > Hey Jason -- thanks for the input -- I was just
> > digging
> > > > a bit
> > > > > >> > > deeper
> > > > > >> > > > >> into
> > > > > >> > > > >> > > the design + implementation of the validation calls
> > here
> > > > and
> > > > > >> > what
> > > > > >> > > > you
> > > > > >> > > > >> say
> > > > > >> > > > >> > > makes sense.
> > > > > >> > > > >> > >
> > > > > >> > > > >> > > I was wondering about compatibility here. When we
> > send
> > > > > >> requests
> > > > > >> > > > >> > > between brokers, we want to ensure that the receiving
> > > > broker
> > > > > >> > > > >> understands
> > > > > >> > > > >> > > the request (specifically the new fields). Typically
> > > > this is
> > > > > >> > done
> > > > > >> > > > via
> > > > > >> > > > >> > > IBP/metadata version.
> > > > > >> > > > >> > > I'm trying to think if there is a way around it but
> > I'm
> > > > not
> > > > > >> sure
> > > > > >> > > > there
> > > > > >> > > > >> > is.
> > > > > >> > > > >> > >
> > > > > >> > > > >> > > As for the improvements -- can you clarify how the
> > > > multiple
> > > > > >> > > > >> transactional
> > > > > >> > > > >> > > IDs would help here? Were you thinking of a case
> > where we
> > > > > >> > > wait/batch
> > > > > >> > > > >> > > multiple produce requests together? My understanding
> > for
> > > > now
> > > > > >> > was 1
> > > > > >> > > > >> > > transactional ID and one validation per 1 produce
> > > > request.
> > > > > >> > > > >> > >
> > > > > >> > > > >> > > Finally with respect to the authorizations, I think
> > it
> > > > makes
> > > > > >> > sense
> > > > > >> > > > to
> > > > > >> > > > >> > skip
> > > > > >> > > > >> > > topic authorizations, but I'm a bit confused by the
> > > > "leader
> > > > > >> ID"
> > > > > >> > > > field.
> > > > > >> > > > >> > > Wouldn't we just want to flag the request as from a
> > > > broker
> > > > > >> (does
> > > > > >> > > it
> > > > > >> > > > >> > matter
> > > > > >> > > > >> > > which one?).
> > > > > >> > > > >> > >
> > > > > >> > > > >> > > I think I want to adopt these suggestions, just had
> > a few
> > > > > >> > > questions
> > > > > >> > > > on
> > > > > >> > > > >> > the
> > > > > >> > > > >> > > details.
> > > > > >> > > > >> > >
> > > > > >> > > > >> > > Thanks,
> > > > > >> > > > >> > > Justine
> > > > > >> > > > >> > >
> > > > > >> > > > >> > > On Thu, Jan 5, 2023 at 5:05 PM Jason Gustafson
> > > > > >> > > > >> > <ja...@confluent.io.invalid>
> > > > > >> > > > >> > > wrote:
> > > > > >> > > > >> > >
> > > > > >> > > > >> > >> Hi Justine,
> > > > > >> > > > >> > >>
> > > > > >> > > > >> > >> Thanks for the proposal.
> > > > > >> > > > >> > >>
> > > > > >> > > > >> > >> I was thinking about the implementation a little
> > bit.
> > > > In the
> > > > > >> > > > current
> > > > > >> > > > >> > >> proposal, the behavior depends on whether we have an
> > > > old or
> > > > > >> new
> > > > > >> > > > >> client.
> > > > > >> > > > >> > >> For
> > > > > >> > > > >> > >> old clients, we send `DescribeTransactions` and
> > verify
> > > > the
> > > > > >> > result
> > > > > >> > > > and
> > > > > >> > > > >> > for
> > > > > >> > > > >> > >> new clients, we send `AddPartitionsToTxn`. We might
> > be
> > > > able
> > > > > >> to
> > > > > >> > > > >> simplify
> > > > > >> > > > >> > >> the
> > > > > >> > > > >> > >> implementation if we can use the same request type.
> > For
> > > > > >> > example,
> > > > > >> > > > >> what if
> > > > > >> > > > >> > >> we
> > > > > >> > > > >> > >> bump the protocol version for `AddPartitionsToTxn`
> > and
> > > > add a
> > > > > >> > > > >> > >> `validateOnly`
> > > > > >> > > > >> > >> flag? For older versions, we can set
> > > > `validateOnly=true` so
> > > > > >> > that
> > > > > >> > > > the
> > > > > >> > > > >> > >> request only returns successfully if the partition
> > had
> > > > > >> already
> > > > > >> > > been
> > > > > >> > > > >> > added.
> > > > > >> > > > >> > >> For new versions, we can set `validateOnly=false`
> > and
> > > > the
> > > > > >> > > partition
> > > > > >> > > > >> will
> > > > > >> > > > >> > >> be
> > > > > >> > > > >> > >> added to the transaction. The other slightly
> > annoying
> > > > thing
> > > > > >> > that
> > > > > >> > > > this
> > > > > >> > > > >> > >> would
> > > > > >> > > > >> > >> get around is the need to collect the transaction
> > state
> > > > for
> > > > > >> all
> > > > > >> > > > >> > partitions
> > > > > >> > > > >> > >> even when we only care about a subset.
> > > > > >> > > > >> > >>
> > > > > >> > > > >> > >> Some additional improvements to consider:
> > > > > >> > > > >> > >>
> > > > > >> > > > >> > >> - We can give `AddPartitionsToTxn` better batch
> > support
> > > > for
> > > > > >> > > > >> inter-broker
> > > > > >> > > > >> > >> usage. Currently we only allow one
> > `TransactionalId` to
> > > > be
> > > > > >> > > > specified,
> > > > > >> > > > >> > but
> > > > > >> > > > >> > >> the broker may get some benefit being able to batch
> > > > across
> > > > > >> > > multiple
> > > > > >> > > > >> > >> transactions.
> > > > > >> > > > >> > >> - Another small improvement is skipping topic
> > > > authorization
> > > > > >> > > checks
> > > > > >> > > > >> for
> > > > > >> > > > >> > >> `AddPartitionsToTxn` when the request is from a
> > broker.
> > > > > >> Perhaps
> > > > > >> > > we
> > > > > >> > > > >> can
> > > > > >> > > > >> > add
> > > > > >> > > > >> > >> a field for the `LeaderId` or something like that
> > and
> > > > > >> require
> > > > > >> > > > CLUSTER
> > > > > >> > > > >> > >> permission when set.
> > > > > >> > > > >> > >>
> > > > > >> > > > >> > >> Best,
> > > > > >> > > > >> > >> Jason
> > > > > >> > > > >> > >>
> > > > > >> > > > >> > >>
> > > > > >> > > > >> > >>
> > > > > >> > > > >> > >> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
> > > > > >> > <jun@confluent.io.invalid
> > > > > >> > > >
> > > > > >> > > > >> > wrote:
> > > > > >> > > > >> > >>
> > > > > >> > > > >> > >> > Hi, Justine,
> > > > > >> > > > >> > >> >
> > > > > >> > > > >> > >> > Thanks for the explanation. It makes sense to me
> > now.
> > > > > >> > > > >> > >> >
> > > > > >> > > > >> > >> > Jun
> > > > > >> > > > >> > >> >
> > > > > >> > > > >> > >> > On Mon, Dec 19, 2022 at 1:42 PM Justine Olshan
> > > > > >> > > > >> > >> > <jo...@confluent.io.invalid>
> > > > > >> > > > >> > >> > wrote:
> > > > > >> > > > >> > >> >
> > > > > >> > > > >> > >> > > Hi Jun,
> > > > > >> > > > >> > >> > >
> > > > > >> > > > >> > >> > > My understanding of the mechanism is that when
> > we
> > > > get to
> > > > > >> > the
> > > > > >> > > > last
> > > > > >> > > > >> > >> epoch,
> > > > > >> > > > >> > >> > we
> > > > > >> > > > >> > >> > > increment to the fencing/last epoch and if any
> > > > further
> > > > > >> > > requests
> > > > > >> > > > >> come
> > > > > >> > > > >> > >> in
> > > > > >> > > > >> > >> > for
> > > > > >> > > > >> > >> > > this producer ID they are fenced. Then the
> > producer
> > > > > >> gets a
> > > > > >> > > new
> > > > > >> > > > ID
> > > > > >> > > > >> > and
> > > > > >> > > > >> > >> > > restarts with epoch/sequence 0. The fenced epoch
> > > > sticks
> > > > > >> > > around
> > > > > >> > > > >> for
> > > > > >> > > > >> > the
> > > > > >> > > > >> > >> > > duration of producer.id.expiration.ms and
> > blocks
> > > > any
> > > > > >> late
> > > > > >> > > > >> messages
> > > > > >> > > > >> > >> > there.
> > > > > >> > > > >> > >> > > The new ID will get to take advantage of the
> > > > improved
> > > > > >> > > semantics
> > > > > >> > > > >> > around
> > > > > >> > > > >> > >> > > non-zero start sequences. So I think we are
> > covered.
> > > > > >> > > > >> > >> > >
> > > > > >> > > > >> > >> > > The only potential issue is overloading the
> > cache,
> > > > but
> > > > > >> > > > hopefully
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> > > improvements (lowered producer.id.expiration.ms
> > )
> > > > will
> > > > > >> help
> > > > > >> > > > with
> > > > > >> > > > >> > that.
> > > > > >> > > > >> > >> > Let
> > > > > >> > > > >> > >> > > me know if you still have concerns.
> > > > > >> > > > >> > >> > >
> > > > > >> > > > >> > >> > > Thanks,
> > > > > >> > > > >> > >> > > Justine
> > > > > >> > > > >> > >> > >
> > > > > >> > > > >> > >> > > On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
> > > > > >> > > > >> <ju...@confluent.io.invalid>
> > > > > >> > > > >> > >> > wrote:
> > > > > >> > > > >> > >> > >
> > > > > >> > > > >> > >> > > > Hi, Justine,
> > > > > >> > > > >> > >> > > >
> > > > > >> > > > >> > >> > > > Thanks for the explanation.
> > > > > >> > > > >> > >> > > >
> > > > > >> > > > >> > >> > > > 70. The proposed fencing logic doesn't apply
> > when
> > > > pid
> > > > > >> > > > changes,
> > > > > >> > > > >> is
> > > > > >> > > > >> > >> that
> > > > > >> > > > >> > >> > > > right? If so, I am not sure how complete we
> > are
> > > > > >> > addressing
> > > > > >> > > > this
> > > > > >> > > > >> > >> issue
> > > > > >> > > > >> > >> > if
> > > > > >> > > > >> > >> > > > the pid changes more frequently.
> > > > > >> > > > >> > >> > > >
> > > > > >> > > > >> > >> > > > Thanks,
> > > > > >> > > > >> > >> > > >
> > > > > >> > > > >> > >> > > > Jun
> > > > > >> > > > >> > >> > > >
> > > > > >> > > > >> > >> > > >
> > > > > >> > > > >> > >> > > >
> > > > > >> > > > >> > >> > > > On Fri, Dec 16, 2022 at 9:16 AM Justine Olshan
> > > > > >> > > > >> > >> > > > <jo...@confluent.io.invalid>
> > > > > >> > > > >> > >> > > > wrote:
> > > > > >> > > > >> > >> > > >
> > > > > >> > > > >> > >> > > > > Hi Jun,
> > > > > >> > > > >> > >> > > > >
> > > > > >> > > > >> > >> > > > > Thanks for replying!
> > > > > >> > > > >> > >> > > > >
> > > > > >> > > > >> > >> > > > > 70.We already do the overflow mechanism, so
> > my
> > > > > >> change
> > > > > >> > > would
> > > > > >> > > > >> just
> > > > > >> > > > >> > >> make
> > > > > >> > > > >> > >> > > it
> > > > > >> > > > >> > >> > > > > happen more often.
> > > > > >> > > > >> > >> > > > > I was also not suggesting a new field in the
> > > > log,
> > > > > >> but
> > > > > >> > in
> > > > > >> > > > the
> > > > > >> > > > >> > >> > response,
> > > > > >> > > > >> > >> > > > > which would be gated by the client version.
> > > > Sorry if
> > > > > >> > > > >> something
> > > > > >> > > > >> > >> there
> > > > > >> > > > >> > >> > is
> > > > > >> > > > >> > >> > > > > unclear. I think we are starting to diverge.
> > > > > >> > > > >> > >> > > > > The goal of this KIP is to not change to the
> > > > marker
> > > > > >> > > format
> > > > > >> > > > at
> > > > > >> > > > >> > all.
> > > > > >> > > > >> > >> > > > >
> > > > > >> > > > >> > >> > > > > 71. Yes, I guess I was going under the
> > > > assumption
> > > > > >> that
> > > > > >> > > the
> > > > > >> > > > >> log
> > > > > >> > > > >> > >> would
> > > > > >> > > > >> > >> > > just
> > > > > >> > > > >> > >> > > > > look at its last epoch and treat it as the
> > > > current
> > > > > >> > > epoch. I
> > > > > >> > > > >> > >> suppose
> > > > > >> > > > >> > >> > we
> > > > > >> > > > >> > >> > > > can
> > > > > >> > > > >> > >> > > > > have some special logic that if the last
> > epoch
> > > > was
> > > > > >> on a
> > > > > >> > > > >> marker
> > > > > >> > > > >> > we
> > > > > >> > > > >> > >> > > > actually
> > > > > >> > > > >> > >> > > > > expect the next epoch or something like
> > that. We
> > > > > >> just
> > > > > >> > > need
> > > > > >> > > > to
> > > > > >> > > > >> > >> > > distinguish
> > > > > >> > > > >> > >> > > > > based on whether we had a commit/abort
> > marker.
> > > > > >> > > > >> > >> > > > >
> > > > > >> > > > >> > >> > > > > 72.
> > > > > >> > > > >> > >> > > > > > if the producer epoch hasn't been bumped
> > on
> > > > the
> > > > > >> > > > >> > >> > > > > broker, it seems that the stucked message
> > will
> > > > fail
> > > > > >> the
> > > > > >> > > > >> sequence
> > > > > >> > > > >> > >> > > > validation
> > > > > >> > > > >> > >> > > > > and will be ignored. If the producer epoch
> > has
> > > > been
> > > > > >> > > bumped,
> > > > > >> > > > >> we
> > > > > >> > > > >> > >> ignore
> > > > > >> > > > >> > >> > > the
> > > > > >> > > > >> > >> > > > > sequence check and the stuck message could
> > be
> > > > > >> appended
> > > > > >> > to
> > > > > >> > > > the
> > > > > >> > > > >> > log.
> > > > > >> > > > >> > >> > So,
> > > > > >> > > > >> > >> > > is
> > > > > >> > > > >> > >> > > > > the latter case that we want to guard?
> > > > > >> > > > >> > >> > > > >
> > > > > >> > > > >> > >> > > > > I'm not sure I follow that "the message will
> > > > fail
> > > > > >> the
> > > > > >> > > > >> sequence
> > > > > >> > > > >> > >> > > > validation".
> > > > > >> > > > >> > >> > > > > In some of these cases, we had an abort
> > marker
> > > > (due
> > > > > >> to
> > > > > >> > an
> > > > > >> > > > >> error)
> > > > > >> > > > >> > >> and
> > > > > >> > > > >> > >> > > then
> > > > > >> > > > >> > >> > > > > the late message comes in with the correct
> > > > sequence
> > > > > >> > > number.
> > > > > >> > > > >> This
> > > > > >> > > > >> > >> is a
> > > > > >> > > > >> > >> > > > case
> > > > > >> > > > >> > >> > > > > covered by the KIP.
> > > > > >> > > > >> > >> > > > > The latter case is actually not something
> > we've
> > > > > >> > > considered
> > > > > >> > > > >> > here. I
> > > > > >> > > > >> > >> > > think
> > > > > >> > > > >> > >> > > > > generally when we bump the epoch, we are
> > > > accepting
> > > > > >> that
> > > > > >> > > the
> > > > > >> > > > >> > >> sequence
> > > > > >> > > > >> > >> > > does
> > > > > >> > > > >> > >> > > > > not need to be checked anymore. My
> > > > understanding is
> > > > > >> > also
> > > > > >> > > > >> that we
> > > > > >> > > > >> > >> > don't
> > > > > >> > > > >> > >> > > > > typically bump epoch mid transaction (based
> > on a
> > > > > >> quick
> > > > > >> > > look
> > > > > >> > > > >> at
> > > > > >> > > > >> > the
> > > > > >> > > > >> > >> > > code)
> > > > > >> > > > >> > >> > > > > but let me know if that is the case.
> > > > > >> > > > >> > >> > > > >
> > > > > >> > > > >> > >> > > > > Thanks,
> > > > > >> > > > >> > >> > > > > Justine
> > > > > >> > > > >> > >> > > > >
> > > > > >> > > > >> > >> > > > > On Thu, Dec 15, 2022 at 12:23 PM Jun Rao
> > > > > >> > > > >> > <jun@confluent.io.invalid
> > > > > >> > > > >> > >> >
> > > > > >> > > > >> > >> > > > wrote:
> > > > > >> > > > >> > >> > > > >
> > > > > >> > > > >> > >> > > > > > Hi, Justine,
> > > > > >> > > > >> > >> > > > > >
> > > > > >> > > > >> > >> > > > > > Thanks for the reply.
> > > > > >> > > > >> > >> > > > > >
> > > > > >> > > > >> > >> > > > > > 70. Assigning a new pid on int overflow
> > seems
> > > > a
> > > > > >> bit
> > > > > >> > > > hacky.
> > > > > >> > > > >> If
> > > > > >> > > > >> > we
> > > > > >> > > > >> > >> > > need a
> > > > > >> > > > >> > >> > > > > txn
> > > > > >> > > > >> > >> > > > > > level id, it will be better to model this
> > > > > >> explicitly.
> > > > > >> > > > >> Adding a
> > > > > >> > > > >> > >> new
> > > > > >> > > > >> > >> > > > field
> > > > > >> > > > >> > >> > > > > > would require a bit more work since it
> > > > requires a
> > > > > >> new
> > > > > >> > > txn
> > > > > >> > > > >> > marker
> > > > > >> > > > >> > >> > > format
> > > > > >> > > > >> > >> > > > > in
> > > > > >> > > > >> > >> > > > > > the log. So, we probably need to guard it
> > > > with an
> > > > > >> IBP
> > > > > >> > > or
> > > > > >> > > > >> > >> metadata
> > > > > >> > > > >> > >> > > > version
> > > > > >> > > > >> > >> > > > > > and document the impact on downgrade once
> > the
> > > > new
> > > > > >> > > format
> > > > > >> > > > is
> > > > > >> > > > >> > >> written
> > > > > >> > > > >> > >> > > to
> > > > > >> > > > >> > >> > > > > the
> > > > > >> > > > >> > >> > > > > > log.
> > > > > >> > > > >> > >> > > > > >
> > > > > >> > > > >> > >> > > > > > 71. Hmm, once the marker is written, the
> > > > partition
> > > > > >> > will
> > > > > >> > > > >> expect
> > > > > >> > > > >> > >> the
> > > > > >> > > > >> > >> > > next
> > > > > >> > > > >> > >> > > > > > append to be on the next epoch. Does that
> > > > cover
> > > > > >> the
> > > > > >> > > case
> > > > > >> > > > >> you
> > > > > >> > > > >> > >> > > mentioned?
> > > > > >> > > > >> > >> > > > > >
> > > > > >> > > > >> > >> > > > > > 72. Also, just to be clear on the stucked
> > > > message
> > > > > >> > issue
> > > > > >> > > > >> > >> described
> > > > > >> > > > >> > >> > in
> > > > > >> > > > >> > >> > > > the
> > > > > >> > > > >> > >> > > > > > motivation. With EoS, we also validate the
> > > > > >> sequence
> > > > > >> > id
> > > > > >> > > > for
> > > > > >> > > > >> > >> > > idempotency.
> > > > > >> > > > >> > >> > > > > So,
> > > > > >> > > > >> > >> > > > > > with the current logic, if the producer
> > epoch
> > > > > >> hasn't
> > > > > >> > > been
> > > > > >> > > > >> > >> bumped on
> > > > > >> > > > >> > >> > > the
> > > > > >> > > > >> > >> > > > > > broker, it seems that the stucked message
> > will
> > > > > >> fail
> > > > > >> > the
> > > > > >> > > > >> > sequence
> > > > > >> > > > >> > >> > > > > validation
> > > > > >> > > > >> > >> > > > > > and will be ignored. If the producer
> > epoch has
> > > > > >> been
> > > > > >> > > > >> bumped, we
> > > > > >> > > > >> > >> > ignore
> > > > > >> > > > >> > >> > > > the
> > > > > >> > > > >> > >> > > > > > sequence check and the stuck message
> > could be
> > > > > >> > appended
> > > > > >> > > to
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> log.
> > > > > >> > > > >> > >> > > So,
> > > > > >> > > > >> > >> > > > is
> > > > > >> > > > >> > >> > > > > > the latter case that we want to guard?
> > > > > >> > > > >> > >> > > > > >
> > > > > >> > > > >> > >> > > > > > Thanks,
> > > > > >> > > > >> > >> > > > > >
> > > > > >> > > > >> > >> > > > > > Jun
> > > > > >> > > > >> > >> > > > > >
> > > > > >> > > > >> > >> > > > > > On Wed, Dec 14, 2022 at 10:44 AM Justine
> > > > Olshan
> > > > > >> > > > >> > >> > > > > > <jo...@confluent.io.invalid> wrote:
> > > > > >> > > > >> > >> > > > > >
> > > > > >> > > > >> > >> > > > > > > Matthias — thanks again for taking time
> > to
> > > > look
> > > > > >> a
> > > > > >> > > this.
> > > > > >> > > > >> You
> > > > > >> > > > >> > >> said:
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > > My proposal was only focusing to avoid
> > > > > >> dangling
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > transactions if records are added
> > without
> > > > > >> > registered
> > > > > >> > > > >> > >> partition.
> > > > > >> > > > >> > >> > --
> > > > > >> > > > >> > >> > > > > Maybe
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > you can add a few more details to the
> > KIP
> > > > about
> > > > > >> > this
> > > > > >> > > > >> > scenario
> > > > > >> > > > >> > >> for
> > > > > >> > > > >> > >> > > > > better
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > documentation purpose?
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > I'm not sure I understand what you mean
> > > > here.
> > > > > >> The
> > > > > >> > > > >> motivation
> > > > > >> > > > >> > >> > > section
> > > > > >> > > > >> > >> > > > > > > describes two scenarios about how the
> > record
> > > > > >> can be
> > > > > >> > > > added
> > > > > >> > > > >> > >> > without a
> > > > > >> > > > >> > >> > > > > > > registered partition:
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > > 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.
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > > Another way hanging transactions can
> > > > occur is
> > > > > >> > that
> > > > > >> > > a
> > > > > >> > > > >> > client
> > > > > >> > > > >> > >> is
> > > > > >> > > > >> > >> > > > buggy
> > > > > >> > > > >> > >> > > > > > and
> > > > > >> > > > >> > >> > > > > > > may somehow try to write to a partition
> > > > before
> > > > > >> it
> > > > > >> > > adds
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> > > partition
> > > > > >> > > > >> > >> > > > to
> > > > > >> > > > >> > >> > > > > > the
> > > > > >> > > > >> > >> > > > > > > transaction.
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > For the first example of this would it
> > be
> > > > > >> helpful
> > > > > >> > to
> > > > > >> > > > say
> > > > > >> > > > >> > that
> > > > > >> > > > >> > >> > this
> > > > > >> > > > >> > >> > > > > > message
> > > > > >> > > > >> > >> > > > > > > comes in after the abort, but before the
> > > > > >> partition
> > > > > >> > is
> > > > > >> > > > >> added
> > > > > >> > > > >> > to
> > > > > >> > > > >> > >> > the
> > > > > >> > > > >> > >> > > > next
> > > > > >> > > > >> > >> > > > > > > transaction so it becomes "hanging."
> > > > Perhaps the
> > > > > >> > next
> > > > > >> > > > >> > sentence
> > > > > >> > > > >> > >> > > > > describing
> > > > > >> > > > >> > >> > > > > > > the message becoming part of the next
> > > > > >> transaction
> > > > > >> > (a
> > > > > >> > > > >> > different
> > > > > >> > > > >> > >> > > case)
> > > > > >> > > > >> > >> > > > > was
> > > > > >> > > > >> > >> > > > > > > not properly differentiated.
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > Jun — thanks for reading the KIP.
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > 70. The int typing was a concern.
> > Currently
> > > > we
> > > > > >> > have a
> > > > > >> > > > >> > >> mechanism
> > > > > >> > > > >> > >> > in
> > > > > >> > > > >> > >> > > > > place
> > > > > >> > > > >> > >> > > > > > to
> > > > > >> > > > >> > >> > > > > > > fence the final epoch when the epoch is
> > > > about to
> > > > > >> > > > overflow
> > > > > >> > > > >> > and
> > > > > >> > > > >> > >> > > assign
> > > > > >> > > > >> > >> > > > a
> > > > > >> > > > >> > >> > > > > > new
> > > > > >> > > > >> > >> > > > > > > producer ID with epoch 0. Of course,
> > this
> > > > is a
> > > > > >> bit
> > > > > >> > > > tricky
> > > > > >> > > > >> > >> when it
> > > > > >> > > > >> > >> > > > comes
> > > > > >> > > > >> > >> > > > > > to
> > > > > >> > > > >> > >> > > > > > > the response back to the client.
> > > > > >> > > > >> > >> > > > > > > Making this a long could be another
> > option,
> > > > but
> > > > > >> I
> > > > > >> > > > wonder
> > > > > >> > > > >> are
> > > > > >> > > > >> > >> > there
> > > > > >> > > > >> > >> > > > any
> > > > > >> > > > >> > >> > > > > > > implications on changing this field if
> > the
> > > > > >> epoch is
> > > > > >> > > > >> > persisted
> > > > > >> > > > >> > >> to
> > > > > >> > > > >> > >> > > > disk?
> > > > > >> > > > >> > >> > > > > > I'd
> > > > > >> > > > >> > >> > > > > > > need to check the usages.
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > 71.This was something Matthias asked
> > about
> > > > as
> > > > > >> > well. I
> > > > > >> > > > was
> > > > > >> > > > >> > >> > > > considering a
> > > > > >> > > > >> > >> > > > > > > possible edge case where a produce
> > request
> > > > from
> > > > > >> a
> > > > > >> > new
> > > > > >> > > > >> > >> transaction
> > > > > >> > > > >> > >> > > > > somehow
> > > > > >> > > > >> > >> > > > > > > gets sent right after the marker is
> > > > written, but
> > > > > >> > > before
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> > > producer
> > > > > >> > > > >> > >> > > > is
> > > > > >> > > > >> > >> > > > > > > alerted of the newly bumped epoch. In
> > this
> > > > > >> case, we
> > > > > >> > > may
> > > > > >> > > > >> > >> include
> > > > > >> > > > >> > >> > > this
> > > > > >> > > > >> > >> > > > > > record
> > > > > >> > > > >> > >> > > > > > > when we don't want to. I suppose we
> > could
> > > > try
> > > > > >> to do
> > > > > >> > > > >> > something
> > > > > >> > > > >> > >> > > client
> > > > > >> > > > >> > >> > > > > side
> > > > > >> > > > >> > >> > > > > > > to bump the epoch after sending an
> > endTxn as
> > > > > >> well
> > > > > >> > in
> > > > > >> > > > this
> > > > > >> > > > >> > >> > scenario
> > > > > >> > > > >> > >> > > —
> > > > > >> > > > >> > >> > > > > but
> > > > > >> > > > >> > >> > > > > > I
> > > > > >> > > > >> > >> > > > > > > wonder how it would work when the
> > server is
> > > > > >> > aborting
> > > > > >> > > > >> based
> > > > > >> > > > >> > on
> > > > > >> > > > >> > >> a
> > > > > >> > > > >> > >> > > > > > server-side
> > > > > >> > > > >> > >> > > > > > > error. I could also be missing
> > something and
> > > > > >> this
> > > > > >> > > > >> scenario
> > > > > >> > > > >> > is
> > > > > >> > > > >> > >> > > > actually
> > > > > >> > > > >> > >> > > > > > not
> > > > > >> > > > >> > >> > > > > > > possible.
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > Thanks again to everyone reading and
> > > > commenting.
> > > > > >> > Let
> > > > > >> > > me
> > > > > >> > > > >> know
> > > > > >> > > > >> > >> > about
> > > > > >> > > > >> > >> > > > any
> > > > > >> > > > >> > >> > > > > > > further questions or comments.
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > Justine
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > On Wed, Dec 14, 2022 at 9:41 AM Jun Rao
> > > > > >> > > > >> > >> <jun@confluent.io.invalid
> > > > > >> > > > >> > >> > >
> > > > > >> > > > >> > >> > > > > > wrote:
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > > Hi, Justine,
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > Thanks for the KIP. A couple of
> > comments.
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > 70. Currently, the producer epoch is
> > an
> > > > int.
> > > > > >> I am
> > > > > >> > > not
> > > > > >> > > > >> sure
> > > > > >> > > > >> > >> if
> > > > > >> > > > >> > >> > > it's
> > > > > >> > > > >> > >> > > > > > enough
> > > > > >> > > > >> > >> > > > > > > > to accommodate all transactions in the
> > > > > >> lifetime
> > > > > >> > of
> > > > > >> > > a
> > > > > >> > > > >> > >> producer.
> > > > > >> > > > >> > >> > > > Should
> > > > > >> > > > >> > >> > > > > > we
> > > > > >> > > > >> > >> > > > > > > > change that to a long or add a new
> > long
> > > > field
> > > > > >> > like
> > > > > >> > > > >> txnId?
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > 71. "it will write the prepare commit
> > > > message
> > > > > >> > with
> > > > > >> > > a
> > > > > >> > > > >> > bumped
> > > > > >> > > > >> > >> > epoch
> > > > > >> > > > >> > >> > > > and
> > > > > >> > > > >> > >> > > > > > > send
> > > > > >> > > > >> > >> > > > > > > > WriteTxnMarkerRequests with the bumped
> > > > epoch."
> > > > > >> > Hmm,
> > > > > >> > > > the
> > > > > >> > > > >> > >> epoch
> > > > > >> > > > >> > >> > is
> > > > > >> > > > >> > >> > > > > > > associated
> > > > > >> > > > >> > >> > > > > > > > with the current txn right? So, it
> > seems
> > > > > >> weird to
> > > > > >> > > > >> write a
> > > > > >> > > > >> > >> > commit
> > > > > >> > > > >> > >> > > > > > message
> > > > > >> > > > >> > >> > > > > > > > with a bumped epoch. Should we only
> > bump
> > > > up
> > > > > >> the
> > > > > >> > > epoch
> > > > > >> > > > >> in
> > > > > >> > > > >> > >> > > > > EndTxnResponse
> > > > > >> > > > >> > >> > > > > > > and
> > > > > >> > > > >> > >> > > > > > > > rename the field to sth like
> > > > > >> nextProducerEpoch?
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > Thanks,
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > Jun
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > On Mon, Dec 12, 2022 at 8:54 PM
> > Matthias
> > > > J.
> > > > > >> Sax <
> > > > > >> > > > >> > >> > > mjsax@apache.org>
> > > > > >> > > > >> > >> > > > > > > wrote:
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > Thanks for the background.
> > > > > >> > > > >> > >> > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > 20/30: SGTM. My proposal was only
> > > > focusing
> > > > > >> to
> > > > > >> > > avoid
> > > > > >> > > > >> > >> dangling
> > > > > >> > > > >> > >> > > > > > > > > transactions if records are added
> > > > without
> > > > > >> > > > registered
> > > > > >> > > > >> > >> > partition.
> > > > > >> > > > >> > >> > > > --
> > > > > >> > > > >> > >> > > > > > > Maybe
> > > > > >> > > > >> > >> > > > > > > > > you can add a few more details to
> > the
> > > > KIP
> > > > > >> about
> > > > > >> > > > this
> > > > > >> > > > >> > >> scenario
> > > > > >> > > > >> > >> > > for
> > > > > >> > > > >> > >> > > > > > > better
> > > > > >> > > > >> > >> > > > > > > > > documentation purpose?
> > > > > >> > > > >> > >> > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > 40: I think you hit a fair point
> > about
> > > > race
> > > > > >> > > > >> conditions
> > > > > >> > > > >> > or
> > > > > >> > > > >> > >> > > client
> > > > > >> > > > >> > >> > > > > bugs
> > > > > >> > > > >> > >> > > > > > > > > (incorrectly not bumping the
> > epoch). The
> > > > > >> > > > >> > >> complexity/confusion
> > > > > >> > > > >> > >> > > for
> > > > > >> > > > >> > >> > > > > > using
> > > > > >> > > > >> > >> > > > > > > > > the bumped epoch I see, is mainly
> > for
> > > > > >> internal
> > > > > >> > > > >> > debugging,
> > > > > >> > > > >> > >> ie,
> > > > > >> > > > >> > >> > > > > > > inspecting
> > > > > >> > > > >> > >> > > > > > > > > log segment dumps -- it seems
> > harder to
> > > > > >> reason
> > > > > >> > > > about
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> > system
> > > > > >> > > > >> > >> > > > for
> > > > > >> > > > >> > >> > > > > > us
> > > > > >> > > > >> > >> > > > > > > > > humans. But if we get better
> > > > guarantees, it
> > > > > >> > would
> > > > > >> > > > be
> > > > > >> > > > >> > >> worth to
> > > > > >> > > > >> > >> > > use
> > > > > >> > > > >> > >> > > > > the
> > > > > >> > > > >> > >> > > > > > > > > bumped epoch.
> > > > > >> > > > >> > >> > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > 60: as I mentioned already, I don't
> > > > know the
> > > > > >> > > broker
> > > > > >> > > > >> > >> internals
> > > > > >> > > > >> > >> > > to
> > > > > >> > > > >> > >> > > > > > > provide
> > > > > >> > > > >> > >> > > > > > > > > more input. So if nobody else chimes
> > > > in, we
> > > > > >> > > should
> > > > > >> > > > >> just
> > > > > >> > > > >> > >> move
> > > > > >> > > > >> > >> > > > > forward
> > > > > >> > > > >> > >> > > > > > > > > with your proposal.
> > > > > >> > > > >> > >> > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > -Matthias
> > > > > >> > > > >> > >> > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > On 12/6/22 4:22 PM, Justine Olshan
> > > > wrote:
> > > > > >> > > > >> > >> > > > > > > > > > Hi all,
> > > > > >> > > > >> > >> > > > > > > > > > After Artem's questions about
> > error
> > > > > >> behavior,
> > > > > >> > > > I've
> > > > > >> > > > >> > >> > > re-evaluated
> > > > > >> > > > >> > >> > > > > the
> > > > > >> > > > >> > >> > > > > > > > > > unknown producer ID exception and
> > had
> > > > some
> > > > > >> > > > >> discussions
> > > > > >> > > > >> > >> > > offline.
> > > > > >> > > > >> > >> > > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > > I think generally it makes sense
> > to
> > > > > >> simplify
> > > > > >> > > > error
> > > > > >> > > > >> > >> handling
> > > > > >> > > > >> > >> > > in
> > > > > >> > > > >> > >> > > > > > cases
> > > > > >> > > > >> > >> > > > > > > > like
> > > > > >> > > > >> > >> > > > > > > > > > this and the UNKNOWN_PRODUCER_ID
> > error
> > > > > >> has a
> > > > > >> > > > pretty
> > > > > >> > > > >> > long
> > > > > >> > > > >> > >> > and
> > > > > >> > > > >> > >> > > > > > > > complicated
> > > > > >> > > > >> > >> > > > > > > > > > history. Because of this, I
> > propose
> > > > > >> adding a
> > > > > >> > > new
> > > > > >> > > > >> error
> > > > > >> > > > >> > >> code
> > > > > >> > > > >> > >> > > > > > > > > ABORTABLE_ERROR
> > > > > >> > > > >> > >> > > > > > > > > > that when encountered by new
> > clients
> > > > > >> (gated
> > > > > >> > by
> > > > > >> > > > the
> > > > > >> > > > >> > >> produce
> > > > > >> > > > >> > >> > > > > request
> > > > > >> > > > >> > >> > > > > > > > > version)
> > > > > >> > > > >> > >> > > > > > > > > > will simply abort the transaction.
> > > > This
> > > > > >> > allows
> > > > > >> > > > the
> > > > > >> > > > >> > >> server
> > > > > >> > > > >> > >> > to
> > > > > >> > > > >> > >> > > > have
> > > > > >> > > > >> > >> > > > > > > some
> > > > > >> > > > >> > >> > > > > > > > > say
> > > > > >> > > > >> > >> > > > > > > > > > in whether the client aborts and
> > makes
> > > > > >> > handling
> > > > > >> > > > >> much
> > > > > >> > > > >> > >> > simpler.
> > > > > >> > > > >> > >> > > > In
> > > > > >> > > > >> > >> > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > > future, we can also use this
> > error in
> > > > > >> other
> > > > > >> > > > >> situations
> > > > > >> > > > >> > >> > where
> > > > > >> > > > >> > >> > > we
> > > > > >> > > > >> > >> > > > > > want
> > > > > >> > > > >> > >> > > > > > > to
> > > > > >> > > > >> > >> > > > > > > > > > abort the transactions. We can
> > even
> > > > use on
> > > > > >> > > other
> > > > > >> > > > >> apis.
> > > > > >> > > > >> > >> > > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > > I've added this to the KIP. Let me
> > > > know if
> > > > > >> > > there
> > > > > >> > > > >> are
> > > > > >> > > > >> > any
> > > > > >> > > > >> > >> > > > > questions
> > > > > >> > > > >> > >> > > > > > or
> > > > > >> > > > >> > >> > > > > > > > > > issues.
> > > > > >> > > > >> > >> > > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > > Justine
> > > > > >> > > > >> > >> > > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > > On Fri, Dec 2, 2022 at 10:22 AM
> > > > Justine
> > > > > >> > Olshan
> > > > > >> > > <
> > > > > >> > > > >> > >> > > > > > jolshan@confluent.io
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > wrote:
> > > > > >> > > > >> > >> > > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > >> Hey Matthias,
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >> 20/30 — Maybe I also didn't
> > express
> > > > > >> myself
> > > > > >> > > > >> clearly.
> > > > > >> > > > >> > For
> > > > > >> > > > >> > >> > > older
> > > > > >> > > > >> > >> > > > > > > clients
> > > > > >> > > > >> > >> > > > > > > > we
> > > > > >> > > > >> > >> > > > > > > > > >> don't have a way to distinguish
> > > > between a
> > > > > >> > > > previous
> > > > > >> > > > >> > and
> > > > > >> > > > >> > >> the
> > > > > >> > > > >> > >> > > > > current
> > > > > >> > > > >> > >> > > > > > > > > >> transaction since we don't have
> > the
> > > > epoch
> > > > > >> > > bump.
> > > > > >> > > > >> This
> > > > > >> > > > >> > >> means
> > > > > >> > > > >> > >> > > > that
> > > > > >> > > > >> > >> > > > > a
> > > > > >> > > > >> > >> > > > > > > late
> > > > > >> > > > >> > >> > > > > > > > > >> message from the previous
> > transaction
> > > > > >> may be
> > > > > >> > > > >> added to
> > > > > >> > > > >> > >> the
> > > > > >> > > > >> > >> > > new
> > > > > >> > > > >> > >> > > > > one.
> > > > > >> > > > >> > >> > > > > > > > With
> > > > > >> > > > >> > >> > > > > > > > > >> older clients — we can't
> > guarantee
> > > > this
> > > > > >> > won't
> > > > > >> > > > >> happen
> > > > > >> > > > >> > >> if we
> > > > > >> > > > >> > >> > > > > already
> > > > > >> > > > >> > >> > > > > > > > sent
> > > > > >> > > > >> > >> > > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call (why we
> > make
> > > > > >> changes
> > > > > >> > > for
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> newer
> > > > > >> > > > >> > >> > > > > client)
> > > > > >> > > > >> > >> > > > > > > but
> > > > > >> > > > >> > >> > > > > > > > > we
> > > > > >> > > > >> > >> > > > > > > > > >> can at least gate some by
> > ensuring
> > > > that
> > > > > >> the
> > > > > >> > > > >> partition
> > > > > >> > > > >> > >> has
> > > > > >> > > > >> > >> > > been
> > > > > >> > > > >> > >> > > > > > added
> > > > > >> > > > >> > >> > > > > > > > to
> > > > > >> > > > >> > >> > > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >> transaction. The rationale here
> > is
> > > > that
> > > > > >> > there
> > > > > >> > > > are
> > > > > >> > > > >> > >> likely
> > > > > >> > > > >> > >> > > LESS
> > > > > >> > > > >> > >> > > > > late
> > > > > >> > > > >> > >> > > > > > > > > arrivals
> > > > > >> > > > >> > >> > > > > > > > > >> as time goes on, so hopefully
> > most
> > > > late
> > > > > >> > > arrivals
> > > > > >> > > > >> will
> > > > > >> > > > >> > >> come
> > > > > >> > > > >> > >> > > in
> > > > > >> > > > >> > >> > > > > > BEFORE
> > > > > >> > > > >> > >> > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call. Those
> > that
> > > > > >> arrive
> > > > > >> > > > before
> > > > > >> > > > >> > will
> > > > > >> > > > >> > >> be
> > > > > >> > > > >> > >> > > > > properly
> > > > > >> > > > >> > >> > > > > > > > gated
> > > > > >> > > > >> > >> > > > > > > > > >> with the describeTransactions
> > > > approach.
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >> If we take the approach you
> > > > suggested,
> > > > > >> ANY
> > > > > >> > > late
> > > > > >> > > > >> > arrival
> > > > > >> > > > >> > >> > > from a
> > > > > >> > > > >> > >> > > > > > > > previous
> > > > > >> > > > >> > >> > > > > > > > > >> transaction will be added. And we
> > > > don't
> > > > > >> want
> > > > > >> > > > >> that. I
> > > > > >> > > > >> > >> also
> > > > > >> > > > >> > >> > > > don't
> > > > > >> > > > >> > >> > > > > > see
> > > > > >> > > > >> > >> > > > > > > > any
> > > > > >> > > > >> > >> > > > > > > > > >> benefit in sending
> > addPartitionsToTxn
> > > > > >> over
> > > > > >> > the
> > > > > >> > > > >> > >> > describeTxns
> > > > > >> > > > >> > >> > > > > call.
> > > > > >> > > > >> > >> > > > > > > They
> > > > > >> > > > >> > >> > > > > > > > > will
> > > > > >> > > > >> > >> > > > > > > > > >> both be one extra RPC to the Txn
> > > > > >> > coordinator.
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >> To be clear — newer clients will
> > use
> > > > > >> > > > >> > addPartitionsToTxn
> > > > > >> > > > >> > >> > > > instead
> > > > > >> > > > >> > >> > > > > of
> > > > > >> > > > >> > >> > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >> DescribeTxns.
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >> 40)
> > > > > >> > > > >> > >> > > > > > > > > >> My concern is that if we have
> > some
> > > > delay
> > > > > >> in
> > > > > >> > > the
> > > > > >> > > > >> > client
> > > > > >> > > > >> > >> to
> > > > > >> > > > >> > >> > > bump
> > > > > >> > > > >> > >> > > > > the
> > > > > >> > > > >> > >> > > > > > > > > epoch,
> > > > > >> > > > >> > >> > > > > > > > > >> it could continue to send epoch
> > 73
> > > > and
> > > > > >> those
> > > > > >> > > > >> records
> > > > > >> > > > >> > >> would
> > > > > >> > > > >> > >> > > not
> > > > > >> > > > >> > >> > > > > be
> > > > > >> > > > >> > >> > > > > > > > > fenced.
> > > > > >> > > > >> > >> > > > > > > > > >> Perhaps this is not an issue if
> > we
> > > > don't
> > > > > >> > allow
> > > > > >> > > > the
> > > > > >> > > > >> > next
> > > > > >> > > > >> > >> > > > produce
> > > > > >> > > > >> > >> > > > > to
> > > > > >> > > > >> > >> > > > > > > go
> > > > > >> > > > >> > >> > > > > > > > > >> through before the EndTxn request
> > > > > >> returns.
> > > > > >> > I'm
> > > > > >> > > > >> also
> > > > > >> > > > >> > >> > thinking
> > > > > >> > > > >> > >> > > > > about
> > > > > >> > > > >> > >> > > > > > > > > cases of
> > > > > >> > > > >> > >> > > > > > > > > >> failure. I will need to think on
> > > > this a
> > > > > >> bit.
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >> I wasn't sure if it was that
> > > > confusing.
> > > > > >> But
> > > > > >> > if
> > > > > >> > > > we
> > > > > >> > > > >> > >> think it
> > > > > >> > > > >> > >> > > is,
> > > > > >> > > > >> > >> > > > > we
> > > > > >> > > > >> > >> > > > > > > can
> > > > > >> > > > >> > >> > > > > > > > > >> investigate other ways.
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >> 60)
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >> I'm not sure these are the same
> > > > > >> purgatories
> > > > > >> > > > since
> > > > > >> > > > >> one
> > > > > >> > > > >> > >> is a
> > > > > >> > > > >> > >> > > > > produce
> > > > > >> > > > >> > >> > > > > > > > > >> purgatory (I was planning on
> > using a
> > > > > >> > callback
> > > > > >> > > > >> rather
> > > > > >> > > > >> > >> than
> > > > > >> > > > >> > >> > > > > > purgatory)
> > > > > >> > > > >> > >> > > > > > > > and
> > > > > >> > > > >> > >> > > > > > > > > >> the other is simply a request to
> > > > append
> > > > > >> to
> > > > > >> > the
> > > > > >> > > > >> log.
> > > > > >> > > > >> > Not
> > > > > >> > > > >> > >> > sure
> > > > > >> > > > >> > >> > > > we
> > > > > >> > > > >> > >> > > > > > have
> > > > > >> > > > >> > >> > > > > > > > any
> > > > > >> > > > >> > >> > > > > > > > > >> structure here for ordering, but
> > my
> > > > > >> > > > understanding
> > > > > >> > > > >> is
> > > > > >> > > > >> > >> that
> > > > > >> > > > >> > >> > > the
> > > > > >> > > > >> > >> > > > > > broker
> > > > > >> > > > >> > >> > > > > > > > > could
> > > > > >> > > > >> > >> > > > > > > > > >> handle the write request before
> > it
> > > > hears
> > > > > >> > back
> > > > > >> > > > from
> > > > > >> > > > >> > the
> > > > > >> > > > >> > >> Txn
> > > > > >> > > > >> > >> > > > > > > > Coordinator.
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >> Let me know if I misunderstood
> > > > something
> > > > > >> or
> > > > > >> > > > >> something
> > > > > >> > > > >> > >> was
> > > > > >> > > > >> > >> > > > > unclear.
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >> Justine
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >> On Thu, Dec 1, 2022 at 12:15 PM
> > > > Matthias
> > > > > >> J.
> > > > > >> > > Sax
> > > > > >> > > > <
> > > > > >> > > > >> > >> > > > > mjsax@apache.org
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > wrote:
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >>> Thanks for the details Justine!
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>> 20)
> > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>> The client side change for 2 is
> > > > > >> removing
> > > > > >> > the
> > > > > >> > > > >> > >> > addPartitions
> > > > > >> > > > >> > >> > > > to
> > > > > >> > > > >> > >> > > > > > > > > >>> transaction
> > > > > >> > > > >> > >> > > > > > > > > >>>> call. We don't need to make
> > this
> > > > from
> > > > > >> the
> > > > > >> > > > >> producer
> > > > > >> > > > >> > to
> > > > > >> > > > >> > >> > the
> > > > > >> > > > >> > >> > > > txn
> > > > > >> > > > >> > >> > > > > > > > > >>> coordinator,
> > > > > >> > > > >> > >> > > > > > > > > >>>> only server side.
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>> I think I did not express myself
> > > > > >> clearly. I
> > > > > >> > > > >> > understand
> > > > > >> > > > >> > >> > that
> > > > > >> > > > >> > >> > > > we
> > > > > >> > > > >> > >> > > > > > can
> > > > > >> > > > >> > >> > > > > > > > (and
> > > > > >> > > > >> > >> > > > > > > > > >>> should) change the producer to
> > not
> > > > send
> > > > > >> the
> > > > > >> > > > >> > >> > `addPartitions`
> > > > > >> > > > >> > >> > > > > > request
> > > > > >> > > > >> > >> > > > > > > > any
> > > > > >> > > > >> > >> > > > > > > > > >>> longer. But I don't thinks it's
> > > > > >> requirement
> > > > > >> > > to
> > > > > >> > > > >> > change
> > > > > >> > > > >> > >> the
> > > > > >> > > > >> > >> > > > > broker?
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>> What I am trying to say is: as a
> > > > > >> safe-guard
> > > > > >> > > and
> > > > > >> > > > >> > >> > improvement
> > > > > >> > > > >> > >> > > > for
> > > > > >> > > > >> > >> > > > > > > older
> > > > > >> > > > >> > >> > > > > > > > > >>> producers, the partition leader
> > can
> > > > just
> > > > > >> > send
> > > > > >> > > > the
> > > > > >> > > > >> > >> > > > > `addPartitions`
> > > > > >> > > > >> > >> > > > > > > > > >>> request to the TX-coordinator
> > in any
> > > > > >> case
> > > > > >> > --
> > > > > >> > > if
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> old
> > > > > >> > > > >> > >> > > > > producer
> > > > > >> > > > >> > >> > > > > > > > > >>> correctly did send the
> > > > `addPartition`
> > > > > >> > request
> > > > > >> > > > to
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> > > > > > TX-coordinator
> > > > > >> > > > >> > >> > > > > > > > > >>> already, the TX-coordinator can
> > just
> > > > > >> > "ignore"
> > > > > >> > > > is
> > > > > >> > > > >> as
> > > > > >> > > > >> > >> > > > idempotent.
> > > > > >> > > > >> > >> > > > > > > > > However,
> > > > > >> > > > >> > >> > > > > > > > > >>> if the old producer has a bug
> > and
> > > > did
> > > > > >> > forget
> > > > > >> > > to
> > > > > >> > > > >> sent
> > > > > >> > > > >> > >> the
> > > > > >> > > > >> > >> > > > > > > > `addPartition`
> > > > > >> > > > >> > >> > > > > > > > > >>> request, we would now ensure
> > that
> > > > the
> > > > > >> > > partition
> > > > > >> > > > >> is
> > > > > >> > > > >> > >> indeed
> > > > > >> > > > >> > >> > > > added
> > > > > >> > > > >> > >> > > > > > to
> > > > > >> > > > >> > >> > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> TX and thus fix a potential
> > > > producer bug
> > > > > >> > > (even
> > > > > >> > > > >> if we
> > > > > >> > > > >> > >> > don't
> > > > > >> > > > >> > >> > > > get
> > > > > >> > > > >> > >> > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> fencing via the bump epoch). --
> > It
> > > > > >> seems to
> > > > > >> > > be
> > > > > >> > > > a
> > > > > >> > > > >> > good
> > > > > >> > > > >> > >> > > > > > improvement?
> > > > > >> > > > >> > >> > > > > > > Or
> > > > > >> > > > >> > >> > > > > > > > > is
> > > > > >> > > > >> > >> > > > > > > > > >>> there a reason to not do this?
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>> 30)
> > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>> Transaction is ongoing =
> > partition
> > > > was
> > > > > >> > added
> > > > > >> > > > to
> > > > > >> > > > >> > >> > > transaction
> > > > > >> > > > >> > >> > > > > via
> > > > > >> > > > >> > >> > > > > > > > > >>>> addPartitionsToTxn. We check
> > this
> > > > with
> > > > > >> the
> > > > > >> > > > >> > >> > > > > DescribeTransactions
> > > > > >> > > > >> > >> > > > > > > > call.
> > > > > >> > > > >> > >> > > > > > > > > >>> Let
> > > > > >> > > > >> > >> > > > > > > > > >>>> me know if this wasn't
> > sufficiently
> > > > > >> > > explained
> > > > > >> > > > >> here:
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>> If we do what I propose in
> > (20), we
> > > > > >> don't
> > > > > >> > > > really
> > > > > >> > > > >> > need
> > > > > >> > > > >> > >> to
> > > > > >> > > > >> > >> > > make
> > > > > >> > > > >> > >> > > > > > this
> > > > > >> > > > >> > >> > > > > > > > > >>> `DescribeTransaction` call, as
> > the
> > > > > >> > partition
> > > > > >> > > > >> leader
> > > > > >> > > > >> > >> adds
> > > > > >> > > > >> > >> > > the
> > > > > >> > > > >> > >> > > > > > > > partition
> > > > > >> > > > >> > >> > > > > > > > > >>> for older clients and we get
> > this
> > > > check
> > > > > >> for
> > > > > >> > > > free.
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>> 40)
> > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>> The idea here is that if any
> > > > messages
> > > > > >> > > somehow
> > > > > >> > > > >> come
> > > > > >> > > > >> > in
> > > > > >> > > > >> > >> > > before
> > > > > >> > > > >> > >> > > > > we
> > > > > >> > > > >> > >> > > > > > > get
> > > > > >> > > > >> > >> > > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> new
> > > > > >> > > > >> > >> > > > > > > > > >>>> epoch to the producer, they
> > will be
> > > > > >> > fenced.
> > > > > >> > > > >> > However,
> > > > > >> > > > >> > >> if
> > > > > >> > > > >> > >> > we
> > > > > >> > > > >> > >> > > > > don't
> > > > > >> > > > >> > >> > > > > > > > think
> > > > > >> > > > >> > >> > > > > > > > > >>> this
> > > > > >> > > > >> > >> > > > > > > > > >>>> is necessary, it can be
> > discussed
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>> I agree that we should have
> > epoch
> > > > > >> fencing.
> > > > > >> > My
> > > > > >> > > > >> > >> question is
> > > > > >> > > > >> > >> > > > > > > different:
> > > > > >> > > > >> > >> > > > > > > > > >>> Assume we are at epoch 73, and
> > we
> > > > have
> > > > > >> an
> > > > > >> > > > ongoing
> > > > > >> > > > >> > >> > > > transaction,
> > > > > >> > > > >> > >> > > > > > that
> > > > > >> > > > >> > >> > > > > > > > is
> > > > > >> > > > >> > >> > > > > > > > > >>> committed. It seems natural to
> > > > write the
> > > > > >> > > > "prepare
> > > > > >> > > > >> > >> commit"
> > > > > >> > > > >> > >> > > > > marker
> > > > > >> > > > >> > >> > > > > > > and
> > > > > >> > > > >> > >> > > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` both with
> > > > epoch
> > > > > >> 73,
> > > > > >> > > too,
> > > > > >> > > > >> as
> > > > > >> > > > >> > it
> > > > > >> > > > >> > >> > > belongs
> > > > > >> > > > >> > >> > > > > to
> > > > > >> > > > >> > >> > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> current transaction. Of course,
> > we
> > > > now
> > > > > >> also
> > > > > >> > > > bump
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> > epoch
> > > > > >> > > > >> > >> > > > and
> > > > > >> > > > >> > >> > > > > > > expect
> > > > > >> > > > >> > >> > > > > > > > > >>> the next requests to have epoch
> > 74,
> > > > and
> > > > > >> > would
> > > > > >> > > > >> reject
> > > > > >> > > > >> > >> an
> > > > > >> > > > >> > >> > > > request
> > > > > >> > > > >> > >> > > > > > > with
> > > > > >> > > > >> > >> > > > > > > > > >>> epoch 73, as the corresponding
> > TX
> > > > for
> > > > > >> epoch
> > > > > >> > > 73
> > > > > >> > > > >> was
> > > > > >> > > > >> > >> > already
> > > > > >> > > > >> > >> > > > > > > committed.
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>> It seems you propose to write
> > the
> > > > > >> "prepare
> > > > > >> > > > commit
> > > > > >> > > > >> > >> marker"
> > > > > >> > > > >> > >> > > and
> > > > > >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` with
> > epoch 74
> > > > > >> > though,
> > > > > >> > > > what
> > > > > >> > > > >> > >> would
> > > > > >> > > > >> > >> > > work,
> > > > > >> > > > >> > >> > > > > but
> > > > > >> > > > >> > >> > > > > > > it
> > > > > >> > > > >> > >> > > > > > > > > >>> seems confusing. Is there a
> > reason
> > > > why
> > > > > >> we
> > > > > >> > > would
> > > > > >> > > > >> use
> > > > > >> > > > >> > >> the
> > > > > >> > > > >> > >> > > > bumped
> > > > > >> > > > >> > >> > > > > > > epoch
> > > > > >> > > > >> > >> > > > > > > > 74
> > > > > >> > > > >> > >> > > > > > > > > >>> instead of the current epoch 73?
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>> 60)
> > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>> When we are checking if the
> > > > > >> transaction is
> > > > > >> > > > >> ongoing,
> > > > > >> > > > >> > >> we
> > > > > >> > > > >> > >> > > need
> > > > > >> > > > >> > >> > > > to
> > > > > >> > > > >> > >> > > > > > > make
> > > > > >> > > > >> > >> > > > > > > > a
> > > > > >> > > > >> > >> > > > > > > > > >>> round
> > > > > >> > > > >> > >> > > > > > > > > >>>> trip from the leader partition
> > to
> > > > the
> > > > > >> > > > >> transaction
> > > > > >> > > > >> > >> > > > coordinator.
> > > > > >> > > > >> > >> > > > > > In
> > > > > >> > > > >> > >> > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> time
> > > > > >> > > > >> > >> > > > > > > > > >>>> we are waiting for this
> > message to
> > > > come
> > > > > >> > > back,
> > > > > >> > > > in
> > > > > >> > > > >> > >> theory
> > > > > >> > > > >> > >> > we
> > > > > >> > > > >> > >> > > > > could
> > > > > >> > > > >> > >> > > > > > > > have
> > > > > >> > > > >> > >> > > > > > > > > >>> sent
> > > > > >> > > > >> > >> > > > > > > > > >>>> a commit/abort call that would
> > > > make the
> > > > > >> > > > original
> > > > > >> > > > >> > >> result
> > > > > >> > > > >> > >> > of
> > > > > >> > > > >> > >> > > > the
> > > > > >> > > > >> > >> > > > > > > check
> > > > > >> > > > >> > >> > > > > > > > > >>> out of
> > > > > >> > > > >> > >> > > > > > > > > >>>> date. That is why we can check
> > the
> > > > > >> leader
> > > > > >> > > > state
> > > > > >> > > > >> > >> before
> > > > > >> > > > >> > >> > we
> > > > > >> > > > >> > >> > > > > write
> > > > > >> > > > >> > >> > > > > > to
> > > > > >> > > > >> > >> > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> log.
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>> Thanks. Got it.
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>> However, is this really an
> > issue?
> > > > We put
> > > > > >> > the
> > > > > >> > > > >> produce
> > > > > >> > > > >> > >> > > request
> > > > > >> > > > >> > >> > > > in
> > > > > >> > > > >> > >> > > > > > > > > >>> purgatory, so how could we
> > process
> > > > the
> > > > > >> > > > >> > >> > > > `WriteTxnMarkerRequest`
> > > > > >> > > > >> > >> > > > > > > first?
> > > > > >> > > > >> > >> > > > > > > > > >>> Don't we need to put the
> > > > > >> > > > `WriteTxnMarkerRequest`
> > > > > >> > > > >> > into
> > > > > >> > > > >> > >> > > > > purgatory,
> > > > > >> > > > >> > >> > > > > > > too,
> > > > > >> > > > >> > >> > > > > > > > > >>> for this case, and process both
> > > > request
> > > > > >> > > > in-order?
> > > > > >> > > > >> > >> (Again,
> > > > > >> > > > >> > >> > > my
> > > > > >> > > > >> > >> > > > > > broker
> > > > > >> > > > >> > >> > > > > > > > > >>> knowledge is limited and maybe
> > we
> > > > don't
> > > > > >> > > > maintain
> > > > > >> > > > >> > >> request
> > > > > >> > > > >> > >> > > > order
> > > > > >> > > > >> > >> > > > > > for
> > > > > >> > > > >> > >> > > > > > > > this
> > > > > >> > > > >> > >> > > > > > > > > >>> case, what seems to be an issue
> > > > IMHO,
> > > > > >> and I
> > > > > >> > > am
> > > > > >> > > > >> > >> wondering
> > > > > >> > > > >> > >> > if
> > > > > >> > > > >> > >> > > > > > > changing
> > > > > >> > > > >> > >> > > > > > > > > >>> request handling to preserve
> > order
> > > > for
> > > > > >> this
> > > > > >> > > > case
> > > > > >> > > > >> > >> might be
> > > > > >> > > > >> > >> > > the
> > > > > >> > > > >> > >> > > > > > > cleaner
> > > > > >> > > > >> > >> > > > > > > > > >>> solution?)
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>> -Matthias
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>> On 11/30/22 3:28 PM, Artem
> > Livshits
> > > > > >> wrote:
> > > > > >> > > > >> > >> > > > > > > > > >>>> Hi Justine,
> > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>> I think the interesting part is
> > > > not in
> > > > > >> > this
> > > > > >> > > > >> logic
> > > > > >> > > > >> > >> > (because
> > > > > >> > > > >> > >> > > > it
> > > > > >> > > > >> > >> > > > > > > tries
> > > > > >> > > > >> > >> > > > > > > > to
> > > > > >> > > > >> > >> > > > > > > > > >>>> figure out when
> > > > UNKNOWN_PRODUCER_ID is
> > > > > >> > > > retriable
> > > > > >> > > > >> > and
> > > > > >> > > > >> > >> if
> > > > > >> > > > >> > >> > > it's
> > > > > >> > > > >> > >> > > > > > > > > retryable,
> > > > > >> > > > >> > >> > > > > > > > > >>>> it's definitely not fatal), but
> > > > what
> > > > > >> > happens
> > > > > >> > > > >> when
> > > > > >> > > > >> > >> this
> > > > > >> > > > >> > >> > > logic
> > > > > >> > > > >> > >> > > > > > > doesn't
> > > > > >> > > > >> > >> > > > > > > > > >>> return
> > > > > >> > > > >> > >> > > > > > > > > >>>> 'true' and falls through.  In
> > the
> > > > old
> > > > > >> > > clients
> > > > > >> > > > it
> > > > > >> > > > >> > >> seems
> > > > > >> > > > >> > >> > to
> > > > > >> > > > >> > >> > > be
> > > > > >> > > > >> > >> > > > > > > fatal,
> > > > > >> > > > >> > >> > > > > > > > if
> > > > > >> > > > >> > >> > > > > > > > > >>> we
> > > > > >> > > > >> > >> > > > > > > > > >>>> keep the behavior in the new
> > > > clients,
> > > > > >> I'd
> > > > > >> > > > >> expect it
> > > > > >> > > > >> > >> > would
> > > > > >> > > > >> > >> > > be
> > > > > >> > > > >> > >> > > > > > fatal
> > > > > >> > > > >> > >> > > > > > > > as
> > > > > >> > > > >> > >> > > > > > > > > >>> well.
> > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>> -Artem
> > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>> On Tue, Nov 29, 2022 at 11:57
> > AM
> > > > > >> Justine
> > > > > >> > > > Olshan
> > > > > >> > > > >> > >> > > > > > > > > >>>> <jo...@confluent.io.invalid>
> > > > wrote:
> > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> Hi Artem and Jeff,
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> Thanks for taking a look and
> > > > sorry for
> > > > > >> > the
> > > > > >> > > > slow
> > > > > >> > > > >> > >> > response.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> You both mentioned the change
> > to
> > > > > >> handle
> > > > > >> > > > >> > >> > > UNKNOWN_PRODUCER_ID
> > > > > >> > > > >> > >> > > > > > > errors.
> > > > > >> > > > >> > >> > > > > > > > > To
> > > > > >> > > > >> > >> > > > > > > > > >>> be
> > > > > >> > > > >> > >> > > > > > > > > >>>>> clear — this error code will
> > only
> > > > be
> > > > > >> sent
> > > > > >> > > > again
> > > > > >> > > > >> > when
> > > > > >> > > > >> > >> > the
> > > > > >> > > > >> > >> > > > > > client's
> > > > > >> > > > >> > >> > > > > > > > > >>> request
> > > > > >> > > > >> > >> > > > > > > > > >>>>> version is high enough to
> > ensure
> > > > we
> > > > > >> > handle
> > > > > >> > > it
> > > > > >> > > > >> > >> > correctly.
> > > > > >> > > > >> > >> > > > > > > > > >>>>> The current (Java) client
> > handles
> > > > > >> this by
> > > > > >> > > the
> > > > > >> > > > >> > >> following
> > > > > >> > > > >> > >> > > > > > (somewhat
> > > > > >> > > > >> > >> > > > > > > > > long)
> > > > > >> > > > >> > >> > > > > > > > > >>>>> code snippet:
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> // An UNKNOWN_PRODUCER_ID
> > means
> > > > that
> > > > > >> we
> > > > > >> > > have
> > > > > >> > > > >> lost
> > > > > >> > > > >> > >> the
> > > > > >> > > > >> > >> > > > > producer
> > > > > >> > > > >> > >> > > > > > > > state
> > > > > >> > > > >> > >> > > > > > > > > >>> on the
> > > > > >> > > > >> > >> > > > > > > > > >>>>> broker. Depending on the log
> > start
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> // offset, we may want to
> > retry
> > > > > >> these, as
> > > > > >> > > > >> > described
> > > > > >> > > > >> > >> for
> > > > > >> > > > >> > >> > > > each
> > > > > >> > > > >> > >> > > > > > case
> > > > > >> > > > >> > >> > > > > > > > > >>> below. If
> > > > > >> > > > >> > >> > > > > > > > > >>>>> none of those apply, then for
> > the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> // idempotent producer, we
> > will
> > > > > >> locally
> > > > > >> > > bump
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> epoch
> > > > > >> > > > >> > >> > > and
> > > > > >> > > > >> > >> > > > > > reset
> > > > > >> > > > >> > >> > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>>>> sequence numbers of in-flight
> > > > batches
> > > > > >> > from
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> // sequence 0, then retry the
> > > > failed
> > > > > >> > batch,
> > > > > >> > > > >> which
> > > > > >> > > > >> > >> > should
> > > > > >> > > > >> > >> > > > now
> > > > > >> > > > >> > >> > > > > > > > succeed.
> > > > > >> > > > >> > >> > > > > > > > > >>> For
> > > > > >> > > > >> > >> > > > > > > > > >>>>> the transactional producer,
> > allow
> > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> // batch to fail. When
> > processing
> > > > the
> > > > > >> > > failed
> > > > > >> > > > >> > batch,
> > > > > >> > > > >> > >> we
> > > > > >> > > > >> > >> > > will
> > > > > >> > > > >> > >> > > > > > > > > transition
> > > > > >> > > > >> > >> > > > > > > > > >>> to
> > > > > >> > > > >> > >> > > > > > > > > >>>>> an abortable error and set a
> > flag
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> // indicating that we need to
> > > > bump the
> > > > > >> > > epoch
> > > > > >> > > > >> (if
> > > > > >> > > > >> > >> > > supported
> > > > > >> > > > >> > >> > > > by
> > > > > >> > > > >> > >> > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> broker).
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> if (error ==
> > > > > >> > Errors.*UNKNOWN_PRODUCER_ID*)
> > > > > >> > > {
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>       if
> > (response.logStartOffset
> > > > ==
> > > > > >> -1)
> > > > > >> > {
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // We don't know
> > the log
> > > > > >> start
> > > > > >> > > > offset
> > > > > >> > > > >> > with
> > > > > >> > > > >> > >> > this
> > > > > >> > > > >> > >> > > > > > > response.
> > > > > >> > > > >> > >> > > > > > > > > We
> > > > > >> > > > >> > >> > > > > > > > > >>> should
> > > > > >> > > > >> > >> > > > > > > > > >>>>> just retry the request until
> > we
> > > > get
> > > > > >> it.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // The
> > > > UNKNOWN_PRODUCER_ID
> > > > > >> > error
> > > > > >> > > > code
> > > > > >> > > > >> > was
> > > > > >> > > > >> > >> > added
> > > > > >> > > > >> > >> > > > > along
> > > > > >> > > > >> > >> > > > > > > > with
> > > > > >> > > > >> > >> > > > > > > > > >>> the new
> > > > > >> > > > >> > >> > > > > > > > > >>>>> ProduceResponse which
> > includes the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // logStartOffset.
> > So
> > > > the
> > > > > >> '-1'
> > > > > >> > > > >> sentinel
> > > > > >> > > > >> > is
> > > > > >> > > > >> > >> > not
> > > > > >> > > > >> > >> > > > for
> > > > > >> > > > >> > >> > > > > > > > backward
> > > > > >> > > > >> > >> > > > > > > > > >>>>> compatibility. Instead, it is
> > > > possible
> > > > > >> > for
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // a broker to not
> > know
> > > > the
> > > > > >> > > > >> > >> logStartOffset at
> > > > > >> > > > >> > >> > > > when
> > > > > >> > > > >> > >> > > > > it
> > > > > >> > > > >> > >> > > > > > > is
> > > > > >> > > > >> > >> > > > > > > > > >>> returning
> > > > > >> > > > >> > >> > > > > > > > > >>>>> the response because the
> > partition
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // may have moved
> > away
> > > > from
> > > > > >> the
> > > > > >> > > > >> broker
> > > > > >> > > > >> > >> from
> > > > > >> > > > >> > >> > the
> > > > > >> > > > >> > >> > > > > time
> > > > > >> > > > >> > >> > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> error was
> > > > > >> > > > >> > >> > > > > > > > > >>>>> initially raised to the time
> > the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // response was
> > being
> > > > > >> > > constructed.
> > > > > >> > > > In
> > > > > >> > > > >> > >> these
> > > > > >> > > > >> > >> > > > cases,
> > > > > >> > > > >> > >> > > > > we
> > > > > >> > > > >> > >> > > > > > > > > should
> > > > > >> > > > >> > >> > > > > > > > > >>> just
> > > > > >> > > > >> > >> > > > > > > > > >>>>> retry the request: we are
> > > > guaranteed
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // to eventually
> > get a
> > > > > >> > > > logStartOffset
> > > > > >> > > > >> > once
> > > > > >> > > > >> > >> > > things
> > > > > >> > > > >> > >> > > > > > > settle
> > > > > >> > > > >> > >> > > > > > > > > down.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>       if
> > > > > >> (batch.sequenceHasBeenReset()) {
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // When the first
> > > > inflight
> > > > > >> > batch
> > > > > >> > > > >> fails
> > > > > >> > > > >> > >> due to
> > > > > >> > > > >> > >> > > the
> > > > > >> > > > >> > >> > > > > > > > > truncation
> > > > > >> > > > >> > >> > > > > > > > > >>> case,
> > > > > >> > > > >> > >> > > > > > > > > >>>>> then the sequences of all the
> > > > other
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // in flight batches
> > > > would
> > > > > >> have
> > > > > >> > > > been
> > > > > >> > > > >> > >> > restarted
> > > > > >> > > > >> > >> > > > from
> > > > > >> > > > >> > >> > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> beginning.
> > > > > >> > > > >> > >> > > > > > > > > >>>>> However, when those responses
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // come back from
> > the
> > > > > >> broker,
> > > > > >> > > they
> > > > > >> > > > >> would
> > > > > >> > > > >> > >> also
> > > > > >> > > > >> > >> > > > come
> > > > > >> > > > >> > >> > > > > > with
> > > > > >> > > > >> > >> > > > > > > > an
> > > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID error. In
> > this
> > > > > >> case,
> > > > > >> > we
> > > > > >> > > > >> should
> > > > > >> > > > >> > >> not
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // reset the
> > sequence
> > > > > >> numbers
> > > > > >> > to
> > > > > >> > > > the
> > > > > >> > > > >> > >> > beginning.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>       } else if
> > > > > >> > > > >> > >> > > > >
> > (lastAckedOffset(batch.topicPartition).orElse(
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > > > > >> > > > >> > >> > > > response.logStartOffset) {
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // The head of the
> > log
> > > > has
> > > > > >> been
> > > > > >> > > > >> removed,
> > > > > >> > > > >> > >> > > probably
> > > > > >> > > > >> > >> > > > > due
> > > > > >> > > > >> > >> > > > > > > to
> > > > > >> > > > >> > >> > > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>>>> retention time elapsing. In
> > this
> > > > case,
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // we expect to
> > lose the
> > > > > >> > producer
> > > > > >> > > > >> state.
> > > > > >> > > > >> > >> For
> > > > > >> > > > >> > >> > > the
> > > > > >> > > > >> > >> > > > > > > > > transactional
> > > > > >> > > > >> > >> > > > > > > > > >>>>> producer, reset the sequences
> > of
> > > > all
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // inflight batches
> > to
> > > > be
> > > > > >> from
> > > > > >> > > the
> > > > > >> > > > >> > >> beginning
> > > > > >> > > > >> > >> > > and
> > > > > >> > > > >> > >> > > > > > retry
> > > > > >> > > > >> > >> > > > > > > > > them,
> > > > > >> > > > >> > >> > > > > > > > > >>> so
> > > > > >> > > > >> > >> > > > > > > > > >>>>> that the transaction does not
> > > > need to
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // be aborted. For
> > the
> > > > > >> > idempotent
> > > > > >> > > > >> > >> producer,
> > > > > >> > > > >> > >> > > bump
> > > > > >> > > > >> > >> > > > > the
> > > > > >> > > > >> > >> > > > > > > > epoch
> > > > > >> > > > >> > >> > > > > > > > > to
> > > > > >> > > > >> > >> > > > > > > > > >>> avoid
> > > > > >> > > > >> > >> > > > > > > > > >>>>> reusing (sequence, epoch)
> > pairs
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           if
> > (isTransactional()) {
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > >
> > > > > >> > > >
> > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > > > > >> > > > >> > >> > > > > > > > > >>>>> this.producerIdAndEpoch);
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           } else {
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > >
> > > > > >> requestEpochBumpForPartition(batch.topicPartition);
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           }
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>       if (!isTransactional())
> > {
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // For the
> > idempotent
> > > > > >> producer,
> > > > > >> > > > >> always
> > > > > >> > > > >> > >> retry
> > > > > >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > > > >> > > > >> > >> > > > > > > > > >>>>> errors. If the batch has the
> > > > current
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           // producer ID and
> > > > epoch,
> > > > > >> > > request a
> > > > > >> > > > >> bump
> > > > > >> > > > >> > >> of
> > > > > >> > > > >> > >> > the
> > > > > >> > > > >> > >> > > > > > epoch.
> > > > > >> > > > >> > >> > > > > > > > > >>> Otherwise
> > > > > >> > > > >> > >> > > > > > > > > >>>>> just retry the produce.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > >
> > > > requestEpochBumpForPartition(batch.topicPartition);
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> }
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> I was considering keeping this
> > > > > >> behavior —
> > > > > >> > > but
> > > > > >> > > > >> am
> > > > > >> > > > >> > >> open
> > > > > >> > > > >> > >> > to
> > > > > >> > > > >> > >> > > > > > > > simplifying
> > > > > >> > > > >> > >> > > > > > > > > >>> it.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> We are leaving changes to
> > older
> > > > > >> clients
> > > > > >> > off
> > > > > >> > > > the
> > > > > >> > > > >> > >> table
> > > > > >> > > > >> > >> > > here
> > > > > >> > > > >> > >> > > > > > since
> > > > > >> > > > >> > >> > > > > > > it
> > > > > >> > > > >> > >> > > > > > > > > >>> caused
> > > > > >> > > > >> > >> > > > > > > > > >>>>> many issues for clients in the
> > > > past.
> > > > > >> > > > Previously
> > > > > >> > > > >> > this
> > > > > >> > > > >> > >> > was
> > > > > >> > > > >> > >> > > a
> > > > > >> > > > >> > >> > > > > > fatal
> > > > > >> > > > >> > >> > > > > > > > > error
> > > > > >> > > > >> > >> > > > > > > > > >>> and
> > > > > >> > > > >> > >> > > > > > > > > >>>>> we didn't have the mechanisms
> > in
> > > > > >> place to
> > > > > >> > > > >> detect
> > > > > >> > > > >> > >> when
> > > > > >> > > > >> > >> > > this
> > > > > >> > > > >> > >> > > > > was
> > > > > >> > > > >> > >> > > > > > a
> > > > > >> > > > >> > >> > > > > > > > > >>> legitimate
> > > > > >> > > > >> > >> > > > > > > > > >>>>> case vs some bug or gap in the
> > > > > >> protocol.
> > > > > >> > > > >> Ensuring
> > > > > >> > > > >> > >> each
> > > > > >> > > > >> > >> > > > > > > transaction
> > > > > >> > > > >> > >> > > > > > > > > has
> > > > > >> > > > >> > >> > > > > > > > > >>> its
> > > > > >> > > > >> > >> > > > > > > > > >>>>> own epoch should close this
> > gap.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> And to address Jeff's second
> > > > point:
> > > > > >> > > > >> > >> > > > > > > > > >>>>> *does the typical produce
> > request
> > > > path
> > > > > >> > > append
> > > > > >> > > > >> > >> records
> > > > > >> > > > >> > >> > to
> > > > > >> > > > >> > >> > > > > local
> > > > > >> > > > >> > >> > > > > > > log
> > > > > >> > > > >> > >> > > > > > > > > >>> along*
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> *with the
> > currentTxnFirstOffset
> > > > > >> > > information?
> > > > > >> > > > I
> > > > > >> > > > >> > would
> > > > > >> > > > >> > >> > like
> > > > > >> > > > >> > >> > > > to
> > > > > >> > > > >> > >> > > > > > > > > >>> understand*
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> *when the field is written to
> > > > disk.*
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> Yes, the first produce request
> > > > > >> populates
> > > > > >> > > this
> > > > > >> > > > >> > field
> > > > > >> > > > >> > >> and
> > > > > >> > > > >> > >> > > > > writes
> > > > > >> > > > >> > >> > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> offset
> > > > > >> > > > >> > >> > > > > > > > > >>>>> as part of the record batch
> > and
> > > > also
> > > > > >> to
> > > > > >> > the
> > > > > >> > > > >> > producer
> > > > > >> > > > >> > >> > > state
> > > > > >> > > > >> > >> > > > > > > > snapshot.
> > > > > >> > > > >> > >> > > > > > > > > >>> When
> > > > > >> > > > >> > >> > > > > > > > > >>>>> we reload the records on
> > restart
> > > > > >> and/or
> > > > > >> > > > >> > >> reassignment,
> > > > > >> > > > >> > >> > we
> > > > > >> > > > >> > >> > > > > > > repopulate
> > > > > >> > > > >> > >> > > > > > > > > >>> this
> > > > > >> > > > >> > >> > > > > > > > > >>>>> field with the snapshot from
> > disk
> > > > > >> along
> > > > > >> > > with
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> rest
> > > > > >> > > > >> > >> > of
> > > > > >> > > > >> > >> > > > the
> > > > > >> > > > >> > >> > > > > > > > producer
> > > > > >> > > > >> > >> > > > > > > > > >>>>> state.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> Let me know if there are
> > further
> > > > > >> comments
> > > > > >> > > > >> and/or
> > > > > >> > > > >> > >> > > questions.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> Thanks,
> > > > > >> > > > >> > >> > > > > > > > > >>>>> Justine
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> On Tue, Nov 22, 2022 at 9:00
> > PM
> > > > Jeff
> > > > > >> Kim
> > > > > >> > > > >> > >> > > > > > > > > <jeff.kim@confluent.io.invalid
> > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>> wrote:
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> Hi Justine,
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> Thanks for the KIP! I have
> > two
> > > > > >> > questions:
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> 1) For new clients, we can
> > once
> > > > again
> > > > > >> > > return
> > > > > >> > > > >> an
> > > > > >> > > > >> > >> error
> > > > > >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> for sequences
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> that are non-zero when there
> > is
> > > > no
> > > > > >> > > producer
> > > > > >> > > > >> state
> > > > > >> > > > >> > >> > > present
> > > > > >> > > > >> > >> > > > on
> > > > > >> > > > >> > >> > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> server.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> This will indicate we missed
> > the
> > > > 0
> > > > > >> > > sequence
> > > > > >> > > > >> and
> > > > > >> > > > >> > we
> > > > > >> > > > >> > >> > don't
> > > > > >> > > > >> > >> > > > yet
> > > > > >> > > > >> > >> > > > > > > want
> > > > > >> > > > >> > >> > > > > > > > to
> > > > > >> > > > >> > >> > > > > > > > > >>>>> write
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> to the log.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> I would like to understand
> > the
> > > > > >> current
> > > > > >> > > > >> behavior
> > > > > >> > > > >> > to
> > > > > >> > > > >> > >> > > handle
> > > > > >> > > > >> > >> > > > > > older
> > > > > >> > > > >> > >> > > > > > > > > >>> clients,
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> and if there are any changes
> > we
> > > > are
> > > > > >> > > making.
> > > > > >> > > > >> Maybe
> > > > > >> > > > >> > >> I'm
> > > > > >> > > > >> > >> > > > > missing
> > > > > >> > > > >> > >> > > > > > > > > >>> something,
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> but we would want to identify
> > > > > >> whether we
> > > > > >> > > > >> missed
> > > > > >> > > > >> > >> the 0
> > > > > >> > > > >> > >> > > > > sequence
> > > > > >> > > > >> > >> > > > > > > for
> > > > > >> > > > >> > >> > > > > > > > > >>> older
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> clients, no?
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> 2) Upon returning from the
> > > > > >> transaction
> > > > > >> > > > >> > >> coordinator, we
> > > > > >> > > > >> > >> > > can
> > > > > >> > > > >> > >> > > > > set
> > > > > >> > > > >> > >> > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> transaction
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> as ongoing on the leader by
> > > > > >> populating
> > > > > >> > > > >> > >> > > > currentTxnFirstOffset
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> through the typical produce
> > > > request
> > > > > >> > > > handling.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> does the typical produce
> > request
> > > > path
> > > > > >> > > append
> > > > > >> > > > >> > >> records
> > > > > >> > > > >> > >> > to
> > > > > >> > > > >> > >> > > > > local
> > > > > >> > > > >> > >> > > > > > > log
> > > > > >> > > > >> > >> > > > > > > > > >>> along
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> with the
> > currentTxnFirstOffset
> > > > > >> > > information?
> > > > > >> > > > I
> > > > > >> > > > >> > would
> > > > > >> > > > >> > >> > like
> > > > > >> > > > >> > >> > > > to
> > > > > >> > > > >> > >> > > > > > > > > understand
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> when the field is written to
> > > > disk.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> Thanks,
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> Jeff
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> On Tue, Nov 22, 2022 at 4:44
> > PM
> > > > Artem
> > > > > >> > > > Livshits
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> <alivshits@confluent.io
> > .invalid>
> > > > > >> wrote:
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> Hi Justine,
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> Thank you for the KIP.  I
> > have
> > > > one
> > > > > >> > > > question.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> 5) For new clients, we can
> > once
> > > > > >> again
> > > > > >> > > > return
> > > > > >> > > > >> an
> > > > > >> > > > >> > >> error
> > > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> I believe we had problems
> > in the
> > > > > >> past
> > > > > >> > > with
> > > > > >> > > > >> > >> returning
> > > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> because it was considered
> > fatal
> > > > and
> > > > > >> > > > required
> > > > > >> > > > >> > >> client
> > > > > >> > > > >> > >> > > > > restart.
> > > > > >> > > > >> > >> > > > > > > It
> > > > > >> > > > >> > >> > > > > > > > > >>> would
> > > > > >> > > > >> > >> > > > > > > > > >>>>> be
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> good to spell out the new
> > client
> > > > > >> > behavior
> > > > > >> > > > >> when
> > > > > >> > > > >> > it
> > > > > >> > > > >> > >> > > > receives
> > > > > >> > > > >> > >> > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > error.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> -Artem
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> On Tue, Nov 22, 2022 at
> > 10:00 AM
> > > > > >> > Justine
> > > > > >> > > > >> Olshan
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > <jo...@confluent.io.invalid>
> > > > > >> wrote:
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks for taking a look
> > > > Matthias.
> > > > > >> > I've
> > > > > >> > > > >> tried
> > > > > >> > > > >> > to
> > > > > >> > > > >> > >> > > answer
> > > > > >> > > > >> > >> > > > > your
> > > > > >> > > > >> > >> > > > > > > > > >>>>> questions
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> below:
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 10)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Right — so the hanging
> > > > transaction
> > > > > >> > only
> > > > > >> > > > >> occurs
> > > > > >> > > > >> > >> when
> > > > > >> > > > >> > >> > we
> > > > > >> > > > >> > >> > > > > have
> > > > > >> > > > >> > >> > > > > > a
> > > > > >> > > > >> > >> > > > > > > > late
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> message
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> come in and the partition
> > is
> > > > never
> > > > > >> > added
> > > > > >> > > > to
> > > > > >> > > > >> a
> > > > > >> > > > >> > >> > > > transaction
> > > > > >> > > > >> > >> > > > > > > again.
> > > > > >> > > > >> > >> > > > > > > > > If
> > > > > >> > > > >> > >> > > > > > > > > >>>>> we
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> never add the partition to
> > a
> > > > > >> > > transaction,
> > > > > >> > > > we
> > > > > >> > > > >> > will
> > > > > >> > > > >> > >> > > never
> > > > > >> > > > >> > >> > > > > > write
> > > > > >> > > > >> > >> > > > > > > a
> > > > > >> > > > >> > >> > > > > > > > > >>>>> marker
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> and
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> never advance the LSO.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> If we do end up adding the
> > > > > >> partition
> > > > > >> > to
> > > > > >> > > > the
> > > > > >> > > > >> > >> > > transaction
> > > > > >> > > > >> > >> > > > (I
> > > > > >> > > > >> > >> > > > > > > > suppose
> > > > > >> > > > >> > >> > > > > > > > > >>>>> this
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> can
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> happen before or after the
> > late
> > > > > >> > message
> > > > > >> > > > >> comes
> > > > > >> > > > >> > in)
> > > > > >> > > > >> > >> > then
> > > > > >> > > > >> > >> > > > we
> > > > > >> > > > >> > >> > > > > > will
> > > > > >> > > > >> > >> > > > > > > > > >>>>> include
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> late message in the next
> > > > > >> (incorrect)
> > > > > >> > > > >> > transaction.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> So perhaps it is clearer to
> > > > make
> > > > > >> the
> > > > > >> > > > >> > distinction
> > > > > >> > > > >> > >> > > between
> > > > > >> > > > >> > >> > > > > > > > messages
> > > > > >> > > > >> > >> > > > > > > > > >>>>> that
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> eventually get added to the
> > > > > >> > transaction
> > > > > >> > > > (but
> > > > > >> > > > >> > the
> > > > > >> > > > >> > >> > wrong
> > > > > >> > > > >> > >> > > > > one)
> > > > > >> > > > >> > >> > > > > > or
> > > > > >> > > > >> > >> > > > > > > > > >>>>> messages
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> that never get added and
> > become
> > > > > >> > hanging.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 20)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> The client side change for
> > 2 is
> > > > > >> > removing
> > > > > >> > > > the
> > > > > >> > > > >> > >> > > > addPartitions
> > > > > >> > > > >> > >> > > > > > to
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> transaction
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> call. We don't need to make
> > > > this
> > > > > >> from
> > > > > >> > > the
> > > > > >> > > > >> > >> producer
> > > > > >> > > > >> > >> > to
> > > > > >> > > > >> > >> > > > the
> > > > > >> > > > >> > >> > > > > > txn
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> coordinator,
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> only server side.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> In my opinion, the issue
> > with
> > > > the
> > > > > >> > > > >> > >> addPartitionsToTxn
> > > > > >> > > > >> > >> > > > call
> > > > > >> > > > >> > >> > > > > > for
> > > > > >> > > > >> > >> > > > > > > > > older
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> clients
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> is that we don't have the
> > epoch
> > > > > >> bump,
> > > > > >> > so
> > > > > >> > > > we
> > > > > >> > > > >> > don't
> > > > > >> > > > >> > >> > know
> > > > > >> > > > >> > >> > > > if
> > > > > >> > > > >> > >> > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>> message
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> belongs to the previous
> > > > > >> transaction or
> > > > > >> > > > this
> > > > > >> > > > >> > one.
> > > > > >> > > > >> > >> We
> > > > > >> > > > >> > >> > > need
> > > > > >> > > > >> > >> > > > > to
> > > > > >> > > > >> > >> > > > > > > > check
> > > > > >> > > > >> > >> > > > > > > > > if
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> partition has been added to
> > > > this
> > > > > >> > > > >> transaction.
> > > > > >> > > > >> > Of
> > > > > >> > > > >> > >> > > course,
> > > > > >> > > > >> > >> > > > > > this
> > > > > >> > > > >> > >> > > > > > > > > means
> > > > > >> > > > >> > >> > > > > > > > > >>>>> we
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> won't completely cover the
> > case
> > > > > >> where
> > > > > >> > we
> > > > > >> > > > >> have a
> > > > > >> > > > >> > >> > really
> > > > > >> > > > >> > >> > > > > late
> > > > > >> > > > >> > >> > > > > > > > > message
> > > > > >> > > > >> > >> > > > > > > > > >>>>> and
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> we
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> have added the partition to
> > > > the new
> > > > > >> > > > >> > transaction,
> > > > > >> > > > >> > >> but
> > > > > >> > > > >> > >> > > > > that's
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> unfortunately
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> something we will need the
> > new
> > > > > >> clients
> > > > > >> > > to
> > > > > >> > > > >> > cover.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 30)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Transaction is ongoing =
> > > > partition
> > > > > >> was
> > > > > >> > > > >> added to
> > > > > >> > > > >> > >> > > > > transaction
> > > > > >> > > > >> > >> > > > > > > via
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> addPartitionsToTxn. We
> > check
> > > > this
> > > > > >> with
> > > > > >> > > the
> > > > > >> > > > >> > >> > > > > > > DescribeTransactions
> > > > > >> > > > >> > >> > > > > > > > > >>> call.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> Let
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> me know if this wasn't
> > > > sufficiently
> > > > > >> > > > >> explained
> > > > > >> > > > >> > >> here:
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > >
> > > > > >> > > > >> > >> > > > >
> > > > > >> > > > >> > >> > > >
> > > > > >> > > > >> > >> > >
> > > > > >> > > > >> > >> >
> > > > > >> > > > >> > >>
> > > > > >> > > > >> >
> > > > > >> > > > >>
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 40)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> The idea here is that if
> > any
> > > > > >> messages
> > > > > >> > > > >> somehow
> > > > > >> > > > >> > >> come
> > > > > >> > > > >> > >> > in
> > > > > >> > > > >> > >> > > > > before
> > > > > >> > > > >> > >> > > > > > > we
> > > > > >> > > > >> > >> > > > > > > > > get
> > > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> new
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> epoch to the producer, they
> > > > will be
> > > > > >> > > > fenced.
> > > > > >> > > > >> > >> However,
> > > > > >> > > > >> > >> > > if
> > > > > >> > > > >> > >> > > > we
> > > > > >> > > > >> > >> > > > > > > don't
> > > > > >> > > > >> > >> > > > > > > > > >>>>> think
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> this
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> is necessary, it can be
> > > > discussed
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 50)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> It should be synchronous
> > > > because
> > > > > >> if we
> > > > > >> > > > have
> > > > > >> > > > >> an
> > > > > >> > > > >> > >> event
> > > > > >> > > > >> > >> > > > (ie,
> > > > > >> > > > >> > >> > > > > an
> > > > > >> > > > >> > >> > > > > > > > > error)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> that
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> causes us to need to abort
> > the
> > > > > >> > > > transaction,
> > > > > >> > > > >> we
> > > > > >> > > > >> > >> need
> > > > > >> > > > >> > >> > to
> > > > > >> > > > >> > >> > > > > know
> > > > > >> > > > >> > >> > > > > > > > which
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> partitions to send
> > transaction
> > > > > >> markers
> > > > > >> > > to.
> > > > > >> > > > >> We
> > > > > >> > > > >> > >> know
> > > > > >> > > > >> > >> > the
> > > > > >> > > > >> > >> > > > > > > > partitions
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> because
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> we added them to the
> > > > coordinator
> > > > > >> via
> > > > > >> > the
> > > > > >> > > > >> > >> > > > > addPartitionsToTxn
> > > > > >> > > > >> > >> > > > > > > > call.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Previously we have had
> > > > asynchronous
> > > > > >> > > calls
> > > > > >> > > > in
> > > > > >> > > > >> > the
> > > > > >> > > > >> > >> > past
> > > > > >> > > > >> > >> > > > (ie,
> > > > > >> > > > >> > >> > > > > > > > writing
> > > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> commit markers when the
> > > > > >> transaction is
> > > > > >> > > > >> > completed)
> > > > > >> > > > >> > >> > but
> > > > > >> > > > >> > >> > > > > often
> > > > > >> > > > >> > >> > > > > > > this
> > > > > >> > > > >> > >> > > > > > > > > >>> just
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> causes confusion as we
> > need to
> > > > wait
> > > > > >> > for
> > > > > >> > > > some
> > > > > >> > > > >> > >> > > operations
> > > > > >> > > > >> > >> > > > to
> > > > > >> > > > >> > >> > > > > > > > > complete.
> > > > > >> > > > >> > >> > > > > > > > > >>>>> In
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> writing commit markers
> > case,
> > > > > >> clients
> > > > > >> > > often
> > > > > >> > > > >> see
> > > > > >> > > > >> > >> > > > > > > > > >>>>> CONCURRENT_TRANSACTIONs
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> error messages and that
> > can be
> > > > > >> > > confusing.
> > > > > >> > > > >> For
> > > > > >> > > > >> > >> that
> > > > > >> > > > >> > >> > > > reason,
> > > > > >> > > > >> > >> > > > > > it
> > > > > >> > > > >> > >> > > > > > > > may
> > > > > >> > > > >> > >> > > > > > > > > be
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> simpler to just have
> > > > synchronous
> > > > > >> > calls —
> > > > > >> > > > >> > >> especially
> > > > > >> > > > >> > >> > if
> > > > > >> > > > >> > >> > > > we
> > > > > >> > > > >> > >> > > > > > need
> > > > > >> > > > >> > >> > > > > > > > to
> > > > > >> > > > >> > >> > > > > > > > > >>>>> block
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> on
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> some operation's completion
> > > > anyway
> > > > > >> > > before
> > > > > >> > > > we
> > > > > >> > > > >> > can
> > > > > >> > > > >> > >> > start
> > > > > >> > > > >> > >> > > > the
> > > > > >> > > > >> > >> > > > > > > next
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> transaction. And yes, I
> > meant
> > > > > >> > > > coordinator. I
> > > > > >> > > > >> > will
> > > > > >> > > > >> > >> > fix
> > > > > >> > > > >> > >> > > > > that.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 60)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> When we are checking if the
> > > > > >> > transaction
> > > > > >> > > is
> > > > > >> > > > >> > >> ongoing,
> > > > > >> > > > >> > >> > we
> > > > > >> > > > >> > >> > > > > need
> > > > > >> > > > >> > >> > > > > > to
> > > > > >> > > > >> > >> > > > > > > > > make
> > > > > >> > > > >> > >> > > > > > > > > >>> a
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> round
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> trip from the leader
> > partition
> > > > to
> > > > > >> the
> > > > > >> > > > >> > transaction
> > > > > >> > > > >> > >> > > > > > coordinator.
> > > > > >> > > > >> > >> > > > > > > > In
> > > > > >> > > > >> > >> > > > > > > > > >>> the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> time
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> we are waiting for this
> > > > message to
> > > > > >> > come
> > > > > >> > > > >> back,
> > > > > >> > > > >> > in
> > > > > >> > > > >> > >> > > theory
> > > > > >> > > > >> > >> > > > we
> > > > > >> > > > >> > >> > > > > > > could
> > > > > >> > > > >> > >> > > > > > > > > >>> have
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> sent
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> a commit/abort call that
> > would
> > > > make
> > > > > >> > the
> > > > > >> > > > >> > original
> > > > > >> > > > >> > >> > > result
> > > > > >> > > > >> > >> > > > of
> > > > > >> > > > >> > >> > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > check
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> out
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> of
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> date. That is why we can
> > check
> > > > the
> > > > > >> > > leader
> > > > > >> > > > >> state
> > > > > >> > > > >> > >> > before
> > > > > >> > > > >> > >> > > > we
> > > > > >> > > > >> > >> > > > > > > write
> > > > > >> > > > >> > >> > > > > > > > to
> > > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> log.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> I'm happy to update the
> > KIP if
> > > > > >> some of
> > > > > >> > > > these
> > > > > >> > > > >> > >> things
> > > > > >> > > > >> > >> > > were
> > > > > >> > > > >> > >> > > > > not
> > > > > >> > > > >> > >> > > > > > > > > clear.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks,
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Justine
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> On Mon, Nov 21, 2022 at
> > 7:11 PM
> > > > > >> > Matthias
> > > > > >> > > > J.
> > > > > >> > > > >> > Sax <
> > > > > >> > > > >> > >> > > > > > > > mjsax@apache.org
> > > > > >> > > > >> > >> > > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> wrote:
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Thanks for the KIP.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Couple of clarification
> > > > questions
> > > > > >> (I
> > > > > >> > am
> > > > > >> > > > >> not a
> > > > > >> > > > >> > >> > broker
> > > > > >> > > > >> > >> > > > > expert
> > > > > >> > > > >> > >> > > > > > > do
> > > > > >> > > > >> > >> > > > > > > > > >>>>> maybe
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> some question are obvious
> > for
> > > > > >> others,
> > > > > >> > > but
> > > > > >> > > > >> not
> > > > > >> > > > >> > >> for
> > > > > >> > > > >> > >> > me
> > > > > >> > > > >> > >> > > > with
> > > > > >> > > > >> > >> > > > > > my
> > > > > >> > > > >> > >> > > > > > > > lack
> > > > > >> > > > >> > >> > > > > > > > > >>>>> of
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> broker knowledge).
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (10)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> 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.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> What happens if the
> > message
> > > > come
> > > > > >> in
> > > > > >> > > > before
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> next
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> addPartitionsToTxn
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> request? It seems the
> > broker
> > > > > >> hosting
> > > > > >> > > the
> > > > > >> > > > >> data
> > > > > >> > > > >> > >> > > > partitions
> > > > > >> > > > >> > >> > > > > > > won't
> > > > > >> > > > >> > >> > > > > > > > > know
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> anything about it and
> > append
> > > > it to
> > > > > >> > the
> > > > > >> > > > >> > >> partition,
> > > > > >> > > > >> > >> > > too?
> > > > > >> > > > >> > >> > > > > What
> > > > > >> > > > >> > >> > > > > > > is
> > > > > >> > > > >> > >> > > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> difference between both
> > cases?
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Also, it seems a TX would
> > only
> > > > > >> hang,
> > > > > >> > if
> > > > > >> > > > >> there
> > > > > >> > > > >> > >> is no
> > > > > >> > > > >> > >> > > > > > following
> > > > > >> > > > >> > >> > > > > > > > TX
> > > > > >> > > > >> > >> > > > > > > > > >>>>> that
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> is
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> either committer or
> > aborted?
> > > > Thus,
> > > > > >> > for
> > > > > >> > > > the
> > > > > >> > > > >> > case
> > > > > >> > > > >> > >> > > above,
> > > > > >> > > > >> > >> > > > > the
> > > > > >> > > > >> > >> > > > > > TX
> > > > > >> > > > >> > >> > > > > > > > > might
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> actually not hang (of
> > course,
> > > > we
> > > > > >> > might
> > > > > >> > > > get
> > > > > >> > > > >> an
> > > > > >> > > > >> > >> EOS
> > > > > >> > > > >> > >> > > > > violation
> > > > > >> > > > >> > >> > > > > > > if
> > > > > >> > > > >> > >> > > > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> first
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> TX was aborted and the
> > second
> > > > > >> > > committed,
> > > > > >> > > > or
> > > > > >> > > > >> > the
> > > > > >> > > > >> > >> > other
> > > > > >> > > > >> > >> > > > way
> > > > > >> > > > >> > >> > > > > > > > > around).
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (20)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Of course, 1 and 2
> > require
> > > > > >> > client-side
> > > > > >> > > > >> > >> changes, so
> > > > > >> > > > >> > >> > > for
> > > > > >> > > > >> > >> > > > > > older
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> clients,
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> those approaches won’t
> > apply.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> For (1) I understand why a
> > > > client
> > > > > >> > > change
> > > > > >> > > > is
> > > > > >> > > > >> > >> > > necessary,
> > > > > >> > > > >> > >> > > > > but
> > > > > >> > > > >> > >> > > > > > > not
> > > > > >> > > > >> > >> > > > > > > > > sure
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> why
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> we need a client change
> > for
> > > > (2).
> > > > > >> Can
> > > > > >> > > you
> > > > > >> > > > >> > >> elaborate?
> > > > > >> > > > >> > >> > > --
> > > > > >> > > > >> > >> > > > > > Later
> > > > > >> > > > >> > >> > > > > > > > you
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> explain
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> that we should send a
> > > > > >> > > > >> > >> DescribeTransactionRequest,
> > > > > >> > > > >> > >> > > but I
> > > > > >> > > > >> > >> > > > > am
> > > > > >> > > > >> > >> > > > > > > not
> > > > > >> > > > >> > >> > > > > > > > > sure
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> why?
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Can't we not just do an
> > > > implicit
> > > > > >> > > > >> > >> AddPartiitonToTx,
> > > > > >> > > > >> > >> > > too?
> > > > > >> > > > >> > >> > > > > If
> > > > > >> > > > >> > >> > > > > > > the
> > > > > >> > > > >> > >> > > > > > > > > old
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> producer correctly
> > registered
> > > > the
> > > > > >> > > > partition
> > > > > >> > > > >> > >> > already,
> > > > > >> > > > >> > >> > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> TX-coordinator
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> can just ignore it as
> > it's an
> > > > > >> > > idempotent
> > > > > >> > > > >> > >> operation?
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (30)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> To cover older clients,
> > we
> > > > will
> > > > > >> > > ensure a
> > > > > >> > > > >> > >> > transaction
> > > > > >> > > > >> > >> > > > is
> > > > > >> > > > >> > >> > > > > > > > ongoing
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> before
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> we write to a transaction
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Not sure what you mean by
> > > > this?
> > > > > >> Can
> > > > > >> > you
> > > > > >> > > > >> > >> elaborate?
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (40)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> [the TX-coordinator] will
> > > > write
> > > > > >> the
> > > > > >> > > > >> prepare
> > > > > >> > > > >> > >> commit
> > > > > >> > > > >> > >> > > > > message
> > > > > >> > > > >> > >> > > > > > > > with
> > > > > >> > > > >> > >> > > > > > > > > a
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>> bumped
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> epoch and send
> > > > > >> WriteTxnMarkerRequests
> > > > > >> > > > with
> > > > > >> > > > >> the
> > > > > >> > > > >> > >> > bumped
> > > > > >> > > > >> > >> > > > > > epoch.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Why do we use the bumped
> > > > epoch for
> > > > > >> > > both?
> > > > > >> > > > It
> > > > > >> > > > >> > >> seems
> > > > > >> > > > >> > >> > > more
> > > > > >> > > > >> > >> > > > > > > > intuitive
> > > > > >> > > > >> > >> > > > > > > > > to
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> use
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> the current epoch, and
> > only
> > > > return
> > > > > >> > the
> > > > > >> > > > >> bumped
> > > > > >> > > > >> > >> epoch
> > > > > >> > > > >> > >> > > to
> > > > > >> > > > >> > >> > > > > the
> > > > > >> > > > >> > >> > > > > > > > > >>>>> producer?
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (50) "Implicit
> > > > > >> > > AddPartitionToTransaction"
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Why does the implicitly
> > sent
> > > > > >> request
> > > > > >> > > need
> > > > > >> > > > >> to
> > > > > >> > > > >> > be
> > > > > >> > > > >> > >> > > > > > synchronous?
> > > > > >> > > > >> > >> > > > > > > > The
> > > > > >> > > > >> > >> > > > > > > > > >>>>> KIP
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> also says
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> in case we need to abort
> > and
> > > > > >> need to
> > > > > >> > > > know
> > > > > >> > > > >> > which
> > > > > >> > > > >> > >> > > > > partitions
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> What do you mean by this?
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> we don’t want to write
> > to it
> > > > > >> before
> > > > > >> > we
> > > > > >> > > > >> store
> > > > > >> > > > >> > in
> > > > > >> > > > >> > >> > the
> > > > > >> > > > >> > >> > > > > > > > transaction
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>> manager
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Do you mean TX-coordinator
> > > > > >> instead of
> > > > > >> > > > >> > "manager"?
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (60)
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> For older clients and
> > ensuring
> > > > > >> that
> > > > > >> > the
> > > > > >> > > > TX
> > > > > >> > > > >> is
> > > > > >> > > > >> > >> > > ongoing,
> > > > > >> > > > >> > >> > > > > you
> > > > > >> > > > >> > >> > > > > > > > > >>>>> describe a
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> race condition. I am not
> > sure
> > > > if I
> > > > > >> > can
> > > > > >> > > > >> follow
> > > > > >> > > > >> > >> here.
> > > > > >> > > > >> > >> > > Can
> > > > > >> > > > >> > >> > > > > you
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> elaborate?
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> -Matthias
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> On 11/18/22 1:21 PM,
> > Justine
> > > > > >> Olshan
> > > > > >> > > > wrote:
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Hey all!
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> I'd like to start a
> > > > discussion
> > > > > >> on my
> > > > > >> > > > >> proposal
> > > > > >> > > > >> > >> to
> > > > > >> > > > >> > >> > add
> > > > > >> > > > >> > >> > > > > some
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> server-side
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> checks on transactions to
> > > > avoid
> > > > > >> > > hanging
> > > > > >> > > > >> > >> > > transactions.
> > > > > >> > > > >> > >> > > > I
> > > > > >> > > > >> > >> > > > > > know
> > > > > >> > > > >> > >> > > > > > > > > this
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> has
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> been
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> an issue for some time,
> > so I
> > > > > >> really
> > > > > >> > > hope
> > > > > >> > > > >> this
> > > > > >> > > > >> > >> KIP
> > > > > >> > > > >> > >> > > will
> > > > > >> > > > >> > >> > > > > be
> > > > > >> > > > >> > >> > > > > > > > > helpful
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> for
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> many
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> users of EOS.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> The KIP includes changes
> > that
> > > > > >> will
> > > > > >> > be
> > > > > >> > > > >> > >> compatible
> > > > > >> > > > >> > >> > > with
> > > > > >> > > > >> > >> > > > > old
> > > > > >> > > > >> > >> > > > > > > > > clients
> > > > > >> > > > >> > >> > > > > > > > > >>>>>> and
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> changes to improve
> > > > performance
> > > > > >> and
> > > > > >> > > > >> > correctness
> > > > > >> > > > >> > >> on
> > > > > >> > > > >> > >> > > new
> > > > > >> > > > >> > >> > > > > > > clients.
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Please take a look and
> > leave
> > > > any
> > > > > >> > > > comments
> > > > > >> > > > >> you
> > > > > >> > > > >> > >> may
> > > > > >> > > > >> > >> > > > have!
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> KIP:
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > >
> > > > > >> > > > >> > >> > > > >
> > > > > >> > > > >> > >> > > >
> > > > > >> > > > >> > >> > >
> > > > > >> > > > >> > >> >
> > > > > >> > > > >> > >>
> > > > > >> > > > >> >
> > > > > >> > > > >>
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> JIRA:
> > > > > >> > > > >> > >> > > >
> > https://issues.apache.org/jira/browse/KAFKA-14402
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Thanks!
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Justine
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > > >> > > > >> > >> > > > > > > > > >>>
> > > > > >> > > > >> > >> > > > > > > > > >>
> > > > > >> > > > >> > >> > > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > > >
> > > > > >> > > > >> > >> > > > > > > >
> > > > > >> > > > >> > >> > > > > > >
> > > > > >> > > > >> > >> > > > > >
> > > > > >> > > > >> > >> > > > >
> > > > > >> > > > >> > >> > > >
> > > > > >> > > > >> > >> > >
> > > > > >> > > > >> > >> >
> > > > > >> > > > >> > >>
> > > > > >> > > > >> > >
> > > > > >> > > > >> >
> > > > > >> > > > >>
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > > >
> > > >
> >

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
That's a fair point about other clients.

I think the abortable error case is interesting because I'm curious how
other clients would handle this. I assume they would need to implement
handling for the error code unless they did something like "any unknown
error codes/any codes that aren't x,y,z are retriable." I would hope that
unknown error codes were fatal, and if the code was implemented it would
abort the transaction. But I will think on this too.

As for InvalidRecord -- you mentioned it was not fatal, but I'm taking a
look through the code. We would see this on handling the produce response.
If I recall correctly, we check if errors are retriable. I think this error
would not be retriable. But I guess the concern here is that it is not
enough for just that batch to fail. I guess I hadn't considered fully
fencing the old producer but there are valid arguments here why we would
want to.

Thanks,
Justine

On Fri, Jan 20, 2023 at 2:35 PM Guozhang Wang <gu...@gmail.com>
wrote:

> Thanks Justine for the replies! I agree with most of your thoughts.
>
> Just for 3/7), though I agree for our own AK producer, since we do
> "nextRequest(boolean hasIncompleteBatches)", we guarantee the end-txn
> would not be sent until we've effectively flushed, but I was referring
> to any future bugs or other buggy clients that the same client may get
> into this situation, in which case we should give the client a clear
> msg that "you did something wrong, and hence now you should fatally
> close yourself". What I'm concerned about is that, by seeing an
> "abortable error" or in some rare cases an "invalid record", the
> client could not realize "something that's really bad happened". So
> it's not about adding a new error, it's mainly about those real buggy
> situations causing such "should never happen" cases, the errors return
> would not be informative enough.
>
> Thinking in other ways, if we believe that for most cases such error
> codes would not reach the original clients since they would be
> disconnected or even gone by that time, and only in some rare cases
> they would still be seen by the sending clients, then why not make
> them more fatal and more specific than generic.
>
> Guozhang
>
> On Fri, Jan 20, 2023 at 1:59 PM Justine Olshan
> <jo...@confluent.io.invalid> wrote:
> >
> > Hey Guozhang. Thanks for taking a look and for the detailed comments!
> I'll
> > do my best to address below.
> >
> > 1. I see what you are saying here, but I think I need to look through the
> > sequence of events you mention. Typically we've seen this issue in a few
> > cases.
> >
> >  One is when we have a producer disconnect when trying to produce.
> > Typically in these cases, we abort the transaction. We've seen that after
> > the markers are written, the disconnection can sometimes cause the
> request
> > to get flushed to the broker. In this case, we don't need client handling
> > because the producer we are responding to is gone. We just needed to make
> > sure we didn't write to the log on the broker side. I'm trying to think
> of
> > a case where we do have the client to return to. I'd think the same
> client
> > couldn't progress to committing the transaction unless the produce
> request
> > returned right? Of course, there is the incorrectly written clients case.
> > I'll think on this a bit more and let you know if I come up with another
> > scenario when we would return to an active client when the transaction is
> > no longer ongoing.
> >
> > I was not aware that we checked the result of a send after we commit
> > though. I'll need to look into that a bit more.
> >
> > 2. There were some questions about this in the discussion. The plan is to
> > handle overflow with the mechanism we currently have in the producer. If
> we
> > try to bump and the epoch will overflow, we actually allocate a new
> > producer ID. I need to confirm the fencing logic on the last epoch (ie,
> we
> > probably shouldn't allow any records to be produced with the final epoch
> > since we can never properly fence that one).
> >
> > 3. I can agree with you that the current error handling is messy. I
> recall
> > taking a look at your KIP a while back, but I think I mostly saw the
> > section about how the errors were wrapped. Maybe I need to take another
> > look. As for abortable error, the idea was that the handling would be
> > simple -- if this error is seen, the transaction should be aborted -- no
> > other logic about previous state or requests necessary. Is your concern
> > simply about adding new errors? We were hoping to have an error that
> would
> > have one meaning and many of the current errors have a history of meaning
> > different things on different client versions. That was the main
> motivation
> > for adding a new error.
> >
> > 4. This is a good point about record timestamp reordering. Timestamps
> don't
> > affect compaction, but they do affect retention deletion. For that, kafka
> > considers the largest timestamp in the segment, so I think a small amount
> > of reordering (hopefully on the order of milliseconds or even seconds)
> will
> > be ok. We take timestamps from clients so there is already a possibility
> > for some drift and non-monotonically increasing timestamps.
> >
> > 5. Thanks for catching. The error is there, but it's actually that those
> > fields should be 4+! Due to how the message generator works, I actually
> > have to redefine those fields inside the `"AddPartitionsToTxnTransaction`
> > block for it to build correctly. I'll fix it to be correct.
> >
> > 6. Correct -- we will only add the request to purgatory if the cache has
> no
> > ongoing transaction. I can change the wording to make that clearer that
> we
> > only place the request in purgatory if we need to contact the transaction
> > coordinator.
> >
> > 7. We did take a look at some of the errors and it was hard to come up
> with
> > a good one. I agree that InvalidTxnStateException is ideal except for the
> > fact that it hasn't been returned on Produce requests before. The error
> > handling for clients is a bit vague (which is why I opened KAFKA-14439
> > <https://issues.apache.org/jira/browse/KAFKA-14439>), but the decision
> we
> > made here was to only return errors that have been previously returned to
> > producers. As for not being fatal, I think part of the theory was that in
> > many cases, the producer would be disconnected. (See point 1) and this
> > would just be an error to return from the server. I did plan to think
> about
> > other cases, so let me know if you think of any as well!
> >
> > Lots to say! Let me know if you have further thoughts!
> > Justine
> >
> > On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang <
> guozhang.wang.us@gmail.com>
> > wrote:
> >
> > > Hello Justine,
> > >
> > > Thanks for the great write-up! I made a quick pass through it and here
> > > are some thoughts (I have not been able to read through this thread so
> > > pardon me if they have overlapped or subsumed by previous comments):
> > >
> > > First are some meta ones:
> > >
> > > 1. I think we need to also improve the client's experience once we
> > > have this defence in place. More concretely, say a user's producer
> > > code is like following:
> > >
> > > future = producer.send();
> > > // producer.flush();
> > > producer.commitTransaction();
> > > future.get();
> > >
> > > Which resulted in the order of a) produce-request sent by producer, b)
> > > end-txn-request sent by producer, c) end-txn-response sent back, d)
> > > txn-marker-request sent from coordinator to partition leader, e)
> > > produce-request finally received by the partition leader, before this
> > > KIP e) step would be accepted causing a dangling txn; now it would be
> > > rejected in step e) which is good. But from the client's point of view
> > > now it becomes confusing since the `commitTransaction()` returns
> > > successfully, but the "future" throws an invalid-epoch error, and they
> > > are not sure if the transaction did succeed or not. In fact, it
> > > "partially succeeded" with some msgs being rejected but others
> > > committed successfully.
> > >
> > > Of course the easy way to avoid this is, always call
> > > "producer.flush()" before commitTxn and that's what we do ourselves,
> > > and what we recommend users do. But I suspect not everyone does it. In
> > > fact I just checked the javadoc in KafkaProducer and our code snippet
> > > does not include a `flush()` call. So I'm thinking maybe we can in
> > > side the `commitTxn` code to enforce flushing before sending the
> > > end-txn request.
> > >
> > > 2. I'd like to clarify a bit details on "just add partitions to the
> > > transaction on the first produce request during a transaction". My
> > > understanding is that the partition leader's cache has the producer id
> > > / sequence / epoch for the latest txn, either on-going or is completed
> > > (upon receiving the marker request from coordinator). When a produce
> > > request is received, if
> > >
> > > * producer's epoch < cached epoch, or producer's epoch == cached epoch
> > > but the latest txn is completed, leader directly reject with
> > > invalid-epoch.
> > > * producer's epoch > cached epoch, park the the request and send
> > > add-partitions request to coordinator.
> > >
> > > In order to do it, does the coordinator need to bump the sequence and
> > > reset epoch to 0 when the next epoch is going to overflow? If no need
> > > to do so, then how we handle the (admittedly rare, but still may
> > > happen) epoch overflow situation?
> > >
> > > 3. I'm a bit concerned about adding a generic "ABORTABLE_ERROR" given
> > > we already have a pretty messy error classification and error handling
> > > on the producer clients side --- I have a summary about the issues and
> > > a proposal to address this in
> > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
> > > -- I understand we do not want to use "UNKNOWN_PRODUCER_ID" anymore
> > > and in fact we intend to deprecate it in KIP-360 and eventually remove
> > > it; but I'm wondering can we still use specific error codes. E.g. what
> > > about "InvalidProducerEpochException" since for new clients, the
> > > actual reason this would actually be rejected is indeed because the
> > > epoch on the coordinator caused the add-partitions-request from the
> > > brokers to be rejected anyways?
> > >
> > > 4. It seems we put the producer request into purgatory before we ever
> > > append the records, while other producer's records may still be
> > > appended during the time; and that potentially may result in some
> > > re-ordering compared with reception order. I'm not super concerned
> > > about it since Kafka does not guarantee reception ordering across
> > > producers anyways, but it may make the timestamps of records inside a
> > > partition to be more out-of-ordered. Are we aware of any scenarios
> > > such as future enhancements on log compactions that may be affected by
> > > this effect?
> > >
> > > Below are just minor comments:
> > >
> > > 5. In "AddPartitionsToTxnTransaction" field of
> > > "AddPartitionsToTxnRequest" RPC, the versions of those inner fields
> > > are "0-3" while I thought they should be "0+" still?
> > >
> > > 6. Regarding "we can place the request in a purgatory of sorts and
> > > check if there is any state for the transaction on the broker": i
> > > think at this time when we just do the checks against the cached
> > > state, we do not need to put the request to purgatory yet?
> > >
> > > 7. This is related to 3) above. I feel using "InvalidRecordException"
> > > for older clients may also be a bit confusing, and also it is not
> > > fatal -- for old clients, it better to be fatal since this indicates
> > > the clients is doing something wrong and hence it should be closed.
> > > And in general I'd prefer to use slightly more specific meaning error
> > > codes for clients. That being said, I also feel
> > > "InvalidProducerEpochException" is not suitable for old versioned
> > > clients, and we'd have to pick one that old clients recognize. I'd
> > > prefer "InvalidTxnStateException" but that one is supposed to be
> > > returned from txn coordinators only today. I'd suggest we do a quick
> > > check in the current client's code path and see if that one would be
> > > handled if it's from a produce-response, and if yes, use this one;
> > > otherwise, use "ProducerFencedException" which is much less meaningful
> > > but it's still a fatal error.
> > >
> > >
> > > Thanks,
> > > Guozhang
> > >
> > >
> > >
> > > On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
> > > <jo...@confluent.io.invalid> wrote:
> > > >
> > > > Yeah -- looks like we already have code to handle bumping the epoch
> and
> > > > when the epoch is Short.MAX_VALUE, we get a new producer ID. Since
> this
> > > is
> > > > already the behavior, do we want to change it further?
> > > >
> > > > Justine
> > > >
> > > > On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <jolshan@confluent.io
> >
> > > wrote:
> > > >
> > > > > Hey all, just wanted to quickly update and say I've modified the
> KIP to
> > > > > explicitly mention that AddOffsetCommitsToTxnRequest will be
> replaced
> > > by
> > > > > a coordinator-side (inter-broker) AddPartitionsToTxn implicit
> request.
> > > This
> > > > > mirrors the user partitions and will implicitly add offset
> partitions
> > > to
> > > > > transactions when we commit offsets on them. We will deprecate
> > > AddOffsetCommitsToTxnRequest
> > > > > for new clients.
> > > > >
> > > > > Also to address Artem's comments --
> > > > > I'm a bit unsure if the changes here will change the previous
> behavior
> > > for
> > > > > fencing producers. In the case you mention in the first paragraph,
> are
> > > you
> > > > > saying we bump the epoch before we try to abort the transaction? I
> > > think I
> > > > > need to understand the scenarios you mention a bit better.
> > > > >
> > > > > As for the second part -- I think it makes sense to have some sort
> of
> > > > > "sentinel" epoch to signal epoch is about to overflow (I think we
> sort
> > > of
> > > > > have this value in place in some ways) so we can codify it in the
> KIP.
> > > I'll
> > > > > look into that and try to update soon.
> > > > >
> > > > > Thanks,
> > > > > Justine.
> > > > >
> > > > > On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
> > > > > <al...@confluent.io.invalid> wrote:
> > > > >
> > > > >> It's good to know that KIP-588 addressed some of the issues.
> Looking
> > > at
> > > > >> the code, it still looks like there are some cases that would
> result
> > > in
> > > > >> fatal error, e.g. PRODUCER_FENCED is issued by the transaction
> > > coordinator
> > > > >> if epoch doesn't match, and the client treats it as a fatal error
> > > (code in
> > > > >> TransactionManager request handling).  If we consider, for
> example,
> > > > >> committing a transaction that returns a timeout, but actually
> > > succeeds,
> > > > >> trying to abort it or re-commit may result in PRODUCER_FENCED
> error
> > > > >> (because of epoch bump).
> > > > >>
> > > > >> For failed commits, specifically, we need to know the actual
> outcome,
> > > > >> because if we return an error the application may think that the
> > > > >> transaction is aborted and redo the work, leading to duplicates.
> > > > >>
> > > > >> Re: overflowing epoch.  We could either do it on the TC and return
> > > both
> > > > >> producer id and epoch (e.g. change the protocol), or signal the
> client
> > > > >> that
> > > > >> it needs to get a new producer id.  Checking for max epoch could
> be a
> > > > >> reasonable signal, the value to check should probably be present
> in
> > > the
> > > > >> KIP
> > > > >> as this is effectively a part of the contract.  Also, the TC
> should
> > > > >> probably return an error if the client didn't change producer id
> after
> > > > >> hitting max epoch.
> > > > >>
> > > > >> -Artem
> > > > >>
> > > > >>
> > > > >> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
> > > > >> <jo...@confluent.io.invalid> wrote:
> > > > >>
> > > > >> > Thanks for the discussion Artem.
> > > > >> >
> > > > >> > With respect to the handling of fenced producers, we have some
> > > behavior
> > > > >> > already in place. As of KIP-588:
> > > > >> >
> > > > >> >
> > > > >>
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> > > > >> > ,
> > > > >> > we handle timeouts more gracefully. The producer can recover.
> > > > >> >
> > > > >> > Produce requests can also recover from epoch fencing by
> aborting the
> > > > >> > transaction and starting over.
> > > > >> >
> > > > >> > What other cases were you considering that would cause us to
> have a
> > > > >> fenced
> > > > >> > epoch but we'd want to recover?
> > > > >> >
> > > > >> > The first point about handling epoch overflows is fair. I think
> > > there is
> > > > >> > some logic we'd need to consider. (ie, if we are one away from
> the
> > > max
> > > > >> > epoch, we need to reset the producer ID.) I'm still wondering if
> > > there
> > > > >> is a
> > > > >> > way to direct this from the response, or if everything should be
> > > done on
> > > > >> > the client side. Let me know if you have any thoughts here.
> > > > >> >
> > > > >> > Thanks,
> > > > >> > Justine
> > > > >> >
> > > > >> > On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
> > > > >> > <al...@confluent.io.invalid> wrote:
> > > > >> >
> > > > >> > > There are some workflows in the client that are implied by
> > > protocol
> > > > >> > > changes, e.g.:
> > > > >> > >
> > > > >> > > - for new clients, epoch changes with every transaction and
> can
> > > > >> overflow,
> > > > >> > > in old clients this condition was handled transparently,
> because
> > > epoch
> > > > >> > was
> > > > >> > > bumped in InitProducerId and it would return a new producer
> id if
> > > > >> epoch
> > > > >> > > overflows, the new clients would need to implement some
> workflow
> > > to
> > > > >> > refresh
> > > > >> > > producer id
> > > > >> > > - how to handle fenced producers, for new clients epoch
> changes
> > > with
> > > > >> > every
> > > > >> > > transaction, so in presence of failures during commits /
> aborts,
> > > the
> > > > >> > > producer could get easily fenced, old clients would pretty
> much
> > > would
> > > > >> get
> > > > >> > > fenced when a new incarnation of the producer was initialized
> with
> > > > >> > > InitProducerId so it's ok to treat as a fatal error, the new
> > > clients
> > > > >> > would
> > > > >> > > need to implement some workflow to handle that error,
> otherwise
> > > they
> > > > >> > could
> > > > >> > > get fenced by themselves
> > > > >> > > - in particular (as a subset of the previous issue), what
> would
> > > the
> > > > >> > client
> > > > >> > > do if it got a timeout during commit?  commit could've
> succeeded
> > > or
> > > > >> > failed
> > > > >> > >
> > > > >> > > Not sure if this has to be defined in the KIP as implementing
> > > those
> > > > >> > > probably wouldn't require protocol changes, but we have
> multiple
> > > > >> > > implementations of Kafka clients, so probably would be good to
> > > have
> > > > >> some
> > > > >> > > client implementation guidance.  Could also be done as a
> separate
> > > doc.
> > > > >> > >
> > > > >> > > -Artem
> > > > >> > >
> > > > >> > > On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
> > > > >> > <jolshan@confluent.io.invalid
> > > > >> > > >
> > > > >> > > wrote:
> > > > >> > >
> > > > >> > > > Hey all, I've updated the KIP to incorporate Jason's
> > > suggestions.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > 1. Use AddPartitionsToTxn + verify flag to check on old
> clients
> > > > >> > > > 2. Updated AddPartitionsToTxn API to support transaction
> > > batching
> > > > >> > > > 3. Mention IBP bump
> > > > >> > > > 4. Mention auth change on new AddPartitionsToTxn version.
> > > > >> > > >
> > > > >> > > > I'm planning on opening a vote soon.
> > > > >> > > > Thanks,
> > > > >> > > > Justine
> > > > >> > > >
> > > > >> > > > On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <
> > > jolshan@confluent.io
> > > > >> >
> > > > >> > > > wrote:
> > > > >> > > >
> > > > >> > > > > Thanks Jason. Those changes make sense to me. I will
> update
> > > the
> > > > >> KIP.
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > > > On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
> > > > >> > > > <ja...@confluent.io.invalid>
> > > > >> > > > > wrote:
> > > > >> > > > >
> > > > >> > > > >> Hey Justine,
> > > > >> > > > >>
> > > > >> > > > >> > I was wondering about compatibility here. When we send
> > > requests
> > > > >> > > > >> between brokers, we want to ensure that the receiving
> broker
> > > > >> > > understands
> > > > >> > > > >> the request (specifically the new fields). Typically
> this is
> > > done
> > > > >> > via
> > > > >> > > > >> IBP/metadata version.
> > > > >> > > > >> I'm trying to think if there is a way around it but I'm
> not
> > > sure
> > > > >> > there
> > > > >> > > > is.
> > > > >> > > > >>
> > > > >> > > > >> Yes. I think we would gate usage of this behind an IBP
> bump.
> > > Does
> > > > >> > that
> > > > >> > > > >> seem
> > > > >> > > > >> reasonable?
> > > > >> > > > >>
> > > > >> > > > >> > As for the improvements -- can you clarify how the
> multiple
> > > > >> > > > >> transactional
> > > > >> > > > >> IDs would help here? Were you thinking of a case where we
> > > > >> wait/batch
> > > > >> > > > >> multiple produce requests together? My understanding for
> now
> > > was
> > > > >> 1
> > > > >> > > > >> transactional ID and one validation per 1 produce
> request.
> > > > >> > > > >>
> > > > >> > > > >> Each call to `AddPartitionsToTxn` is essentially a write
> to
> > > the
> > > > >> > > > >> transaction
> > > > >> > > > >> log and must block on replication. The more we can fit
> into a
> > > > >> single
> > > > >> > > > >> request, the more writes we can do in parallel. The
> > > alternative
> > > > >> is
> > > > >> > to
> > > > >> > > > make
> > > > >> > > > >> use of more connections, but usually we prefer batching
> > > since the
> > > > >> > > > network
> > > > >> > > > >> stack is not really optimized for high connection/request
> > > loads.
> > > > >> > > > >>
> > > > >> > > > >> > Finally with respect to the authorizations, I think it
> > > makes
> > > > >> sense
> > > > >> > > to
> > > > >> > > > >> skip
> > > > >> > > > >> topic authorizations, but I'm a bit confused by the
> "leader
> > > ID"
> > > > >> > field.
> > > > >> > > > >> Wouldn't we just want to flag the request as from a
> broker
> > > (does
> > > > >> it
> > > > >> > > > matter
> > > > >> > > > >> which one?).
> > > > >> > > > >>
> > > > >> > > > >> We could also make it version-based. For the next
> version, we
> > > > >> could
> > > > >> > > > >> require
> > > > >> > > > >> CLUSTER auth. So clients would not be able to use the API
> > > > >> anymore,
> > > > >> > > which
> > > > >> > > > >> is
> > > > >> > > > >> probably what we want.
> > > > >> > > > >>
> > > > >> > > > >> -Jason
> > > > >> > > > >>
> > > > >> > > > >> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> > > > >> > > > >> <jo...@confluent.io.invalid>
> > > > >> > > > >> wrote:
> > > > >> > > > >>
> > > > >> > > > >> > As a follow up, I was just thinking about the batching
> a
> > > bit
> > > > >> more.
> > > > >> > > > >> > I suppose if we have one request in flight and we
> queue up
> > > the
> > > > >> > other
> > > > >> > > > >> > produce requests in some sort of purgatory, we could
> send
> > > > >> > > information
> > > > >> > > > >> out
> > > > >> > > > >> > for all of them rather than one by one. So that would
> be a
> > > > >> benefit
> > > > >> > > of
> > > > >> > > > >> > batching partitions to add per transaction.
> > > > >> > > > >> >
> > > > >> > > > >> > I'll need to think a bit more on the design of this
> part
> > > of the
> > > > >> > KIP,
> > > > >> > > > and
> > > > >> > > > >> > will update the KIP in the next few days.
> > > > >> > > > >> >
> > > > >> > > > >> > Thanks,
> > > > >> > > > >> > Justine
> > > > >> > > > >> >
> > > > >> > > > >> > On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan <
> > > > >> > > jolshan@confluent.io>
> > > > >> > > > >> > wrote:
> > > > >> > > > >> >
> > > > >> > > > >> > > Hey Jason -- thanks for the input -- I was just
> digging
> > > a bit
> > > > >> > > deeper
> > > > >> > > > >> into
> > > > >> > > > >> > > the design + implementation of the validation calls
> here
> > > and
> > > > >> > what
> > > > >> > > > you
> > > > >> > > > >> say
> > > > >> > > > >> > > makes sense.
> > > > >> > > > >> > >
> > > > >> > > > >> > > I was wondering about compatibility here. When we
> send
> > > > >> requests
> > > > >> > > > >> > > between brokers, we want to ensure that the receiving
> > > broker
> > > > >> > > > >> understands
> > > > >> > > > >> > > the request (specifically the new fields). Typically
> > > this is
> > > > >> > done
> > > > >> > > > via
> > > > >> > > > >> > > IBP/metadata version.
> > > > >> > > > >> > > I'm trying to think if there is a way around it but
> I'm
> > > not
> > > > >> sure
> > > > >> > > > there
> > > > >> > > > >> > is.
> > > > >> > > > >> > >
> > > > >> > > > >> > > As for the improvements -- can you clarify how the
> > > multiple
> > > > >> > > > >> transactional
> > > > >> > > > >> > > IDs would help here? Were you thinking of a case
> where we
> > > > >> > > wait/batch
> > > > >> > > > >> > > multiple produce requests together? My understanding
> for
> > > now
> > > > >> > was 1
> > > > >> > > > >> > > transactional ID and one validation per 1 produce
> > > request.
> > > > >> > > > >> > >
> > > > >> > > > >> > > Finally with respect to the authorizations, I think
> it
> > > makes
> > > > >> > sense
> > > > >> > > > to
> > > > >> > > > >> > skip
> > > > >> > > > >> > > topic authorizations, but I'm a bit confused by the
> > > "leader
> > > > >> ID"
> > > > >> > > > field.
> > > > >> > > > >> > > Wouldn't we just want to flag the request as from a
> > > broker
> > > > >> (does
> > > > >> > > it
> > > > >> > > > >> > matter
> > > > >> > > > >> > > which one?).
> > > > >> > > > >> > >
> > > > >> > > > >> > > I think I want to adopt these suggestions, just had
> a few
> > > > >> > > questions
> > > > >> > > > on
> > > > >> > > > >> > the
> > > > >> > > > >> > > details.
> > > > >> > > > >> > >
> > > > >> > > > >> > > Thanks,
> > > > >> > > > >> > > Justine
> > > > >> > > > >> > >
> > > > >> > > > >> > > On Thu, Jan 5, 2023 at 5:05 PM Jason Gustafson
> > > > >> > > > >> > <ja...@confluent.io.invalid>
> > > > >> > > > >> > > wrote:
> > > > >> > > > >> > >
> > > > >> > > > >> > >> Hi Justine,
> > > > >> > > > >> > >>
> > > > >> > > > >> > >> Thanks for the proposal.
> > > > >> > > > >> > >>
> > > > >> > > > >> > >> I was thinking about the implementation a little
> bit.
> > > In the
> > > > >> > > > current
> > > > >> > > > >> > >> proposal, the behavior depends on whether we have an
> > > old or
> > > > >> new
> > > > >> > > > >> client.
> > > > >> > > > >> > >> For
> > > > >> > > > >> > >> old clients, we send `DescribeTransactions` and
> verify
> > > the
> > > > >> > result
> > > > >> > > > and
> > > > >> > > > >> > for
> > > > >> > > > >> > >> new clients, we send `AddPartitionsToTxn`. We might
> be
> > > able
> > > > >> to
> > > > >> > > > >> simplify
> > > > >> > > > >> > >> the
> > > > >> > > > >> > >> implementation if we can use the same request type.
> For
> > > > >> > example,
> > > > >> > > > >> what if
> > > > >> > > > >> > >> we
> > > > >> > > > >> > >> bump the protocol version for `AddPartitionsToTxn`
> and
> > > add a
> > > > >> > > > >> > >> `validateOnly`
> > > > >> > > > >> > >> flag? For older versions, we can set
> > > `validateOnly=true` so
> > > > >> > that
> > > > >> > > > the
> > > > >> > > > >> > >> request only returns successfully if the partition
> had
> > > > >> already
> > > > >> > > been
> > > > >> > > > >> > added.
> > > > >> > > > >> > >> For new versions, we can set `validateOnly=false`
> and
> > > the
> > > > >> > > partition
> > > > >> > > > >> will
> > > > >> > > > >> > >> be
> > > > >> > > > >> > >> added to the transaction. The other slightly
> annoying
> > > thing
> > > > >> > that
> > > > >> > > > this
> > > > >> > > > >> > >> would
> > > > >> > > > >> > >> get around is the need to collect the transaction
> state
> > > for
> > > > >> all
> > > > >> > > > >> > partitions
> > > > >> > > > >> > >> even when we only care about a subset.
> > > > >> > > > >> > >>
> > > > >> > > > >> > >> Some additional improvements to consider:
> > > > >> > > > >> > >>
> > > > >> > > > >> > >> - We can give `AddPartitionsToTxn` better batch
> support
> > > for
> > > > >> > > > >> inter-broker
> > > > >> > > > >> > >> usage. Currently we only allow one
> `TransactionalId` to
> > > be
> > > > >> > > > specified,
> > > > >> > > > >> > but
> > > > >> > > > >> > >> the broker may get some benefit being able to batch
> > > across
> > > > >> > > multiple
> > > > >> > > > >> > >> transactions.
> > > > >> > > > >> > >> - Another small improvement is skipping topic
> > > authorization
> > > > >> > > checks
> > > > >> > > > >> for
> > > > >> > > > >> > >> `AddPartitionsToTxn` when the request is from a
> broker.
> > > > >> Perhaps
> > > > >> > > we
> > > > >> > > > >> can
> > > > >> > > > >> > add
> > > > >> > > > >> > >> a field for the `LeaderId` or something like that
> and
> > > > >> require
> > > > >> > > > CLUSTER
> > > > >> > > > >> > >> permission when set.
> > > > >> > > > >> > >>
> > > > >> > > > >> > >> Best,
> > > > >> > > > >> > >> Jason
> > > > >> > > > >> > >>
> > > > >> > > > >> > >>
> > > > >> > > > >> > >>
> > > > >> > > > >> > >> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
> > > > >> > <jun@confluent.io.invalid
> > > > >> > > >
> > > > >> > > > >> > wrote:
> > > > >> > > > >> > >>
> > > > >> > > > >> > >> > Hi, Justine,
> > > > >> > > > >> > >> >
> > > > >> > > > >> > >> > Thanks for the explanation. It makes sense to me
> now.
> > > > >> > > > >> > >> >
> > > > >> > > > >> > >> > Jun
> > > > >> > > > >> > >> >
> > > > >> > > > >> > >> > On Mon, Dec 19, 2022 at 1:42 PM Justine Olshan
> > > > >> > > > >> > >> > <jo...@confluent.io.invalid>
> > > > >> > > > >> > >> > wrote:
> > > > >> > > > >> > >> >
> > > > >> > > > >> > >> > > Hi Jun,
> > > > >> > > > >> > >> > >
> > > > >> > > > >> > >> > > My understanding of the mechanism is that when
> we
> > > get to
> > > > >> > the
> > > > >> > > > last
> > > > >> > > > >> > >> epoch,
> > > > >> > > > >> > >> > we
> > > > >> > > > >> > >> > > increment to the fencing/last epoch and if any
> > > further
> > > > >> > > requests
> > > > >> > > > >> come
> > > > >> > > > >> > >> in
> > > > >> > > > >> > >> > for
> > > > >> > > > >> > >> > > this producer ID they are fenced. Then the
> producer
> > > > >> gets a
> > > > >> > > new
> > > > >> > > > ID
> > > > >> > > > >> > and
> > > > >> > > > >> > >> > > restarts with epoch/sequence 0. The fenced epoch
> > > sticks
> > > > >> > > around
> > > > >> > > > >> for
> > > > >> > > > >> > the
> > > > >> > > > >> > >> > > duration of producer.id.expiration.ms and
> blocks
> > > any
> > > > >> late
> > > > >> > > > >> messages
> > > > >> > > > >> > >> > there.
> > > > >> > > > >> > >> > > The new ID will get to take advantage of the
> > > improved
> > > > >> > > semantics
> > > > >> > > > >> > around
> > > > >> > > > >> > >> > > non-zero start sequences. So I think we are
> covered.
> > > > >> > > > >> > >> > >
> > > > >> > > > >> > >> > > The only potential issue is overloading the
> cache,
> > > but
> > > > >> > > > hopefully
> > > > >> > > > >> the
> > > > >> > > > >> > >> > > improvements (lowered producer.id.expiration.ms
> )
> > > will
> > > > >> help
> > > > >> > > > with
> > > > >> > > > >> > that.
> > > > >> > > > >> > >> > Let
> > > > >> > > > >> > >> > > me know if you still have concerns.
> > > > >> > > > >> > >> > >
> > > > >> > > > >> > >> > > Thanks,
> > > > >> > > > >> > >> > > Justine
> > > > >> > > > >> > >> > >
> > > > >> > > > >> > >> > > On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
> > > > >> > > > >> <ju...@confluent.io.invalid>
> > > > >> > > > >> > >> > wrote:
> > > > >> > > > >> > >> > >
> > > > >> > > > >> > >> > > > Hi, Justine,
> > > > >> > > > >> > >> > > >
> > > > >> > > > >> > >> > > > Thanks for the explanation.
> > > > >> > > > >> > >> > > >
> > > > >> > > > >> > >> > > > 70. The proposed fencing logic doesn't apply
> when
> > > pid
> > > > >> > > > changes,
> > > > >> > > > >> is
> > > > >> > > > >> > >> that
> > > > >> > > > >> > >> > > > right? If so, I am not sure how complete we
> are
> > > > >> > addressing
> > > > >> > > > this
> > > > >> > > > >> > >> issue
> > > > >> > > > >> > >> > if
> > > > >> > > > >> > >> > > > the pid changes more frequently.
> > > > >> > > > >> > >> > > >
> > > > >> > > > >> > >> > > > Thanks,
> > > > >> > > > >> > >> > > >
> > > > >> > > > >> > >> > > > Jun
> > > > >> > > > >> > >> > > >
> > > > >> > > > >> > >> > > >
> > > > >> > > > >> > >> > > >
> > > > >> > > > >> > >> > > > On Fri, Dec 16, 2022 at 9:16 AM Justine Olshan
> > > > >> > > > >> > >> > > > <jo...@confluent.io.invalid>
> > > > >> > > > >> > >> > > > wrote:
> > > > >> > > > >> > >> > > >
> > > > >> > > > >> > >> > > > > Hi Jun,
> > > > >> > > > >> > >> > > > >
> > > > >> > > > >> > >> > > > > Thanks for replying!
> > > > >> > > > >> > >> > > > >
> > > > >> > > > >> > >> > > > > 70.We already do the overflow mechanism, so
> my
> > > > >> change
> > > > >> > > would
> > > > >> > > > >> just
> > > > >> > > > >> > >> make
> > > > >> > > > >> > >> > > it
> > > > >> > > > >> > >> > > > > happen more often.
> > > > >> > > > >> > >> > > > > I was also not suggesting a new field in the
> > > log,
> > > > >> but
> > > > >> > in
> > > > >> > > > the
> > > > >> > > > >> > >> > response,
> > > > >> > > > >> > >> > > > > which would be gated by the client version.
> > > Sorry if
> > > > >> > > > >> something
> > > > >> > > > >> > >> there
> > > > >> > > > >> > >> > is
> > > > >> > > > >> > >> > > > > unclear. I think we are starting to diverge.
> > > > >> > > > >> > >> > > > > The goal of this KIP is to not change to the
> > > marker
> > > > >> > > format
> > > > >> > > > at
> > > > >> > > > >> > all.
> > > > >> > > > >> > >> > > > >
> > > > >> > > > >> > >> > > > > 71. Yes, I guess I was going under the
> > > assumption
> > > > >> that
> > > > >> > > the
> > > > >> > > > >> log
> > > > >> > > > >> > >> would
> > > > >> > > > >> > >> > > just
> > > > >> > > > >> > >> > > > > look at its last epoch and treat it as the
> > > current
> > > > >> > > epoch. I
> > > > >> > > > >> > >> suppose
> > > > >> > > > >> > >> > we
> > > > >> > > > >> > >> > > > can
> > > > >> > > > >> > >> > > > > have some special logic that if the last
> epoch
> > > was
> > > > >> on a
> > > > >> > > > >> marker
> > > > >> > > > >> > we
> > > > >> > > > >> > >> > > > actually
> > > > >> > > > >> > >> > > > > expect the next epoch or something like
> that. We
> > > > >> just
> > > > >> > > need
> > > > >> > > > to
> > > > >> > > > >> > >> > > distinguish
> > > > >> > > > >> > >> > > > > based on whether we had a commit/abort
> marker.
> > > > >> > > > >> > >> > > > >
> > > > >> > > > >> > >> > > > > 72.
> > > > >> > > > >> > >> > > > > > if the producer epoch hasn't been bumped
> on
> > > the
> > > > >> > > > >> > >> > > > > broker, it seems that the stucked message
> will
> > > fail
> > > > >> the
> > > > >> > > > >> sequence
> > > > >> > > > >> > >> > > > validation
> > > > >> > > > >> > >> > > > > and will be ignored. If the producer epoch
> has
> > > been
> > > > >> > > bumped,
> > > > >> > > > >> we
> > > > >> > > > >> > >> ignore
> > > > >> > > > >> > >> > > the
> > > > >> > > > >> > >> > > > > sequence check and the stuck message could
> be
> > > > >> appended
> > > > >> > to
> > > > >> > > > the
> > > > >> > > > >> > log.
> > > > >> > > > >> > >> > So,
> > > > >> > > > >> > >> > > is
> > > > >> > > > >> > >> > > > > the latter case that we want to guard?
> > > > >> > > > >> > >> > > > >
> > > > >> > > > >> > >> > > > > I'm not sure I follow that "the message will
> > > fail
> > > > >> the
> > > > >> > > > >> sequence
> > > > >> > > > >> > >> > > > validation".
> > > > >> > > > >> > >> > > > > In some of these cases, we had an abort
> marker
> > > (due
> > > > >> to
> > > > >> > an
> > > > >> > > > >> error)
> > > > >> > > > >> > >> and
> > > > >> > > > >> > >> > > then
> > > > >> > > > >> > >> > > > > the late message comes in with the correct
> > > sequence
> > > > >> > > number.
> > > > >> > > > >> This
> > > > >> > > > >> > >> is a
> > > > >> > > > >> > >> > > > case
> > > > >> > > > >> > >> > > > > covered by the KIP.
> > > > >> > > > >> > >> > > > > The latter case is actually not something
> we've
> > > > >> > > considered
> > > > >> > > > >> > here. I
> > > > >> > > > >> > >> > > think
> > > > >> > > > >> > >> > > > > generally when we bump the epoch, we are
> > > accepting
> > > > >> that
> > > > >> > > the
> > > > >> > > > >> > >> sequence
> > > > >> > > > >> > >> > > does
> > > > >> > > > >> > >> > > > > not need to be checked anymore. My
> > > understanding is
> > > > >> > also
> > > > >> > > > >> that we
> > > > >> > > > >> > >> > don't
> > > > >> > > > >> > >> > > > > typically bump epoch mid transaction (based
> on a
> > > > >> quick
> > > > >> > > look
> > > > >> > > > >> at
> > > > >> > > > >> > the
> > > > >> > > > >> > >> > > code)
> > > > >> > > > >> > >> > > > > but let me know if that is the case.
> > > > >> > > > >> > >> > > > >
> > > > >> > > > >> > >> > > > > Thanks,
> > > > >> > > > >> > >> > > > > Justine
> > > > >> > > > >> > >> > > > >
> > > > >> > > > >> > >> > > > > On Thu, Dec 15, 2022 at 12:23 PM Jun Rao
> > > > >> > > > >> > <jun@confluent.io.invalid
> > > > >> > > > >> > >> >
> > > > >> > > > >> > >> > > > wrote:
> > > > >> > > > >> > >> > > > >
> > > > >> > > > >> > >> > > > > > Hi, Justine,
> > > > >> > > > >> > >> > > > > >
> > > > >> > > > >> > >> > > > > > Thanks for the reply.
> > > > >> > > > >> > >> > > > > >
> > > > >> > > > >> > >> > > > > > 70. Assigning a new pid on int overflow
> seems
> > > a
> > > > >> bit
> > > > >> > > > hacky.
> > > > >> > > > >> If
> > > > >> > > > >> > we
> > > > >> > > > >> > >> > > need a
> > > > >> > > > >> > >> > > > > txn
> > > > >> > > > >> > >> > > > > > level id, it will be better to model this
> > > > >> explicitly.
> > > > >> > > > >> Adding a
> > > > >> > > > >> > >> new
> > > > >> > > > >> > >> > > > field
> > > > >> > > > >> > >> > > > > > would require a bit more work since it
> > > requires a
> > > > >> new
> > > > >> > > txn
> > > > >> > > > >> > marker
> > > > >> > > > >> > >> > > format
> > > > >> > > > >> > >> > > > > in
> > > > >> > > > >> > >> > > > > > the log. So, we probably need to guard it
> > > with an
> > > > >> IBP
> > > > >> > > or
> > > > >> > > > >> > >> metadata
> > > > >> > > > >> > >> > > > version
> > > > >> > > > >> > >> > > > > > and document the impact on downgrade once
> the
> > > new
> > > > >> > > format
> > > > >> > > > is
> > > > >> > > > >> > >> written
> > > > >> > > > >> > >> > > to
> > > > >> > > > >> > >> > > > > the
> > > > >> > > > >> > >> > > > > > log.
> > > > >> > > > >> > >> > > > > >
> > > > >> > > > >> > >> > > > > > 71. Hmm, once the marker is written, the
> > > partition
> > > > >> > will
> > > > >> > > > >> expect
> > > > >> > > > >> > >> the
> > > > >> > > > >> > >> > > next
> > > > >> > > > >> > >> > > > > > append to be on the next epoch. Does that
> > > cover
> > > > >> the
> > > > >> > > case
> > > > >> > > > >> you
> > > > >> > > > >> > >> > > mentioned?
> > > > >> > > > >> > >> > > > > >
> > > > >> > > > >> > >> > > > > > 72. Also, just to be clear on the stucked
> > > message
> > > > >> > issue
> > > > >> > > > >> > >> described
> > > > >> > > > >> > >> > in
> > > > >> > > > >> > >> > > > the
> > > > >> > > > >> > >> > > > > > motivation. With EoS, we also validate the
> > > > >> sequence
> > > > >> > id
> > > > >> > > > for
> > > > >> > > > >> > >> > > idempotency.
> > > > >> > > > >> > >> > > > > So,
> > > > >> > > > >> > >> > > > > > with the current logic, if the producer
> epoch
> > > > >> hasn't
> > > > >> > > been
> > > > >> > > > >> > >> bumped on
> > > > >> > > > >> > >> > > the
> > > > >> > > > >> > >> > > > > > broker, it seems that the stucked message
> will
> > > > >> fail
> > > > >> > the
> > > > >> > > > >> > sequence
> > > > >> > > > >> > >> > > > > validation
> > > > >> > > > >> > >> > > > > > and will be ignored. If the producer
> epoch has
> > > > >> been
> > > > >> > > > >> bumped, we
> > > > >> > > > >> > >> > ignore
> > > > >> > > > >> > >> > > > the
> > > > >> > > > >> > >> > > > > > sequence check and the stuck message
> could be
> > > > >> > appended
> > > > >> > > to
> > > > >> > > > >> the
> > > > >> > > > >> > >> log.
> > > > >> > > > >> > >> > > So,
> > > > >> > > > >> > >> > > > is
> > > > >> > > > >> > >> > > > > > the latter case that we want to guard?
> > > > >> > > > >> > >> > > > > >
> > > > >> > > > >> > >> > > > > > Thanks,
> > > > >> > > > >> > >> > > > > >
> > > > >> > > > >> > >> > > > > > Jun
> > > > >> > > > >> > >> > > > > >
> > > > >> > > > >> > >> > > > > > On Wed, Dec 14, 2022 at 10:44 AM Justine
> > > Olshan
> > > > >> > > > >> > >> > > > > > <jo...@confluent.io.invalid> wrote:
> > > > >> > > > >> > >> > > > > >
> > > > >> > > > >> > >> > > > > > > Matthias — thanks again for taking time
> to
> > > look
> > > > >> a
> > > > >> > > this.
> > > > >> > > > >> You
> > > > >> > > > >> > >> said:
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > > > > My proposal was only focusing to avoid
> > > > >> dangling
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > > > transactions if records are added
> without
> > > > >> > registered
> > > > >> > > > >> > >> partition.
> > > > >> > > > >> > >> > --
> > > > >> > > > >> > >> > > > > Maybe
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > > > you can add a few more details to the
> KIP
> > > about
> > > > >> > this
> > > > >> > > > >> > scenario
> > > > >> > > > >> > >> for
> > > > >> > > > >> > >> > > > > better
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > > > documentation purpose?
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > > > I'm not sure I understand what you mean
> > > here.
> > > > >> The
> > > > >> > > > >> motivation
> > > > >> > > > >> > >> > > section
> > > > >> > > > >> > >> > > > > > > describes two scenarios about how the
> record
> > > > >> can be
> > > > >> > > > added
> > > > >> > > > >> > >> > without a
> > > > >> > > > >> > >> > > > > > > registered partition:
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > > > > 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.
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > > > > Another way hanging transactions can
> > > occur is
> > > > >> > that
> > > > >> > > a
> > > > >> > > > >> > client
> > > > >> > > > >> > >> is
> > > > >> > > > >> > >> > > > buggy
> > > > >> > > > >> > >> > > > > > and
> > > > >> > > > >> > >> > > > > > > may somehow try to write to a partition
> > > before
> > > > >> it
> > > > >> > > adds
> > > > >> > > > >> the
> > > > >> > > > >> > >> > > partition
> > > > >> > > > >> > >> > > > to
> > > > >> > > > >> > >> > > > > > the
> > > > >> > > > >> > >> > > > > > > transaction.
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > > > For the first example of this would it
> be
> > > > >> helpful
> > > > >> > to
> > > > >> > > > say
> > > > >> > > > >> > that
> > > > >> > > > >> > >> > this
> > > > >> > > > >> > >> > > > > > message
> > > > >> > > > >> > >> > > > > > > comes in after the abort, but before the
> > > > >> partition
> > > > >> > is
> > > > >> > > > >> added
> > > > >> > > > >> > to
> > > > >> > > > >> > >> > the
> > > > >> > > > >> > >> > > > next
> > > > >> > > > >> > >> > > > > > > transaction so it becomes "hanging."
> > > Perhaps the
> > > > >> > next
> > > > >> > > > >> > sentence
> > > > >> > > > >> > >> > > > > describing
> > > > >> > > > >> > >> > > > > > > the message becoming part of the next
> > > > >> transaction
> > > > >> > (a
> > > > >> > > > >> > different
> > > > >> > > > >> > >> > > case)
> > > > >> > > > >> > >> > > > > was
> > > > >> > > > >> > >> > > > > > > not properly differentiated.
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > > > Jun — thanks for reading the KIP.
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > > > 70. The int typing was a concern.
> Currently
> > > we
> > > > >> > have a
> > > > >> > > > >> > >> mechanism
> > > > >> > > > >> > >> > in
> > > > >> > > > >> > >> > > > > place
> > > > >> > > > >> > >> > > > > > to
> > > > >> > > > >> > >> > > > > > > fence the final epoch when the epoch is
> > > about to
> > > > >> > > > overflow
> > > > >> > > > >> > and
> > > > >> > > > >> > >> > > assign
> > > > >> > > > >> > >> > > > a
> > > > >> > > > >> > >> > > > > > new
> > > > >> > > > >> > >> > > > > > > producer ID with epoch 0. Of course,
> this
> > > is a
> > > > >> bit
> > > > >> > > > tricky
> > > > >> > > > >> > >> when it
> > > > >> > > > >> > >> > > > comes
> > > > >> > > > >> > >> > > > > > to
> > > > >> > > > >> > >> > > > > > > the response back to the client.
> > > > >> > > > >> > >> > > > > > > Making this a long could be another
> option,
> > > but
> > > > >> I
> > > > >> > > > wonder
> > > > >> > > > >> are
> > > > >> > > > >> > >> > there
> > > > >> > > > >> > >> > > > any
> > > > >> > > > >> > >> > > > > > > implications on changing this field if
> the
> > > > >> epoch is
> > > > >> > > > >> > persisted
> > > > >> > > > >> > >> to
> > > > >> > > > >> > >> > > > disk?
> > > > >> > > > >> > >> > > > > > I'd
> > > > >> > > > >> > >> > > > > > > need to check the usages.
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > > > 71.This was something Matthias asked
> about
> > > as
> > > > >> > well. I
> > > > >> > > > was
> > > > >> > > > >> > >> > > > considering a
> > > > >> > > > >> > >> > > > > > > possible edge case where a produce
> request
> > > from
> > > > >> a
> > > > >> > new
> > > > >> > > > >> > >> transaction
> > > > >> > > > >> > >> > > > > somehow
> > > > >> > > > >> > >> > > > > > > gets sent right after the marker is
> > > written, but
> > > > >> > > before
> > > > >> > > > >> the
> > > > >> > > > >> > >> > > producer
> > > > >> > > > >> > >> > > > is
> > > > >> > > > >> > >> > > > > > > alerted of the newly bumped epoch. In
> this
> > > > >> case, we
> > > > >> > > may
> > > > >> > > > >> > >> include
> > > > >> > > > >> > >> > > this
> > > > >> > > > >> > >> > > > > > record
> > > > >> > > > >> > >> > > > > > > when we don't want to. I suppose we
> could
> > > try
> > > > >> to do
> > > > >> > > > >> > something
> > > > >> > > > >> > >> > > client
> > > > >> > > > >> > >> > > > > side
> > > > >> > > > >> > >> > > > > > > to bump the epoch after sending an
> endTxn as
> > > > >> well
> > > > >> > in
> > > > >> > > > this
> > > > >> > > > >> > >> > scenario
> > > > >> > > > >> > >> > > —
> > > > >> > > > >> > >> > > > > but
> > > > >> > > > >> > >> > > > > > I
> > > > >> > > > >> > >> > > > > > > wonder how it would work when the
> server is
> > > > >> > aborting
> > > > >> > > > >> based
> > > > >> > > > >> > on
> > > > >> > > > >> > >> a
> > > > >> > > > >> > >> > > > > > server-side
> > > > >> > > > >> > >> > > > > > > error. I could also be missing
> something and
> > > > >> this
> > > > >> > > > >> scenario
> > > > >> > > > >> > is
> > > > >> > > > >> > >> > > > actually
> > > > >> > > > >> > >> > > > > > not
> > > > >> > > > >> > >> > > > > > > possible.
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > > > Thanks again to everyone reading and
> > > commenting.
> > > > >> > Let
> > > > >> > > me
> > > > >> > > > >> know
> > > > >> > > > >> > >> > about
> > > > >> > > > >> > >> > > > any
> > > > >> > > > >> > >> > > > > > > further questions or comments.
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > > > Justine
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > > > On Wed, Dec 14, 2022 at 9:41 AM Jun Rao
> > > > >> > > > >> > >> <jun@confluent.io.invalid
> > > > >> > > > >> > >> > >
> > > > >> > > > >> > >> > > > > > wrote:
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > > > > Hi, Justine,
> > > > >> > > > >> > >> > > > > > > >
> > > > >> > > > >> > >> > > > > > > > Thanks for the KIP. A couple of
> comments.
> > > > >> > > > >> > >> > > > > > > >
> > > > >> > > > >> > >> > > > > > > > 70. Currently, the producer epoch is
> an
> > > int.
> > > > >> I am
> > > > >> > > not
> > > > >> > > > >> sure
> > > > >> > > > >> > >> if
> > > > >> > > > >> > >> > > it's
> > > > >> > > > >> > >> > > > > > enough
> > > > >> > > > >> > >> > > > > > > > to accommodate all transactions in the
> > > > >> lifetime
> > > > >> > of
> > > > >> > > a
> > > > >> > > > >> > >> producer.
> > > > >> > > > >> > >> > > > Should
> > > > >> > > > >> > >> > > > > > we
> > > > >> > > > >> > >> > > > > > > > change that to a long or add a new
> long
> > > field
> > > > >> > like
> > > > >> > > > >> txnId?
> > > > >> > > > >> > >> > > > > > > >
> > > > >> > > > >> > >> > > > > > > > 71. "it will write the prepare commit
> > > message
> > > > >> > with
> > > > >> > > a
> > > > >> > > > >> > bumped
> > > > >> > > > >> > >> > epoch
> > > > >> > > > >> > >> > > > and
> > > > >> > > > >> > >> > > > > > > send
> > > > >> > > > >> > >> > > > > > > > WriteTxnMarkerRequests with the bumped
> > > epoch."
> > > > >> > Hmm,
> > > > >> > > > the
> > > > >> > > > >> > >> epoch
> > > > >> > > > >> > >> > is
> > > > >> > > > >> > >> > > > > > > associated
> > > > >> > > > >> > >> > > > > > > > with the current txn right? So, it
> seems
> > > > >> weird to
> > > > >> > > > >> write a
> > > > >> > > > >> > >> > commit
> > > > >> > > > >> > >> > > > > > message
> > > > >> > > > >> > >> > > > > > > > with a bumped epoch. Should we only
> bump
> > > up
> > > > >> the
> > > > >> > > epoch
> > > > >> > > > >> in
> > > > >> > > > >> > >> > > > > EndTxnResponse
> > > > >> > > > >> > >> > > > > > > and
> > > > >> > > > >> > >> > > > > > > > rename the field to sth like
> > > > >> nextProducerEpoch?
> > > > >> > > > >> > >> > > > > > > >
> > > > >> > > > >> > >> > > > > > > > Thanks,
> > > > >> > > > >> > >> > > > > > > >
> > > > >> > > > >> > >> > > > > > > > Jun
> > > > >> > > > >> > >> > > > > > > >
> > > > >> > > > >> > >> > > > > > > >
> > > > >> > > > >> > >> > > > > > > >
> > > > >> > > > >> > >> > > > > > > > On Mon, Dec 12, 2022 at 8:54 PM
> Matthias
> > > J.
> > > > >> Sax <
> > > > >> > > > >> > >> > > mjsax@apache.org>
> > > > >> > > > >> > >> > > > > > > wrote:
> > > > >> > > > >> > >> > > > > > > >
> > > > >> > > > >> > >> > > > > > > > > Thanks for the background.
> > > > >> > > > >> > >> > > > > > > > >
> > > > >> > > > >> > >> > > > > > > > > 20/30: SGTM. My proposal was only
> > > focusing
> > > > >> to
> > > > >> > > avoid
> > > > >> > > > >> > >> dangling
> > > > >> > > > >> > >> > > > > > > > > transactions if records are added
> > > without
> > > > >> > > > registered
> > > > >> > > > >> > >> > partition.
> > > > >> > > > >> > >> > > > --
> > > > >> > > > >> > >> > > > > > > Maybe
> > > > >> > > > >> > >> > > > > > > > > you can add a few more details to
> the
> > > KIP
> > > > >> about
> > > > >> > > > this
> > > > >> > > > >> > >> scenario
> > > > >> > > > >> > >> > > for
> > > > >> > > > >> > >> > > > > > > better
> > > > >> > > > >> > >> > > > > > > > > documentation purpose?
> > > > >> > > > >> > >> > > > > > > > >
> > > > >> > > > >> > >> > > > > > > > > 40: I think you hit a fair point
> about
> > > race
> > > > >> > > > >> conditions
> > > > >> > > > >> > or
> > > > >> > > > >> > >> > > client
> > > > >> > > > >> > >> > > > > bugs
> > > > >> > > > >> > >> > > > > > > > > (incorrectly not bumping the
> epoch). The
> > > > >> > > > >> > >> complexity/confusion
> > > > >> > > > >> > >> > > for
> > > > >> > > > >> > >> > > > > > using
> > > > >> > > > >> > >> > > > > > > > > the bumped epoch I see, is mainly
> for
> > > > >> internal
> > > > >> > > > >> > debugging,
> > > > >> > > > >> > >> ie,
> > > > >> > > > >> > >> > > > > > > inspecting
> > > > >> > > > >> > >> > > > > > > > > log segment dumps -- it seems
> harder to
> > > > >> reason
> > > > >> > > > about
> > > > >> > > > >> the
> > > > >> > > > >> > >> > system
> > > > >> > > > >> > >> > > > for
> > > > >> > > > >> > >> > > > > > us
> > > > >> > > > >> > >> > > > > > > > > humans. But if we get better
> > > guarantees, it
> > > > >> > would
> > > > >> > > > be
> > > > >> > > > >> > >> worth to
> > > > >> > > > >> > >> > > use
> > > > >> > > > >> > >> > > > > the
> > > > >> > > > >> > >> > > > > > > > > bumped epoch.
> > > > >> > > > >> > >> > > > > > > > >
> > > > >> > > > >> > >> > > > > > > > > 60: as I mentioned already, I don't
> > > know the
> > > > >> > > broker
> > > > >> > > > >> > >> internals
> > > > >> > > > >> > >> > > to
> > > > >> > > > >> > >> > > > > > > provide
> > > > >> > > > >> > >> > > > > > > > > more input. So if nobody else chimes
> > > in, we
> > > > >> > > should
> > > > >> > > > >> just
> > > > >> > > > >> > >> move
> > > > >> > > > >> > >> > > > > forward
> > > > >> > > > >> > >> > > > > > > > > with your proposal.
> > > > >> > > > >> > >> > > > > > > > >
> > > > >> > > > >> > >> > > > > > > > >
> > > > >> > > > >> > >> > > > > > > > > -Matthias
> > > > >> > > > >> > >> > > > > > > > >
> > > > >> > > > >> > >> > > > > > > > >
> > > > >> > > > >> > >> > > > > > > > > On 12/6/22 4:22 PM, Justine Olshan
> > > wrote:
> > > > >> > > > >> > >> > > > > > > > > > Hi all,
> > > > >> > > > >> > >> > > > > > > > > > After Artem's questions about
> error
> > > > >> behavior,
> > > > >> > > > I've
> > > > >> > > > >> > >> > > re-evaluated
> > > > >> > > > >> > >> > > > > the
> > > > >> > > > >> > >> > > > > > > > > > unknown producer ID exception and
> had
> > > some
> > > > >> > > > >> discussions
> > > > >> > > > >> > >> > > offline.
> > > > >> > > > >> > >> > > > > > > > > >
> > > > >> > > > >> > >> > > > > > > > > > I think generally it makes sense
> to
> > > > >> simplify
> > > > >> > > > error
> > > > >> > > > >> > >> handling
> > > > >> > > > >> > >> > > in
> > > > >> > > > >> > >> > > > > > cases
> > > > >> > > > >> > >> > > > > > > > like
> > > > >> > > > >> > >> > > > > > > > > > this and the UNKNOWN_PRODUCER_ID
> error
> > > > >> has a
> > > > >> > > > pretty
> > > > >> > > > >> > long
> > > > >> > > > >> > >> > and
> > > > >> > > > >> > >> > > > > > > > complicated
> > > > >> > > > >> > >> > > > > > > > > > history. Because of this, I
> propose
> > > > >> adding a
> > > > >> > > new
> > > > >> > > > >> error
> > > > >> > > > >> > >> code
> > > > >> > > > >> > >> > > > > > > > > ABORTABLE_ERROR
> > > > >> > > > >> > >> > > > > > > > > > that when encountered by new
> clients
> > > > >> (gated
> > > > >> > by
> > > > >> > > > the
> > > > >> > > > >> > >> produce
> > > > >> > > > >> > >> > > > > request
> > > > >> > > > >> > >> > > > > > > > > version)
> > > > >> > > > >> > >> > > > > > > > > > will simply abort the transaction.
> > > This
> > > > >> > allows
> > > > >> > > > the
> > > > >> > > > >> > >> server
> > > > >> > > > >> > >> > to
> > > > >> > > > >> > >> > > > have
> > > > >> > > > >> > >> > > > > > > some
> > > > >> > > > >> > >> > > > > > > > > say
> > > > >> > > > >> > >> > > > > > > > > > in whether the client aborts and
> makes
> > > > >> > handling
> > > > >> > > > >> much
> > > > >> > > > >> > >> > simpler.
> > > > >> > > > >> > >> > > > In
> > > > >> > > > >> > >> > > > > > the
> > > > >> > > > >> > >> > > > > > > > > > future, we can also use this
> error in
> > > > >> other
> > > > >> > > > >> situations
> > > > >> > > > >> > >> > where
> > > > >> > > > >> > >> > > we
> > > > >> > > > >> > >> > > > > > want
> > > > >> > > > >> > >> > > > > > > to
> > > > >> > > > >> > >> > > > > > > > > > abort the transactions. We can
> even
> > > use on
> > > > >> > > other
> > > > >> > > > >> apis.
> > > > >> > > > >> > >> > > > > > > > > >
> > > > >> > > > >> > >> > > > > > > > > > I've added this to the KIP. Let me
> > > know if
> > > > >> > > there
> > > > >> > > > >> are
> > > > >> > > > >> > any
> > > > >> > > > >> > >> > > > > questions
> > > > >> > > > >> > >> > > > > > or
> > > > >> > > > >> > >> > > > > > > > > > issues.
> > > > >> > > > >> > >> > > > > > > > > >
> > > > >> > > > >> > >> > > > > > > > > > Justine
> > > > >> > > > >> > >> > > > > > > > > >
> > > > >> > > > >> > >> > > > > > > > > > On Fri, Dec 2, 2022 at 10:22 AM
> > > Justine
> > > > >> > Olshan
> > > > >> > > <
> > > > >> > > > >> > >> > > > > > jolshan@confluent.io
> > > > >> > > > >> > >> > > > > > > >
> > > > >> > > > >> > >> > > > > > > > > wrote:
> > > > >> > > > >> > >> > > > > > > > > >
> > > > >> > > > >> > >> > > > > > > > > >> Hey Matthias,
> > > > >> > > > >> > >> > > > > > > > > >>
> > > > >> > > > >> > >> > > > > > > > > >>
> > > > >> > > > >> > >> > > > > > > > > >> 20/30 — Maybe I also didn't
> express
> > > > >> myself
> > > > >> > > > >> clearly.
> > > > >> > > > >> > For
> > > > >> > > > >> > >> > > older
> > > > >> > > > >> > >> > > > > > > clients
> > > > >> > > > >> > >> > > > > > > > we
> > > > >> > > > >> > >> > > > > > > > > >> don't have a way to distinguish
> > > between a
> > > > >> > > > previous
> > > > >> > > > >> > and
> > > > >> > > > >> > >> the
> > > > >> > > > >> > >> > > > > current
> > > > >> > > > >> > >> > > > > > > > > >> transaction since we don't have
> the
> > > epoch
> > > > >> > > bump.
> > > > >> > > > >> This
> > > > >> > > > >> > >> means
> > > > >> > > > >> > >> > > > that
> > > > >> > > > >> > >> > > > > a
> > > > >> > > > >> > >> > > > > > > late
> > > > >> > > > >> > >> > > > > > > > > >> message from the previous
> transaction
> > > > >> may be
> > > > >> > > > >> added to
> > > > >> > > > >> > >> the
> > > > >> > > > >> > >> > > new
> > > > >> > > > >> > >> > > > > one.
> > > > >> > > > >> > >> > > > > > > > With
> > > > >> > > > >> > >> > > > > > > > > >> older clients — we can't
> guarantee
> > > this
> > > > >> > won't
> > > > >> > > > >> happen
> > > > >> > > > >> > >> if we
> > > > >> > > > >> > >> > > > > already
> > > > >> > > > >> > >> > > > > > > > sent
> > > > >> > > > >> > >> > > > > > > > > the
> > > > >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call (why we
> make
> > > > >> changes
> > > > >> > > for
> > > > >> > > > >> the
> > > > >> > > > >> > >> newer
> > > > >> > > > >> > >> > > > > client)
> > > > >> > > > >> > >> > > > > > > but
> > > > >> > > > >> > >> > > > > > > > > we
> > > > >> > > > >> > >> > > > > > > > > >> can at least gate some by
> ensuring
> > > that
> > > > >> the
> > > > >> > > > >> partition
> > > > >> > > > >> > >> has
> > > > >> > > > >> > >> > > been
> > > > >> > > > >> > >> > > > > > added
> > > > >> > > > >> > >> > > > > > > > to
> > > > >> > > > >> > >> > > > > > > > > the
> > > > >> > > > >> > >> > > > > > > > > >> transaction. The rationale here
> is
> > > that
> > > > >> > there
> > > > >> > > > are
> > > > >> > > > >> > >> likely
> > > > >> > > > >> > >> > > LESS
> > > > >> > > > >> > >> > > > > late
> > > > >> > > > >> > >> > > > > > > > > arrivals
> > > > >> > > > >> > >> > > > > > > > > >> as time goes on, so hopefully
> most
> > > late
> > > > >> > > arrivals
> > > > >> > > > >> will
> > > > >> > > > >> > >> come
> > > > >> > > > >> > >> > > in
> > > > >> > > > >> > >> > > > > > BEFORE
> > > > >> > > > >> > >> > > > > > > > the
> > > > >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call. Those
> that
> > > > >> arrive
> > > > >> > > > before
> > > > >> > > > >> > will
> > > > >> > > > >> > >> be
> > > > >> > > > >> > >> > > > > properly
> > > > >> > > > >> > >> > > > > > > > gated
> > > > >> > > > >> > >> > > > > > > > > >> with the describeTransactions
> > > approach.
> > > > >> > > > >> > >> > > > > > > > > >>
> > > > >> > > > >> > >> > > > > > > > > >> If we take the approach you
> > > suggested,
> > > > >> ANY
> > > > >> > > late
> > > > >> > > > >> > arrival
> > > > >> > > > >> > >> > > from a
> > > > >> > > > >> > >> > > > > > > > previous
> > > > >> > > > >> > >> > > > > > > > > >> transaction will be added. And we
> > > don't
> > > > >> want
> > > > >> > > > >> that. I
> > > > >> > > > >> > >> also
> > > > >> > > > >> > >> > > > don't
> > > > >> > > > >> > >> > > > > > see
> > > > >> > > > >> > >> > > > > > > > any
> > > > >> > > > >> > >> > > > > > > > > >> benefit in sending
> addPartitionsToTxn
> > > > >> over
> > > > >> > the
> > > > >> > > > >> > >> > describeTxns
> > > > >> > > > >> > >> > > > > call.
> > > > >> > > > >> > >> > > > > > > They
> > > > >> > > > >> > >> > > > > > > > > will
> > > > >> > > > >> > >> > > > > > > > > >> both be one extra RPC to the Txn
> > > > >> > coordinator.
> > > > >> > > > >> > >> > > > > > > > > >>
> > > > >> > > > >> > >> > > > > > > > > >>
> > > > >> > > > >> > >> > > > > > > > > >> To be clear — newer clients will
> use
> > > > >> > > > >> > addPartitionsToTxn
> > > > >> > > > >> > >> > > > instead
> > > > >> > > > >> > >> > > > > of
> > > > >> > > > >> > >> > > > > > > the
> > > > >> > > > >> > >> > > > > > > > > >> DescribeTxns.
> > > > >> > > > >> > >> > > > > > > > > >>
> > > > >> > > > >> > >> > > > > > > > > >>
> > > > >> > > > >> > >> > > > > > > > > >> 40)
> > > > >> > > > >> > >> > > > > > > > > >> My concern is that if we have
> some
> > > delay
> > > > >> in
> > > > >> > > the
> > > > >> > > > >> > client
> > > > >> > > > >> > >> to
> > > > >> > > > >> > >> > > bump
> > > > >> > > > >> > >> > > > > the
> > > > >> > > > >> > >> > > > > > > > > epoch,
> > > > >> > > > >> > >> > > > > > > > > >> it could continue to send epoch
> 73
> > > and
> > > > >> those
> > > > >> > > > >> records
> > > > >> > > > >> > >> would
> > > > >> > > > >> > >> > > not
> > > > >> > > > >> > >> > > > > be
> > > > >> > > > >> > >> > > > > > > > > fenced.
> > > > >> > > > >> > >> > > > > > > > > >> Perhaps this is not an issue if
> we
> > > don't
> > > > >> > allow
> > > > >> > > > the
> > > > >> > > > >> > next
> > > > >> > > > >> > >> > > > produce
> > > > >> > > > >> > >> > > > > to
> > > > >> > > > >> > >> > > > > > > go
> > > > >> > > > >> > >> > > > > > > > > >> through before the EndTxn request
> > > > >> returns.
> > > > >> > I'm
> > > > >> > > > >> also
> > > > >> > > > >> > >> > thinking
> > > > >> > > > >> > >> > > > > about
> > > > >> > > > >> > >> > > > > > > > > cases of
> > > > >> > > > >> > >> > > > > > > > > >> failure. I will need to think on
> > > this a
> > > > >> bit.
> > > > >> > > > >> > >> > > > > > > > > >>
> > > > >> > > > >> > >> > > > > > > > > >> I wasn't sure if it was that
> > > confusing.
> > > > >> But
> > > > >> > if
> > > > >> > > > we
> > > > >> > > > >> > >> think it
> > > > >> > > > >> > >> > > is,
> > > > >> > > > >> > >> > > > > we
> > > > >> > > > >> > >> > > > > > > can
> > > > >> > > > >> > >> > > > > > > > > >> investigate other ways.
> > > > >> > > > >> > >> > > > > > > > > >>
> > > > >> > > > >> > >> > > > > > > > > >>
> > > > >> > > > >> > >> > > > > > > > > >> 60)
> > > > >> > > > >> > >> > > > > > > > > >>
> > > > >> > > > >> > >> > > > > > > > > >> I'm not sure these are the same
> > > > >> purgatories
> > > > >> > > > since
> > > > >> > > > >> one
> > > > >> > > > >> > >> is a
> > > > >> > > > >> > >> > > > > produce
> > > > >> > > > >> > >> > > > > > > > > >> purgatory (I was planning on
> using a
> > > > >> > callback
> > > > >> > > > >> rather
> > > > >> > > > >> > >> than
> > > > >> > > > >> > >> > > > > > purgatory)
> > > > >> > > > >> > >> > > > > > > > and
> > > > >> > > > >> > >> > > > > > > > > >> the other is simply a request to
> > > append
> > > > >> to
> > > > >> > the
> > > > >> > > > >> log.
> > > > >> > > > >> > Not
> > > > >> > > > >> > >> > sure
> > > > >> > > > >> > >> > > > we
> > > > >> > > > >> > >> > > > > > have
> > > > >> > > > >> > >> > > > > > > > any
> > > > >> > > > >> > >> > > > > > > > > >> structure here for ordering, but
> my
> > > > >> > > > understanding
> > > > >> > > > >> is
> > > > >> > > > >> > >> that
> > > > >> > > > >> > >> > > the
> > > > >> > > > >> > >> > > > > > broker
> > > > >> > > > >> > >> > > > > > > > > could
> > > > >> > > > >> > >> > > > > > > > > >> handle the write request before
> it
> > > hears
> > > > >> > back
> > > > >> > > > from
> > > > >> > > > >> > the
> > > > >> > > > >> > >> Txn
> > > > >> > > > >> > >> > > > > > > > Coordinator.
> > > > >> > > > >> > >> > > > > > > > > >>
> > > > >> > > > >> > >> > > > > > > > > >> Let me know if I misunderstood
> > > something
> > > > >> or
> > > > >> > > > >> something
> > > > >> > > > >> > >> was
> > > > >> > > > >> > >> > > > > unclear.
> > > > >> > > > >> > >> > > > > > > > > >>
> > > > >> > > > >> > >> > > > > > > > > >> Justine
> > > > >> > > > >> > >> > > > > > > > > >>
> > > > >> > > > >> > >> > > > > > > > > >> On Thu, Dec 1, 2022 at 12:15 PM
> > > Matthias
> > > > >> J.
> > > > >> > > Sax
> > > > >> > > > <
> > > > >> > > > >> > >> > > > > mjsax@apache.org
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > > > > > wrote:
> > > > >> > > > >> > >> > > > > > > > > >>
> > > > >> > > > >> > >> > > > > > > > > >>> Thanks for the details Justine!
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > > >>>> 20)
> > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > >> > > > >> > >> > > > > > > > > >>>> The client side change for 2 is
> > > > >> removing
> > > > >> > the
> > > > >> > > > >> > >> > addPartitions
> > > > >> > > > >> > >> > > > to
> > > > >> > > > >> > >> > > > > > > > > >>> transaction
> > > > >> > > > >> > >> > > > > > > > > >>>> call. We don't need to make
> this
> > > from
> > > > >> the
> > > > >> > > > >> producer
> > > > >> > > > >> > to
> > > > >> > > > >> > >> > the
> > > > >> > > > >> > >> > > > txn
> > > > >> > > > >> > >> > > > > > > > > >>> coordinator,
> > > > >> > > > >> > >> > > > > > > > > >>>> only server side.
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > > >>> I think I did not express myself
> > > > >> clearly. I
> > > > >> > > > >> > understand
> > > > >> > > > >> > >> > that
> > > > >> > > > >> > >> > > > we
> > > > >> > > > >> > >> > > > > > can
> > > > >> > > > >> > >> > > > > > > > (and
> > > > >> > > > >> > >> > > > > > > > > >>> should) change the producer to
> not
> > > send
> > > > >> the
> > > > >> > > > >> > >> > `addPartitions`
> > > > >> > > > >> > >> > > > > > request
> > > > >> > > > >> > >> > > > > > > > any
> > > > >> > > > >> > >> > > > > > > > > >>> longer. But I don't thinks it's
> > > > >> requirement
> > > > >> > > to
> > > > >> > > > >> > change
> > > > >> > > > >> > >> the
> > > > >> > > > >> > >> > > > > broker?
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > > >>> What I am trying to say is: as a
> > > > >> safe-guard
> > > > >> > > and
> > > > >> > > > >> > >> > improvement
> > > > >> > > > >> > >> > > > for
> > > > >> > > > >> > >> > > > > > > older
> > > > >> > > > >> > >> > > > > > > > > >>> producers, the partition leader
> can
> > > just
> > > > >> > send
> > > > >> > > > the
> > > > >> > > > >> > >> > > > > `addPartitions`
> > > > >> > > > >> > >> > > > > > > > > >>> request to the TX-coordinator
> in any
> > > > >> case
> > > > >> > --
> > > > >> > > if
> > > > >> > > > >> the
> > > > >> > > > >> > >> old
> > > > >> > > > >> > >> > > > > producer
> > > > >> > > > >> > >> > > > > > > > > >>> correctly did send the
> > > `addPartition`
> > > > >> > request
> > > > >> > > > to
> > > > >> > > > >> the
> > > > >> > > > >> > >> > > > > > TX-coordinator
> > > > >> > > > >> > >> > > > > > > > > >>> already, the TX-coordinator can
> just
> > > > >> > "ignore"
> > > > >> > > > is
> > > > >> > > > >> as
> > > > >> > > > >> > >> > > > idempotent.
> > > > >> > > > >> > >> > > > > > > > > However,
> > > > >> > > > >> > >> > > > > > > > > >>> if the old producer has a bug
> and
> > > did
> > > > >> > forget
> > > > >> > > to
> > > > >> > > > >> sent
> > > > >> > > > >> > >> the
> > > > >> > > > >> > >> > > > > > > > `addPartition`
> > > > >> > > > >> > >> > > > > > > > > >>> request, we would now ensure
> that
> > > the
> > > > >> > > partition
> > > > >> > > > >> is
> > > > >> > > > >> > >> indeed
> > > > >> > > > >> > >> > > > added
> > > > >> > > > >> > >> > > > > > to
> > > > >> > > > >> > >> > > > > > > > the
> > > > >> > > > >> > >> > > > > > > > > >>> TX and thus fix a potential
> > > producer bug
> > > > >> > > (even
> > > > >> > > > >> if we
> > > > >> > > > >> > >> > don't
> > > > >> > > > >> > >> > > > get
> > > > >> > > > >> > >> > > > > > the
> > > > >> > > > >> > >> > > > > > > > > >>> fencing via the bump epoch). --
> It
> > > > >> seems to
> > > > >> > > be
> > > > >> > > > a
> > > > >> > > > >> > good
> > > > >> > > > >> > >> > > > > > improvement?
> > > > >> > > > >> > >> > > > > > > Or
> > > > >> > > > >> > >> > > > > > > > > is
> > > > >> > > > >> > >> > > > > > > > > >>> there a reason to not do this?
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > > >>>> 30)
> > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > >> > > > >> > >> > > > > > > > > >>>> Transaction is ongoing =
> partition
> > > was
> > > > >> > added
> > > > >> > > > to
> > > > >> > > > >> > >> > > transaction
> > > > >> > > > >> > >> > > > > via
> > > > >> > > > >> > >> > > > > > > > > >>>> addPartitionsToTxn. We check
> this
> > > with
> > > > >> the
> > > > >> > > > >> > >> > > > > DescribeTransactions
> > > > >> > > > >> > >> > > > > > > > call.
> > > > >> > > > >> > >> > > > > > > > > >>> Let
> > > > >> > > > >> > >> > > > > > > > > >>>> me know if this wasn't
> sufficiently
> > > > >> > > explained
> > > > >> > > > >> here:
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > > >>> If we do what I propose in
> (20), we
> > > > >> don't
> > > > >> > > > really
> > > > >> > > > >> > need
> > > > >> > > > >> > >> to
> > > > >> > > > >> > >> > > make
> > > > >> > > > >> > >> > > > > > this
> > > > >> > > > >> > >> > > > > > > > > >>> `DescribeTransaction` call, as
> the
> > > > >> > partition
> > > > >> > > > >> leader
> > > > >> > > > >> > >> adds
> > > > >> > > > >> > >> > > the
> > > > >> > > > >> > >> > > > > > > > partition
> > > > >> > > > >> > >> > > > > > > > > >>> for older clients and we get
> this
> > > check
> > > > >> for
> > > > >> > > > free.
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > > >>>> 40)
> > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > >> > > > >> > >> > > > > > > > > >>>> The idea here is that if any
> > > messages
> > > > >> > > somehow
> > > > >> > > > >> come
> > > > >> > > > >> > in
> > > > >> > > > >> > >> > > before
> > > > >> > > > >> > >> > > > > we
> > > > >> > > > >> > >> > > > > > > get
> > > > >> > > > >> > >> > > > > > > > > the
> > > > >> > > > >> > >> > > > > > > > > >>> new
> > > > >> > > > >> > >> > > > > > > > > >>>> epoch to the producer, they
> will be
> > > > >> > fenced.
> > > > >> > > > >> > However,
> > > > >> > > > >> > >> if
> > > > >> > > > >> > >> > we
> > > > >> > > > >> > >> > > > > don't
> > > > >> > > > >> > >> > > > > > > > think
> > > > >> > > > >> > >> > > > > > > > > >>> this
> > > > >> > > > >> > >> > > > > > > > > >>>> is necessary, it can be
> discussed
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > > >>> I agree that we should have
> epoch
> > > > >> fencing.
> > > > >> > My
> > > > >> > > > >> > >> question is
> > > > >> > > > >> > >> > > > > > > different:
> > > > >> > > > >> > >> > > > > > > > > >>> Assume we are at epoch 73, and
> we
> > > have
> > > > >> an
> > > > >> > > > ongoing
> > > > >> > > > >> > >> > > > transaction,
> > > > >> > > > >> > >> > > > > > that
> > > > >> > > > >> > >> > > > > > > > is
> > > > >> > > > >> > >> > > > > > > > > >>> committed. It seems natural to
> > > write the
> > > > >> > > > "prepare
> > > > >> > > > >> > >> commit"
> > > > >> > > > >> > >> > > > > marker
> > > > >> > > > >> > >> > > > > > > and
> > > > >> > > > >> > >> > > > > > > > > the
> > > > >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` both with
> > > epoch
> > > > >> 73,
> > > > >> > > too,
> > > > >> > > > >> as
> > > > >> > > > >> > it
> > > > >> > > > >> > >> > > belongs
> > > > >> > > > >> > >> > > > > to
> > > > >> > > > >> > >> > > > > > > the
> > > > >> > > > >> > >> > > > > > > > > >>> current transaction. Of course,
> we
> > > now
> > > > >> also
> > > > >> > > > bump
> > > > >> > > > >> the
> > > > >> > > > >> > >> > epoch
> > > > >> > > > >> > >> > > > and
> > > > >> > > > >> > >> > > > > > > expect
> > > > >> > > > >> > >> > > > > > > > > >>> the next requests to have epoch
> 74,
> > > and
> > > > >> > would
> > > > >> > > > >> reject
> > > > >> > > > >> > >> an
> > > > >> > > > >> > >> > > > request
> > > > >> > > > >> > >> > > > > > > with
> > > > >> > > > >> > >> > > > > > > > > >>> epoch 73, as the corresponding
> TX
> > > for
> > > > >> epoch
> > > > >> > > 73
> > > > >> > > > >> was
> > > > >> > > > >> > >> > already
> > > > >> > > > >> > >> > > > > > > committed.
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > > >>> It seems you propose to write
> the
> > > > >> "prepare
> > > > >> > > > commit
> > > > >> > > > >> > >> marker"
> > > > >> > > > >> > >> > > and
> > > > >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` with
> epoch 74
> > > > >> > though,
> > > > >> > > > what
> > > > >> > > > >> > >> would
> > > > >> > > > >> > >> > > work,
> > > > >> > > > >> > >> > > > > but
> > > > >> > > > >> > >> > > > > > > it
> > > > >> > > > >> > >> > > > > > > > > >>> seems confusing. Is there a
> reason
> > > why
> > > > >> we
> > > > >> > > would
> > > > >> > > > >> use
> > > > >> > > > >> > >> the
> > > > >> > > > >> > >> > > > bumped
> > > > >> > > > >> > >> > > > > > > epoch
> > > > >> > > > >> > >> > > > > > > > 74
> > > > >> > > > >> > >> > > > > > > > > >>> instead of the current epoch 73?
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > > >>>> 60)
> > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > >> > > > >> > >> > > > > > > > > >>>> When we are checking if the
> > > > >> transaction is
> > > > >> > > > >> ongoing,
> > > > >> > > > >> > >> we
> > > > >> > > > >> > >> > > need
> > > > >> > > > >> > >> > > > to
> > > > >> > > > >> > >> > > > > > > make
> > > > >> > > > >> > >> > > > > > > > a
> > > > >> > > > >> > >> > > > > > > > > >>> round
> > > > >> > > > >> > >> > > > > > > > > >>>> trip from the leader partition
> to
> > > the
> > > > >> > > > >> transaction
> > > > >> > > > >> > >> > > > coordinator.
> > > > >> > > > >> > >> > > > > > In
> > > > >> > > > >> > >> > > > > > > > the
> > > > >> > > > >> > >> > > > > > > > > >>> time
> > > > >> > > > >> > >> > > > > > > > > >>>> we are waiting for this
> message to
> > > come
> > > > >> > > back,
> > > > >> > > > in
> > > > >> > > > >> > >> theory
> > > > >> > > > >> > >> > we
> > > > >> > > > >> > >> > > > > could
> > > > >> > > > >> > >> > > > > > > > have
> > > > >> > > > >> > >> > > > > > > > > >>> sent
> > > > >> > > > >> > >> > > > > > > > > >>>> a commit/abort call that would
> > > make the
> > > > >> > > > original
> > > > >> > > > >> > >> result
> > > > >> > > > >> > >> > of
> > > > >> > > > >> > >> > > > the
> > > > >> > > > >> > >> > > > > > > check
> > > > >> > > > >> > >> > > > > > > > > >>> out of
> > > > >> > > > >> > >> > > > > > > > > >>>> date. That is why we can check
> the
> > > > >> leader
> > > > >> > > > state
> > > > >> > > > >> > >> before
> > > > >> > > > >> > >> > we
> > > > >> > > > >> > >> > > > > write
> > > > >> > > > >> > >> > > > > > to
> > > > >> > > > >> > >> > > > > > > > the
> > > > >> > > > >> > >> > > > > > > > > >>> log.
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > > >>> Thanks. Got it.
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > > >>> However, is this really an
> issue?
> > > We put
> > > > >> > the
> > > > >> > > > >> produce
> > > > >> > > > >> > >> > > request
> > > > >> > > > >> > >> > > > in
> > > > >> > > > >> > >> > > > > > > > > >>> purgatory, so how could we
> process
> > > the
> > > > >> > > > >> > >> > > > `WriteTxnMarkerRequest`
> > > > >> > > > >> > >> > > > > > > first?
> > > > >> > > > >> > >> > > > > > > > > >>> Don't we need to put the
> > > > >> > > > `WriteTxnMarkerRequest`
> > > > >> > > > >> > into
> > > > >> > > > >> > >> > > > > purgatory,
> > > > >> > > > >> > >> > > > > > > too,
> > > > >> > > > >> > >> > > > > > > > > >>> for this case, and process both
> > > request
> > > > >> > > > in-order?
> > > > >> > > > >> > >> (Again,
> > > > >> > > > >> > >> > > my
> > > > >> > > > >> > >> > > > > > broker
> > > > >> > > > >> > >> > > > > > > > > >>> knowledge is limited and maybe
> we
> > > don't
> > > > >> > > > maintain
> > > > >> > > > >> > >> request
> > > > >> > > > >> > >> > > > order
> > > > >> > > > >> > >> > > > > > for
> > > > >> > > > >> > >> > > > > > > > this
> > > > >> > > > >> > >> > > > > > > > > >>> case, what seems to be an issue
> > > IMHO,
> > > > >> and I
> > > > >> > > am
> > > > >> > > > >> > >> wondering
> > > > >> > > > >> > >> > if
> > > > >> > > > >> > >> > > > > > > changing
> > > > >> > > > >> > >> > > > > > > > > >>> request handling to preserve
> order
> > > for
> > > > >> this
> > > > >> > > > case
> > > > >> > > > >> > >> might be
> > > > >> > > > >> > >> > > the
> > > > >> > > > >> > >> > > > > > > cleaner
> > > > >> > > > >> > >> > > > > > > > > >>> solution?)
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > > >>> -Matthias
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > > >>> On 11/30/22 3:28 PM, Artem
> Livshits
> > > > >> wrote:
> > > > >> > > > >> > >> > > > > > > > > >>>> Hi Justine,
> > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > >> > > > >> > >> > > > > > > > > >>>> I think the interesting part is
> > > not in
> > > > >> > this
> > > > >> > > > >> logic
> > > > >> > > > >> > >> > (because
> > > > >> > > > >> > >> > > > it
> > > > >> > > > >> > >> > > > > > > tries
> > > > >> > > > >> > >> > > > > > > > to
> > > > >> > > > >> > >> > > > > > > > > >>>> figure out when
> > > UNKNOWN_PRODUCER_ID is
> > > > >> > > > retriable
> > > > >> > > > >> > and
> > > > >> > > > >> > >> if
> > > > >> > > > >> > >> > > it's
> > > > >> > > > >> > >> > > > > > > > > retryable,
> > > > >> > > > >> > >> > > > > > > > > >>>> it's definitely not fatal), but
> > > what
> > > > >> > happens
> > > > >> > > > >> when
> > > > >> > > > >> > >> this
> > > > >> > > > >> > >> > > logic
> > > > >> > > > >> > >> > > > > > > doesn't
> > > > >> > > > >> > >> > > > > > > > > >>> return
> > > > >> > > > >> > >> > > > > > > > > >>>> 'true' and falls through.  In
> the
> > > old
> > > > >> > > clients
> > > > >> > > > it
> > > > >> > > > >> > >> seems
> > > > >> > > > >> > >> > to
> > > > >> > > > >> > >> > > be
> > > > >> > > > >> > >> > > > > > > fatal,
> > > > >> > > > >> > >> > > > > > > > if
> > > > >> > > > >> > >> > > > > > > > > >>> we
> > > > >> > > > >> > >> > > > > > > > > >>>> keep the behavior in the new
> > > clients,
> > > > >> I'd
> > > > >> > > > >> expect it
> > > > >> > > > >> > >> > would
> > > > >> > > > >> > >> > > be
> > > > >> > > > >> > >> > > > > > fatal
> > > > >> > > > >> > >> > > > > > > > as
> > > > >> > > > >> > >> > > > > > > > > >>> well.
> > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > >> > > > >> > >> > > > > > > > > >>>> -Artem
> > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > >> > > > >> > >> > > > > > > > > >>>> On Tue, Nov 29, 2022 at 11:57
> AM
> > > > >> Justine
> > > > >> > > > Olshan
> > > > >> > > > >> > >> > > > > > > > > >>>> <jo...@confluent.io.invalid>
> > > wrote:
> > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>> Hi Artem and Jeff,
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>> Thanks for taking a look and
> > > sorry for
> > > > >> > the
> > > > >> > > > slow
> > > > >> > > > >> > >> > response.
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>> You both mentioned the change
> to
> > > > >> handle
> > > > >> > > > >> > >> > > UNKNOWN_PRODUCER_ID
> > > > >> > > > >> > >> > > > > > > errors.
> > > > >> > > > >> > >> > > > > > > > > To
> > > > >> > > > >> > >> > > > > > > > > >>> be
> > > > >> > > > >> > >> > > > > > > > > >>>>> clear — this error code will
> only
> > > be
> > > > >> sent
> > > > >> > > > again
> > > > >> > > > >> > when
> > > > >> > > > >> > >> > the
> > > > >> > > > >> > >> > > > > > client's
> > > > >> > > > >> > >> > > > > > > > > >>> request
> > > > >> > > > >> > >> > > > > > > > > >>>>> version is high enough to
> ensure
> > > we
> > > > >> > handle
> > > > >> > > it
> > > > >> > > > >> > >> > correctly.
> > > > >> > > > >> > >> > > > > > > > > >>>>> The current (Java) client
> handles
> > > > >> this by
> > > > >> > > the
> > > > >> > > > >> > >> following
> > > > >> > > > >> > >> > > > > > (somewhat
> > > > >> > > > >> > >> > > > > > > > > long)
> > > > >> > > > >> > >> > > > > > > > > >>>>> code snippet:
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>> // An UNKNOWN_PRODUCER_ID
> means
> > > that
> > > > >> we
> > > > >> > > have
> > > > >> > > > >> lost
> > > > >> > > > >> > >> the
> > > > >> > > > >> > >> > > > > producer
> > > > >> > > > >> > >> > > > > > > > state
> > > > >> > > > >> > >> > > > > > > > > >>> on the
> > > > >> > > > >> > >> > > > > > > > > >>>>> broker. Depending on the log
> start
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>> // offset, we may want to
> retry
> > > > >> these, as
> > > > >> > > > >> > described
> > > > >> > > > >> > >> for
> > > > >> > > > >> > >> > > > each
> > > > >> > > > >> > >> > > > > > case
> > > > >> > > > >> > >> > > > > > > > > >>> below. If
> > > > >> > > > >> > >> > > > > > > > > >>>>> none of those apply, then for
> the
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>> // idempotent producer, we
> will
> > > > >> locally
> > > > >> > > bump
> > > > >> > > > >> the
> > > > >> > > > >> > >> epoch
> > > > >> > > > >> > >> > > and
> > > > >> > > > >> > >> > > > > > reset
> > > > >> > > > >> > >> > > > > > > > the
> > > > >> > > > >> > >> > > > > > > > > >>>>> sequence numbers of in-flight
> > > batches
> > > > >> > from
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>> // sequence 0, then retry the
> > > failed
> > > > >> > batch,
> > > > >> > > > >> which
> > > > >> > > > >> > >> > should
> > > > >> > > > >> > >> > > > now
> > > > >> > > > >> > >> > > > > > > > succeed.
> > > > >> > > > >> > >> > > > > > > > > >>> For
> > > > >> > > > >> > >> > > > > > > > > >>>>> the transactional producer,
> allow
> > > the
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>> // batch to fail. When
> processing
> > > the
> > > > >> > > failed
> > > > >> > > > >> > batch,
> > > > >> > > > >> > >> we
> > > > >> > > > >> > >> > > will
> > > > >> > > > >> > >> > > > > > > > > transition
> > > > >> > > > >> > >> > > > > > > > > >>> to
> > > > >> > > > >> > >> > > > > > > > > >>>>> an abortable error and set a
> flag
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>> // indicating that we need to
> > > bump the
> > > > >> > > epoch
> > > > >> > > > >> (if
> > > > >> > > > >> > >> > > supported
> > > > >> > > > >> > >> > > > by
> > > > >> > > > >> > >> > > > > > the
> > > > >> > > > >> > >> > > > > > > > > >>> broker).
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>> if (error ==
> > > > >> > Errors.*UNKNOWN_PRODUCER_ID*)
> > > > >> > > {
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>       if
> (response.logStartOffset
> > > ==
> > > > >> -1)
> > > > >> > {
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           // We don't know
> the log
> > > > >> start
> > > > >> > > > offset
> > > > >> > > > >> > with
> > > > >> > > > >> > >> > this
> > > > >> > > > >> > >> > > > > > > response.
> > > > >> > > > >> > >> > > > > > > > > We
> > > > >> > > > >> > >> > > > > > > > > >>> should
> > > > >> > > > >> > >> > > > > > > > > >>>>> just retry the request until
> we
> > > get
> > > > >> it.
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           // The
> > > UNKNOWN_PRODUCER_ID
> > > > >> > error
> > > > >> > > > code
> > > > >> > > > >> > was
> > > > >> > > > >> > >> > added
> > > > >> > > > >> > >> > > > > along
> > > > >> > > > >> > >> > > > > > > > with
> > > > >> > > > >> > >> > > > > > > > > >>> the new
> > > > >> > > > >> > >> > > > > > > > > >>>>> ProduceResponse which
> includes the
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           // logStartOffset.
> So
> > > the
> > > > >> '-1'
> > > > >> > > > >> sentinel
> > > > >> > > > >> > is
> > > > >> > > > >> > >> > not
> > > > >> > > > >> > >> > > > for
> > > > >> > > > >> > >> > > > > > > > backward
> > > > >> > > > >> > >> > > > > > > > > >>>>> compatibility. Instead, it is
> > > possible
> > > > >> > for
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           // a broker to not
> know
> > > the
> > > > >> > > > >> > >> logStartOffset at
> > > > >> > > > >> > >> > > > when
> > > > >> > > > >> > >> > > > > it
> > > > >> > > > >> > >> > > > > > > is
> > > > >> > > > >> > >> > > > > > > > > >>> returning
> > > > >> > > > >> > >> > > > > > > > > >>>>> the response because the
> partition
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           // may have moved
> away
> > > from
> > > > >> the
> > > > >> > > > >> broker
> > > > >> > > > >> > >> from
> > > > >> > > > >> > >> > the
> > > > >> > > > >> > >> > > > > time
> > > > >> > > > >> > >> > > > > > > the
> > > > >> > > > >> > >> > > > > > > > > >>> error was
> > > > >> > > > >> > >> > > > > > > > > >>>>> initially raised to the time
> the
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           // response was
> being
> > > > >> > > constructed.
> > > > >> > > > In
> > > > >> > > > >> > >> these
> > > > >> > > > >> > >> > > > cases,
> > > > >> > > > >> > >> > > > > we
> > > > >> > > > >> > >> > > > > > > > > should
> > > > >> > > > >> > >> > > > > > > > > >>> just
> > > > >> > > > >> > >> > > > > > > > > >>>>> retry the request: we are
> > > guaranteed
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           // to eventually
> get a
> > > > >> > > > logStartOffset
> > > > >> > > > >> > once
> > > > >> > > > >> > >> > > things
> > > > >> > > > >> > >> > > > > > > settle
> > > > >> > > > >> > >> > > > > > > > > down.
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>       if
> > > > >> (batch.sequenceHasBeenReset()) {
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           // When the first
> > > inflight
> > > > >> > batch
> > > > >> > > > >> fails
> > > > >> > > > >> > >> due to
> > > > >> > > > >> > >> > > the
> > > > >> > > > >> > >> > > > > > > > > truncation
> > > > >> > > > >> > >> > > > > > > > > >>> case,
> > > > >> > > > >> > >> > > > > > > > > >>>>> then the sequences of all the
> > > other
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           // in flight batches
> > > would
> > > > >> have
> > > > >> > > > been
> > > > >> > > > >> > >> > restarted
> > > > >> > > > >> > >> > > > from
> > > > >> > > > >> > >> > > > > > the
> > > > >> > > > >> > >> > > > > > > > > >>> beginning.
> > > > >> > > > >> > >> > > > > > > > > >>>>> However, when those responses
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           // come back from
> the
> > > > >> broker,
> > > > >> > > they
> > > > >> > > > >> would
> > > > >> > > > >> > >> also
> > > > >> > > > >> > >> > > > come
> > > > >> > > > >> > >> > > > > > with
> > > > >> > > > >> > >> > > > > > > > an
> > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID error. In
> this
> > > > >> case,
> > > > >> > we
> > > > >> > > > >> should
> > > > >> > > > >> > >> not
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           // reset the
> sequence
> > > > >> numbers
> > > > >> > to
> > > > >> > > > the
> > > > >> > > > >> > >> > beginning.
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>       } else if
> > > > >> > > > >> > >> > > > >
> (lastAckedOffset(batch.topicPartition).orElse(
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > > > >> > > > >> > >> > > > response.logStartOffset) {
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           // The head of the
> log
> > > has
> > > > >> been
> > > > >> > > > >> removed,
> > > > >> > > > >> > >> > > probably
> > > > >> > > > >> > >> > > > > due
> > > > >> > > > >> > >> > > > > > > to
> > > > >> > > > >> > >> > > > > > > > > the
> > > > >> > > > >> > >> > > > > > > > > >>>>> retention time elapsing. In
> this
> > > case,
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           // we expect to
> lose the
> > > > >> > producer
> > > > >> > > > >> state.
> > > > >> > > > >> > >> For
> > > > >> > > > >> > >> > > the
> > > > >> > > > >> > >> > > > > > > > > transactional
> > > > >> > > > >> > >> > > > > > > > > >>>>> producer, reset the sequences
> of
> > > all
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           // inflight batches
> to
> > > be
> > > > >> from
> > > > >> > > the
> > > > >> > > > >> > >> beginning
> > > > >> > > > >> > >> > > and
> > > > >> > > > >> > >> > > > > > retry
> > > > >> > > > >> > >> > > > > > > > > them,
> > > > >> > > > >> > >> > > > > > > > > >>> so
> > > > >> > > > >> > >> > > > > > > > > >>>>> that the transaction does not
> > > need to
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           // be aborted. For
> the
> > > > >> > idempotent
> > > > >> > > > >> > >> producer,
> > > > >> > > > >> > >> > > bump
> > > > >> > > > >> > >> > > > > the
> > > > >> > > > >> > >> > > > > > > > epoch
> > > > >> > > > >> > >> > > > > > > > > to
> > > > >> > > > >> > >> > > > > > > > > >>> avoid
> > > > >> > > > >> > >> > > > > > > > > >>>>> reusing (sequence, epoch)
> pairs
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           if
> (isTransactional()) {
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > >
> > > > >> > > >
> txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > > > >> > > > >> > >> > > > > > > > > >>>>> this.producerIdAndEpoch);
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           } else {
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > >
> > > > >> requestEpochBumpForPartition(batch.topicPartition);
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           }
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>       if (!isTransactional())
> {
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           // For the
> idempotent
> > > > >> producer,
> > > > >> > > > >> always
> > > > >> > > > >> > >> retry
> > > > >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > > >> > > > >> > >> > > > > > > > > >>>>> errors. If the batch has the
> > > current
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           // producer ID and
> > > epoch,
> > > > >> > > request a
> > > > >> > > > >> bump
> > > > >> > > > >> > >> of
> > > > >> > > > >> > >> > the
> > > > >> > > > >> > >> > > > > > epoch.
> > > > >> > > > >> > >> > > > > > > > > >>> Otherwise
> > > > >> > > > >> > >> > > > > > > > > >>>>> just retry the produce.
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > >
> > > requestEpochBumpForPartition(batch.topicPartition);
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>> }
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>> I was considering keeping this
> > > > >> behavior —
> > > > >> > > but
> > > > >> > > > >> am
> > > > >> > > > >> > >> open
> > > > >> > > > >> > >> > to
> > > > >> > > > >> > >> > > > > > > > simplifying
> > > > >> > > > >> > >> > > > > > > > > >>> it.
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>> We are leaving changes to
> older
> > > > >> clients
> > > > >> > off
> > > > >> > > > the
> > > > >> > > > >> > >> table
> > > > >> > > > >> > >> > > here
> > > > >> > > > >> > >> > > > > > since
> > > > >> > > > >> > >> > > > > > > it
> > > > >> > > > >> > >> > > > > > > > > >>> caused
> > > > >> > > > >> > >> > > > > > > > > >>>>> many issues for clients in the
> > > past.
> > > > >> > > > Previously
> > > > >> > > > >> > this
> > > > >> > > > >> > >> > was
> > > > >> > > > >> > >> > > a
> > > > >> > > > >> > >> > > > > > fatal
> > > > >> > > > >> > >> > > > > > > > > error
> > > > >> > > > >> > >> > > > > > > > > >>> and
> > > > >> > > > >> > >> > > > > > > > > >>>>> we didn't have the mechanisms
> in
> > > > >> place to
> > > > >> > > > >> detect
> > > > >> > > > >> > >> when
> > > > >> > > > >> > >> > > this
> > > > >> > > > >> > >> > > > > was
> > > > >> > > > >> > >> > > > > > a
> > > > >> > > > >> > >> > > > > > > > > >>> legitimate
> > > > >> > > > >> > >> > > > > > > > > >>>>> case vs some bug or gap in the
> > > > >> protocol.
> > > > >> > > > >> Ensuring
> > > > >> > > > >> > >> each
> > > > >> > > > >> > >> > > > > > > transaction
> > > > >> > > > >> > >> > > > > > > > > has
> > > > >> > > > >> > >> > > > > > > > > >>> its
> > > > >> > > > >> > >> > > > > > > > > >>>>> own epoch should close this
> gap.
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>> And to address Jeff's second
> > > point:
> > > > >> > > > >> > >> > > > > > > > > >>>>> *does the typical produce
> request
> > > path
> > > > >> > > append
> > > > >> > > > >> > >> records
> > > > >> > > > >> > >> > to
> > > > >> > > > >> > >> > > > > local
> > > > >> > > > >> > >> > > > > > > log
> > > > >> > > > >> > >> > > > > > > > > >>> along*
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>> *with the
> currentTxnFirstOffset
> > > > >> > > information?
> > > > >> > > > I
> > > > >> > > > >> > would
> > > > >> > > > >> > >> > like
> > > > >> > > > >> > >> > > > to
> > > > >> > > > >> > >> > > > > > > > > >>> understand*
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>> *when the field is written to
> > > disk.*
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>> Yes, the first produce request
> > > > >> populates
> > > > >> > > this
> > > > >> > > > >> > field
> > > > >> > > > >> > >> and
> > > > >> > > > >> > >> > > > > writes
> > > > >> > > > >> > >> > > > > > > the
> > > > >> > > > >> > >> > > > > > > > > >>> offset
> > > > >> > > > >> > >> > > > > > > > > >>>>> as part of the record batch
> and
> > > also
> > > > >> to
> > > > >> > the
> > > > >> > > > >> > producer
> > > > >> > > > >> > >> > > state
> > > > >> > > > >> > >> > > > > > > > snapshot.
> > > > >> > > > >> > >> > > > > > > > > >>> When
> > > > >> > > > >> > >> > > > > > > > > >>>>> we reload the records on
> restart
> > > > >> and/or
> > > > >> > > > >> > >> reassignment,
> > > > >> > > > >> > >> > we
> > > > >> > > > >> > >> > > > > > > repopulate
> > > > >> > > > >> > >> > > > > > > > > >>> this
> > > > >> > > > >> > >> > > > > > > > > >>>>> field with the snapshot from
> disk
> > > > >> along
> > > > >> > > with
> > > > >> > > > >> the
> > > > >> > > > >> > >> rest
> > > > >> > > > >> > >> > of
> > > > >> > > > >> > >> > > > the
> > > > >> > > > >> > >> > > > > > > > producer
> > > > >> > > > >> > >> > > > > > > > > >>>>> state.
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>> Let me know if there are
> further
> > > > >> comments
> > > > >> > > > >> and/or
> > > > >> > > > >> > >> > > questions.
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>> Thanks,
> > > > >> > > > >> > >> > > > > > > > > >>>>> Justine
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>> On Tue, Nov 22, 2022 at 9:00
> PM
> > > Jeff
> > > > >> Kim
> > > > >> > > > >> > >> > > > > > > > > <jeff.kim@confluent.io.invalid
> > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>> wrote:
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>> Hi Justine,
> > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>> Thanks for the KIP! I have
> two
> > > > >> > questions:
> > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>> 1) For new clients, we can
> once
> > > again
> > > > >> > > return
> > > > >> > > > >> an
> > > > >> > > > >> > >> error
> > > > >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > > >> > > > >> > >> > > > > > > > > >>>>>> for sequences
> > > > >> > > > >> > >> > > > > > > > > >>>>>> that are non-zero when there
> is
> > > no
> > > > >> > > producer
> > > > >> > > > >> state
> > > > >> > > > >> > >> > > present
> > > > >> > > > >> > >> > > > on
> > > > >> > > > >> > >> > > > > > the
> > > > >> > > > >> > >> > > > > > > > > >>> server.
> > > > >> > > > >> > >> > > > > > > > > >>>>>> This will indicate we missed
> the
> > > 0
> > > > >> > > sequence
> > > > >> > > > >> and
> > > > >> > > > >> > we
> > > > >> > > > >> > >> > don't
> > > > >> > > > >> > >> > > > yet
> > > > >> > > > >> > >> > > > > > > want
> > > > >> > > > >> > >> > > > > > > > to
> > > > >> > > > >> > >> > > > > > > > > >>>>> write
> > > > >> > > > >> > >> > > > > > > > > >>>>>> to the log.
> > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>> I would like to understand
> the
> > > > >> current
> > > > >> > > > >> behavior
> > > > >> > > > >> > to
> > > > >> > > > >> > >> > > handle
> > > > >> > > > >> > >> > > > > > older
> > > > >> > > > >> > >> > > > > > > > > >>> clients,
> > > > >> > > > >> > >> > > > > > > > > >>>>>> and if there are any changes
> we
> > > are
> > > > >> > > making.
> > > > >> > > > >> Maybe
> > > > >> > > > >> > >> I'm
> > > > >> > > > >> > >> > > > > missing
> > > > >> > > > >> > >> > > > > > > > > >>> something,
> > > > >> > > > >> > >> > > > > > > > > >>>>>> but we would want to identify
> > > > >> whether we
> > > > >> > > > >> missed
> > > > >> > > > >> > >> the 0
> > > > >> > > > >> > >> > > > > sequence
> > > > >> > > > >> > >> > > > > > > for
> > > > >> > > > >> > >> > > > > > > > > >>> older
> > > > >> > > > >> > >> > > > > > > > > >>>>>> clients, no?
> > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>> 2) Upon returning from the
> > > > >> transaction
> > > > >> > > > >> > >> coordinator, we
> > > > >> > > > >> > >> > > can
> > > > >> > > > >> > >> > > > > set
> > > > >> > > > >> > >> > > > > > > the
> > > > >> > > > >> > >> > > > > > > > > >>>>>> transaction
> > > > >> > > > >> > >> > > > > > > > > >>>>>> as ongoing on the leader by
> > > > >> populating
> > > > >> > > > >> > >> > > > currentTxnFirstOffset
> > > > >> > > > >> > >> > > > > > > > > >>>>>> through the typical produce
> > > request
> > > > >> > > > handling.
> > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>> does the typical produce
> request
> > > path
> > > > >> > > append
> > > > >> > > > >> > >> records
> > > > >> > > > >> > >> > to
> > > > >> > > > >> > >> > > > > local
> > > > >> > > > >> > >> > > > > > > log
> > > > >> > > > >> > >> > > > > > > > > >>> along
> > > > >> > > > >> > >> > > > > > > > > >>>>>> with the
> currentTxnFirstOffset
> > > > >> > > information?
> > > > >> > > > I
> > > > >> > > > >> > would
> > > > >> > > > >> > >> > like
> > > > >> > > > >> > >> > > > to
> > > > >> > > > >> > >> > > > > > > > > understand
> > > > >> > > > >> > >> > > > > > > > > >>>>>> when the field is written to
> > > disk.
> > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>> Thanks,
> > > > >> > > > >> > >> > > > > > > > > >>>>>> Jeff
> > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>> On Tue, Nov 22, 2022 at 4:44
> PM
> > > Artem
> > > > >> > > > Livshits
> > > > >> > > > >> > >> > > > > > > > > >>>>>> <alivshits@confluent.io
> .invalid>
> > > > >> wrote:
> > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> Hi Justine,
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> Thank you for the KIP.  I
> have
> > > one
> > > > >> > > > question.
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> 5) For new clients, we can
> once
> > > > >> again
> > > > >> > > > return
> > > > >> > > > >> an
> > > > >> > > > >> > >> error
> > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> I believe we had problems
> in the
> > > > >> past
> > > > >> > > with
> > > > >> > > > >> > >> returning
> > > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> because it was considered
> fatal
> > > and
> > > > >> > > > required
> > > > >> > > > >> > >> client
> > > > >> > > > >> > >> > > > > restart.
> > > > >> > > > >> > >> > > > > > > It
> > > > >> > > > >> > >> > > > > > > > > >>> would
> > > > >> > > > >> > >> > > > > > > > > >>>>> be
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> good to spell out the new
> client
> > > > >> > behavior
> > > > >> > > > >> when
> > > > >> > > > >> > it
> > > > >> > > > >> > >> > > > receives
> > > > >> > > > >> > >> > > > > > the
> > > > >> > > > >> > >> > > > > > > > > error.
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> -Artem
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> On Tue, Nov 22, 2022 at
> 10:00 AM
> > > > >> > Justine
> > > > >> > > > >> Olshan
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> <jo...@confluent.io.invalid>
> > > > >> wrote:
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks for taking a look
> > > Matthias.
> > > > >> > I've
> > > > >> > > > >> tried
> > > > >> > > > >> > to
> > > > >> > > > >> > >> > > answer
> > > > >> > > > >> > >> > > > > your
> > > > >> > > > >> > >> > > > > > > > > >>>>> questions
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> below:
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 10)
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Right — so the hanging
> > > transaction
> > > > >> > only
> > > > >> > > > >> occurs
> > > > >> > > > >> > >> when
> > > > >> > > > >> > >> > we
> > > > >> > > > >> > >> > > > > have
> > > > >> > > > >> > >> > > > > > a
> > > > >> > > > >> > >> > > > > > > > late
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> message
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> come in and the partition
> is
> > > never
> > > > >> > added
> > > > >> > > > to
> > > > >> > > > >> a
> > > > >> > > > >> > >> > > > transaction
> > > > >> > > > >> > >> > > > > > > again.
> > > > >> > > > >> > >> > > > > > > > > If
> > > > >> > > > >> > >> > > > > > > > > >>>>> we
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> never add the partition to
> a
> > > > >> > > transaction,
> > > > >> > > > we
> > > > >> > > > >> > will
> > > > >> > > > >> > >> > > never
> > > > >> > > > >> > >> > > > > > write
> > > > >> > > > >> > >> > > > > > > a
> > > > >> > > > >> > >> > > > > > > > > >>>>> marker
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> and
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> never advance the LSO.
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> If we do end up adding the
> > > > >> partition
> > > > >> > to
> > > > >> > > > the
> > > > >> > > > >> > >> > > transaction
> > > > >> > > > >> > >> > > > (I
> > > > >> > > > >> > >> > > > > > > > suppose
> > > > >> > > > >> > >> > > > > > > > > >>>>> this
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> can
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> happen before or after the
> late
> > > > >> > message
> > > > >> > > > >> comes
> > > > >> > > > >> > in)
> > > > >> > > > >> > >> > then
> > > > >> > > > >> > >> > > > we
> > > > >> > > > >> > >> > > > > > will
> > > > >> > > > >> > >> > > > > > > > > >>>>> include
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> the
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> late message in the next
> > > > >> (incorrect)
> > > > >> > > > >> > transaction.
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> So perhaps it is clearer to
> > > make
> > > > >> the
> > > > >> > > > >> > distinction
> > > > >> > > > >> > >> > > between
> > > > >> > > > >> > >> > > > > > > > messages
> > > > >> > > > >> > >> > > > > > > > > >>>>> that
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> eventually get added to the
> > > > >> > transaction
> > > > >> > > > (but
> > > > >> > > > >> > the
> > > > >> > > > >> > >> > wrong
> > > > >> > > > >> > >> > > > > one)
> > > > >> > > > >> > >> > > > > > or
> > > > >> > > > >> > >> > > > > > > > > >>>>> messages
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> that never get added and
> become
> > > > >> > hanging.
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 20)
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> The client side change for
> 2 is
> > > > >> > removing
> > > > >> > > > the
> > > > >> > > > >> > >> > > > addPartitions
> > > > >> > > > >> > >> > > > > > to
> > > > >> > > > >> > >> > > > > > > > > >>>>>> transaction
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> call. We don't need to make
> > > this
> > > > >> from
> > > > >> > > the
> > > > >> > > > >> > >> producer
> > > > >> > > > >> > >> > to
> > > > >> > > > >> > >> > > > the
> > > > >> > > > >> > >> > > > > > txn
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> coordinator,
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> only server side.
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> In my opinion, the issue
> with
> > > the
> > > > >> > > > >> > >> addPartitionsToTxn
> > > > >> > > > >> > >> > > > call
> > > > >> > > > >> > >> > > > > > for
> > > > >> > > > >> > >> > > > > > > > > older
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> clients
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> is that we don't have the
> epoch
> > > > >> bump,
> > > > >> > so
> > > > >> > > > we
> > > > >> > > > >> > don't
> > > > >> > > > >> > >> > know
> > > > >> > > > >> > >> > > > if
> > > > >> > > > >> > >> > > > > > the
> > > > >> > > > >> > >> > > > > > > > > >>> message
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> belongs to the previous
> > > > >> transaction or
> > > > >> > > > this
> > > > >> > > > >> > one.
> > > > >> > > > >> > >> We
> > > > >> > > > >> > >> > > need
> > > > >> > > > >> > >> > > > > to
> > > > >> > > > >> > >> > > > > > > > check
> > > > >> > > > >> > >> > > > > > > > > if
> > > > >> > > > >> > >> > > > > > > > > >>>>>> the
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> partition has been added to
> > > this
> > > > >> > > > >> transaction.
> > > > >> > > > >> > Of
> > > > >> > > > >> > >> > > course,
> > > > >> > > > >> > >> > > > > > this
> > > > >> > > > >> > >> > > > > > > > > means
> > > > >> > > > >> > >> > > > > > > > > >>>>> we
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> won't completely cover the
> case
> > > > >> where
> > > > >> > we
> > > > >> > > > >> have a
> > > > >> > > > >> > >> > really
> > > > >> > > > >> > >> > > > > late
> > > > >> > > > >> > >> > > > > > > > > message
> > > > >> > > > >> > >> > > > > > > > > >>>>> and
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> we
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> have added the partition to
> > > the new
> > > > >> > > > >> > transaction,
> > > > >> > > > >> > >> but
> > > > >> > > > >> > >> > > > > that's
> > > > >> > > > >> > >> > > > > > > > > >>>>>> unfortunately
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> something we will need the
> new
> > > > >> clients
> > > > >> > > to
> > > > >> > > > >> > cover.
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 30)
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Transaction is ongoing =
> > > partition
> > > > >> was
> > > > >> > > > >> added to
> > > > >> > > > >> > >> > > > > transaction
> > > > >> > > > >> > >> > > > > > > via
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> addPartitionsToTxn. We
> check
> > > this
> > > > >> with
> > > > >> > > the
> > > > >> > > > >> > >> > > > > > > DescribeTransactions
> > > > >> > > > >> > >> > > > > > > > > >>> call.
> > > > >> > > > >> > >> > > > > > > > > >>>>>> Let
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> me know if this wasn't
> > > sufficiently
> > > > >> > > > >> explained
> > > > >> > > > >> > >> here:
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > >
> > > > >> > > > >> > >> > > > > > > >
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > >
> > > > >> > > > >> > >> > > > >
> > > > >> > > > >> > >> > > >
> > > > >> > > > >> > >> > >
> > > > >> > > > >> > >> >
> > > > >> > > > >> > >>
> > > > >> > > > >> >
> > > > >> > > > >>
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 40)
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> The idea here is that if
> any
> > > > >> messages
> > > > >> > > > >> somehow
> > > > >> > > > >> > >> come
> > > > >> > > > >> > >> > in
> > > > >> > > > >> > >> > > > > before
> > > > >> > > > >> > >> > > > > > > we
> > > > >> > > > >> > >> > > > > > > > > get
> > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> new
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> epoch to the producer, they
> > > will be
> > > > >> > > > fenced.
> > > > >> > > > >> > >> However,
> > > > >> > > > >> > >> > > if
> > > > >> > > > >> > >> > > > we
> > > > >> > > > >> > >> > > > > > > don't
> > > > >> > > > >> > >> > > > > > > > > >>>>> think
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> this
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> is necessary, it can be
> > > discussed
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 50)
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> It should be synchronous
> > > because
> > > > >> if we
> > > > >> > > > have
> > > > >> > > > >> an
> > > > >> > > > >> > >> event
> > > > >> > > > >> > >> > > > (ie,
> > > > >> > > > >> > >> > > > > an
> > > > >> > > > >> > >> > > > > > > > > error)
> > > > >> > > > >> > >> > > > > > > > > >>>>>> that
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> causes us to need to abort
> the
> > > > >> > > > transaction,
> > > > >> > > > >> we
> > > > >> > > > >> > >> need
> > > > >> > > > >> > >> > to
> > > > >> > > > >> > >> > > > > know
> > > > >> > > > >> > >> > > > > > > > which
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> partitions to send
> transaction
> > > > >> markers
> > > > >> > > to.
> > > > >> > > > >> We
> > > > >> > > > >> > >> know
> > > > >> > > > >> > >> > the
> > > > >> > > > >> > >> > > > > > > > partitions
> > > > >> > > > >> > >> > > > > > > > > >>>>>> because
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> we added them to the
> > > coordinator
> > > > >> via
> > > > >> > the
> > > > >> > > > >> > >> > > > > addPartitionsToTxn
> > > > >> > > > >> > >> > > > > > > > call.
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Previously we have had
> > > asynchronous
> > > > >> > > calls
> > > > >> > > > in
> > > > >> > > > >> > the
> > > > >> > > > >> > >> > past
> > > > >> > > > >> > >> > > > (ie,
> > > > >> > > > >> > >> > > > > > > > writing
> > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> commit markers when the
> > > > >> transaction is
> > > > >> > > > >> > completed)
> > > > >> > > > >> > >> > but
> > > > >> > > > >> > >> > > > > often
> > > > >> > > > >> > >> > > > > > > this
> > > > >> > > > >> > >> > > > > > > > > >>> just
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> causes confusion as we
> need to
> > > wait
> > > > >> > for
> > > > >> > > > some
> > > > >> > > > >> > >> > > operations
> > > > >> > > > >> > >> > > > to
> > > > >> > > > >> > >> > > > > > > > > complete.
> > > > >> > > > >> > >> > > > > > > > > >>>>> In
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> the
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> writing commit markers
> case,
> > > > >> clients
> > > > >> > > often
> > > > >> > > > >> see
> > > > >> > > > >> > >> > > > > > > > > >>>>> CONCURRENT_TRANSACTIONs
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> error messages and that
> can be
> > > > >> > > confusing.
> > > > >> > > > >> For
> > > > >> > > > >> > >> that
> > > > >> > > > >> > >> > > > reason,
> > > > >> > > > >> > >> > > > > > it
> > > > >> > > > >> > >> > > > > > > > may
> > > > >> > > > >> > >> > > > > > > > > be
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> simpler to just have
> > > synchronous
> > > > >> > calls —
> > > > >> > > > >> > >> especially
> > > > >> > > > >> > >> > if
> > > > >> > > > >> > >> > > > we
> > > > >> > > > >> > >> > > > > > need
> > > > >> > > > >> > >> > > > > > > > to
> > > > >> > > > >> > >> > > > > > > > > >>>>> block
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> on
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> some operation's completion
> > > anyway
> > > > >> > > before
> > > > >> > > > we
> > > > >> > > > >> > can
> > > > >> > > > >> > >> > start
> > > > >> > > > >> > >> > > > the
> > > > >> > > > >> > >> > > > > > > next
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> transaction. And yes, I
> meant
> > > > >> > > > coordinator. I
> > > > >> > > > >> > will
> > > > >> > > > >> > >> > fix
> > > > >> > > > >> > >> > > > > that.
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> 60)
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> When we are checking if the
> > > > >> > transaction
> > > > >> > > is
> > > > >> > > > >> > >> ongoing,
> > > > >> > > > >> > >> > we
> > > > >> > > > >> > >> > > > > need
> > > > >> > > > >> > >> > > > > > to
> > > > >> > > > >> > >> > > > > > > > > make
> > > > >> > > > >> > >> > > > > > > > > >>> a
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> round
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> trip from the leader
> partition
> > > to
> > > > >> the
> > > > >> > > > >> > transaction
> > > > >> > > > >> > >> > > > > > coordinator.
> > > > >> > > > >> > >> > > > > > > > In
> > > > >> > > > >> > >> > > > > > > > > >>> the
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> time
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> we are waiting for this
> > > message to
> > > > >> > come
> > > > >> > > > >> back,
> > > > >> > > > >> > in
> > > > >> > > > >> > >> > > theory
> > > > >> > > > >> > >> > > > we
> > > > >> > > > >> > >> > > > > > > could
> > > > >> > > > >> > >> > > > > > > > > >>> have
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> sent
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> a commit/abort call that
> would
> > > make
> > > > >> > the
> > > > >> > > > >> > original
> > > > >> > > > >> > >> > > result
> > > > >> > > > >> > >> > > > of
> > > > >> > > > >> > >> > > > > > the
> > > > >> > > > >> > >> > > > > > > > > check
> > > > >> > > > >> > >> > > > > > > > > >>>>>> out
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> of
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> date. That is why we can
> check
> > > the
> > > > >> > > leader
> > > > >> > > > >> state
> > > > >> > > > >> > >> > before
> > > > >> > > > >> > >> > > > we
> > > > >> > > > >> > >> > > > > > > write
> > > > >> > > > >> > >> > > > > > > > to
> > > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> log.
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> I'm happy to update the
> KIP if
> > > > >> some of
> > > > >> > > > these
> > > > >> > > > >> > >> things
> > > > >> > > > >> > >> > > were
> > > > >> > > > >> > >> > > > > not
> > > > >> > > > >> > >> > > > > > > > > clear.
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks,
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> Justine
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> On Mon, Nov 21, 2022 at
> 7:11 PM
> > > > >> > Matthias
> > > > >> > > > J.
> > > > >> > > > >> > Sax <
> > > > >> > > > >> > >> > > > > > > > mjsax@apache.org
> > > > >> > > > >> > >> > > > > > > > > >
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> wrote:
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Thanks for the KIP.
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Couple of clarification
> > > questions
> > > > >> (I
> > > > >> > am
> > > > >> > > > >> not a
> > > > >> > > > >> > >> > broker
> > > > >> > > > >> > >> > > > > expert
> > > > >> > > > >> > >> > > > > > > do
> > > > >> > > > >> > >> > > > > > > > > >>>>> maybe
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> some question are obvious
> for
> > > > >> others,
> > > > >> > > but
> > > > >> > > > >> not
> > > > >> > > > >> > >> for
> > > > >> > > > >> > >> > me
> > > > >> > > > >> > >> > > > with
> > > > >> > > > >> > >> > > > > > my
> > > > >> > > > >> > >> > > > > > > > lack
> > > > >> > > > >> > >> > > > > > > > > >>>>> of
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> broker knowledge).
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (10)
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> 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.
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> What happens if the
> message
> > > come
> > > > >> in
> > > > >> > > > before
> > > > >> > > > >> the
> > > > >> > > > >> > >> next
> > > > >> > > > >> > >> > > > > > > > > >>>>>> addPartitionsToTxn
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> request? It seems the
> broker
> > > > >> hosting
> > > > >> > > the
> > > > >> > > > >> data
> > > > >> > > > >> > >> > > > partitions
> > > > >> > > > >> > >> > > > > > > won't
> > > > >> > > > >> > >> > > > > > > > > know
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> anything about it and
> append
> > > it to
> > > > >> > the
> > > > >> > > > >> > >> partition,
> > > > >> > > > >> > >> > > too?
> > > > >> > > > >> > >> > > > > What
> > > > >> > > > >> > >> > > > > > > is
> > > > >> > > > >> > >> > > > > > > > > the
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> difference between both
> cases?
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Also, it seems a TX would
> only
> > > > >> hang,
> > > > >> > if
> > > > >> > > > >> there
> > > > >> > > > >> > >> is no
> > > > >> > > > >> > >> > > > > > following
> > > > >> > > > >> > >> > > > > > > > TX
> > > > >> > > > >> > >> > > > > > > > > >>>>> that
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> is
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> either committer or
> aborted?
> > > Thus,
> > > > >> > for
> > > > >> > > > the
> > > > >> > > > >> > case
> > > > >> > > > >> > >> > > above,
> > > > >> > > > >> > >> > > > > the
> > > > >> > > > >> > >> > > > > > TX
> > > > >> > > > >> > >> > > > > > > > > might
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> actually not hang (of
> course,
> > > we
> > > > >> > might
> > > > >> > > > get
> > > > >> > > > >> an
> > > > >> > > > >> > >> EOS
> > > > >> > > > >> > >> > > > > violation
> > > > >> > > > >> > >> > > > > > > if
> > > > >> > > > >> > >> > > > > > > > > the
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> first
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> TX was aborted and the
> second
> > > > >> > > committed,
> > > > >> > > > or
> > > > >> > > > >> > the
> > > > >> > > > >> > >> > other
> > > > >> > > > >> > >> > > > way
> > > > >> > > > >> > >> > > > > > > > > around).
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (20)
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Of course, 1 and 2
> require
> > > > >> > client-side
> > > > >> > > > >> > >> changes, so
> > > > >> > > > >> > >> > > for
> > > > >> > > > >> > >> > > > > > older
> > > > >> > > > >> > >> > > > > > > > > >>>>>> clients,
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> those approaches won’t
> apply.
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> For (1) I understand why a
> > > client
> > > > >> > > change
> > > > >> > > > is
> > > > >> > > > >> > >> > > necessary,
> > > > >> > > > >> > >> > > > > but
> > > > >> > > > >> > >> > > > > > > not
> > > > >> > > > >> > >> > > > > > > > > sure
> > > > >> > > > >> > >> > > > > > > > > >>>>>> why
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> we need a client change
> for
> > > (2).
> > > > >> Can
> > > > >> > > you
> > > > >> > > > >> > >> elaborate?
> > > > >> > > > >> > >> > > --
> > > > >> > > > >> > >> > > > > > Later
> > > > >> > > > >> > >> > > > > > > > you
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> explain
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> that we should send a
> > > > >> > > > >> > >> DescribeTransactionRequest,
> > > > >> > > > >> > >> > > but I
> > > > >> > > > >> > >> > > > > am
> > > > >> > > > >> > >> > > > > > > not
> > > > >> > > > >> > >> > > > > > > > > sure
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> why?
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Can't we not just do an
> > > implicit
> > > > >> > > > >> > >> AddPartiitonToTx,
> > > > >> > > > >> > >> > > too?
> > > > >> > > > >> > >> > > > > If
> > > > >> > > > >> > >> > > > > > > the
> > > > >> > > > >> > >> > > > > > > > > old
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> producer correctly
> registered
> > > the
> > > > >> > > > partition
> > > > >> > > > >> > >> > already,
> > > > >> > > > >> > >> > > > the
> > > > >> > > > >> > >> > > > > > > > > >>>>>> TX-coordinator
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> can just ignore it as
> it's an
> > > > >> > > idempotent
> > > > >> > > > >> > >> operation?
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (30)
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> To cover older clients,
> we
> > > will
> > > > >> > > ensure a
> > > > >> > > > >> > >> > transaction
> > > > >> > > > >> > >> > > > is
> > > > >> > > > >> > >> > > > > > > > ongoing
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> before
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> we write to a transaction
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Not sure what you mean by
> > > this?
> > > > >> Can
> > > > >> > you
> > > > >> > > > >> > >> elaborate?
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (40)
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> [the TX-coordinator] will
> > > write
> > > > >> the
> > > > >> > > > >> prepare
> > > > >> > > > >> > >> commit
> > > > >> > > > >> > >> > > > > message
> > > > >> > > > >> > >> > > > > > > > with
> > > > >> > > > >> > >> > > > > > > > > a
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>> bumped
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> epoch and send
> > > > >> WriteTxnMarkerRequests
> > > > >> > > > with
> > > > >> > > > >> the
> > > > >> > > > >> > >> > bumped
> > > > >> > > > >> > >> > > > > > epoch.
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Why do we use the bumped
> > > epoch for
> > > > >> > > both?
> > > > >> > > > It
> > > > >> > > > >> > >> seems
> > > > >> > > > >> > >> > > more
> > > > >> > > > >> > >> > > > > > > > intuitive
> > > > >> > > > >> > >> > > > > > > > > to
> > > > >> > > > >> > >> > > > > > > > > >>>>>> use
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> the current epoch, and
> only
> > > return
> > > > >> > the
> > > > >> > > > >> bumped
> > > > >> > > > >> > >> epoch
> > > > >> > > > >> > >> > > to
> > > > >> > > > >> > >> > > > > the
> > > > >> > > > >> > >> > > > > > > > > >>>>> producer?
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (50) "Implicit
> > > > >> > > AddPartitionToTransaction"
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Why does the implicitly
> sent
> > > > >> request
> > > > >> > > need
> > > > >> > > > >> to
> > > > >> > > > >> > be
> > > > >> > > > >> > >> > > > > > synchronous?
> > > > >> > > > >> > >> > > > > > > > The
> > > > >> > > > >> > >> > > > > > > > > >>>>> KIP
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> also says
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> in case we need to abort
> and
> > > > >> need to
> > > > >> > > > know
> > > > >> > > > >> > which
> > > > >> > > > >> > >> > > > > partitions
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> What do you mean by this?
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> we don’t want to write
> to it
> > > > >> before
> > > > >> > we
> > > > >> > > > >> store
> > > > >> > > > >> > in
> > > > >> > > > >> > >> > the
> > > > >> > > > >> > >> > > > > > > > transaction
> > > > >> > > > >> > >> > > > > > > > > >>>>>>> manager
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Do you mean TX-coordinator
> > > > >> instead of
> > > > >> > > > >> > "manager"?
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (60)
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> For older clients and
> ensuring
> > > > >> that
> > > > >> > the
> > > > >> > > > TX
> > > > >> > > > >> is
> > > > >> > > > >> > >> > > ongoing,
> > > > >> > > > >> > >> > > > > you
> > > > >> > > > >> > >> > > > > > > > > >>>>> describe a
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> race condition. I am not
> sure
> > > if I
> > > > >> > can
> > > > >> > > > >> follow
> > > > >> > > > >> > >> here.
> > > > >> > > > >> > >> > > Can
> > > > >> > > > >> > >> > > > > you
> > > > >> > > > >> > >> > > > > > > > > >>>>>> elaborate?
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> -Matthias
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> On 11/18/22 1:21 PM,
> Justine
> > > > >> Olshan
> > > > >> > > > wrote:
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Hey all!
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> I'd like to start a
> > > discussion
> > > > >> on my
> > > > >> > > > >> proposal
> > > > >> > > > >> > >> to
> > > > >> > > > >> > >> > add
> > > > >> > > > >> > >> > > > > some
> > > > >> > > > >> > >> > > > > > > > > >>>>>> server-side
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> checks on transactions to
> > > avoid
> > > > >> > > hanging
> > > > >> > > > >> > >> > > transactions.
> > > > >> > > > >> > >> > > > I
> > > > >> > > > >> > >> > > > > > know
> > > > >> > > > >> > >> > > > > > > > > this
> > > > >> > > > >> > >> > > > > > > > > >>>>>> has
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> been
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> an issue for some time,
> so I
> > > > >> really
> > > > >> > > hope
> > > > >> > > > >> this
> > > > >> > > > >> > >> KIP
> > > > >> > > > >> > >> > > will
> > > > >> > > > >> > >> > > > > be
> > > > >> > > > >> > >> > > > > > > > > helpful
> > > > >> > > > >> > >> > > > > > > > > >>>>>> for
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>> many
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> users of EOS.
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> The KIP includes changes
> that
> > > > >> will
> > > > >> > be
> > > > >> > > > >> > >> compatible
> > > > >> > > > >> > >> > > with
> > > > >> > > > >> > >> > > > > old
> > > > >> > > > >> > >> > > > > > > > > clients
> > > > >> > > > >> > >> > > > > > > > > >>>>>> and
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> changes to improve
> > > performance
> > > > >> and
> > > > >> > > > >> > correctness
> > > > >> > > > >> > >> on
> > > > >> > > > >> > >> > > new
> > > > >> > > > >> > >> > > > > > > clients.
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Please take a look and
> leave
> > > any
> > > > >> > > > comments
> > > > >> > > > >> you
> > > > >> > > > >> > >> may
> > > > >> > > > >> > >> > > > have!
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> KIP:
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > >
> > > > >> > > > >> > >> > > > > > > >
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > >
> > > > >> > > > >> > >> > > > >
> > > > >> > > > >> > >> > > >
> > > > >> > > > >> > >> > >
> > > > >> > > > >> > >> >
> > > > >> > > > >> > >>
> > > > >> > > > >> >
> > > > >> > > > >>
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> JIRA:
> > > > >> > > > >> > >> > > >
> https://issues.apache.org/jira/browse/KAFKA-14402
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Thanks!
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Justine
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > > > >> > >> > > > > > > > > >>>>
> > > > >> > > > >> > >> > > > > > > > > >>>
> > > > >> > > > >> > >> > > > > > > > > >>
> > > > >> > > > >> > >> > > > > > > > > >
> > > > >> > > > >> > >> > > > > > > > >
> > > > >> > > > >> > >> > > > > > > >
> > > > >> > > > >> > >> > > > > > >
> > > > >> > > > >> > >> > > > > >
> > > > >> > > > >> > >> > > > >
> > > > >> > > > >> > >> > > >
> > > > >> > > > >> > >> > >
> > > > >> > > > >> > >> >
> > > > >> > > > >> > >>
> > > > >> > > > >> > >
> > > > >> > > > >> >
> > > > >> > > > >>
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > > >
> > >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Guozhang Wang <gu...@gmail.com>.
Thanks Justine for the replies! I agree with most of your thoughts.

Just for 3/7), though I agree for our own AK producer, since we do
"nextRequest(boolean hasIncompleteBatches)", we guarantee the end-txn
would not be sent until we've effectively flushed, but I was referring
to any future bugs or other buggy clients that the same client may get
into this situation, in which case we should give the client a clear
msg that "you did something wrong, and hence now you should fatally
close yourself". What I'm concerned about is that, by seeing an
"abortable error" or in some rare cases an "invalid record", the
client could not realize "something that's really bad happened". So
it's not about adding a new error, it's mainly about those real buggy
situations causing such "should never happen" cases, the errors return
would not be informative enough.

Thinking in other ways, if we believe that for most cases such error
codes would not reach the original clients since they would be
disconnected or even gone by that time, and only in some rare cases
they would still be seen by the sending clients, then why not make
them more fatal and more specific than generic.

Guozhang

On Fri, Jan 20, 2023 at 1:59 PM Justine Olshan
<jo...@confluent.io.invalid> wrote:
>
> Hey Guozhang. Thanks for taking a look and for the detailed comments! I'll
> do my best to address below.
>
> 1. I see what you are saying here, but I think I need to look through the
> sequence of events you mention. Typically we've seen this issue in a few
> cases.
>
>  One is when we have a producer disconnect when trying to produce.
> Typically in these cases, we abort the transaction. We've seen that after
> the markers are written, the disconnection can sometimes cause the request
> to get flushed to the broker. In this case, we don't need client handling
> because the producer we are responding to is gone. We just needed to make
> sure we didn't write to the log on the broker side. I'm trying to think of
> a case where we do have the client to return to. I'd think the same client
> couldn't progress to committing the transaction unless the produce request
> returned right? Of course, there is the incorrectly written clients case.
> I'll think on this a bit more and let you know if I come up with another
> scenario when we would return to an active client when the transaction is
> no longer ongoing.
>
> I was not aware that we checked the result of a send after we commit
> though. I'll need to look into that a bit more.
>
> 2. There were some questions about this in the discussion. The plan is to
> handle overflow with the mechanism we currently have in the producer. If we
> try to bump and the epoch will overflow, we actually allocate a new
> producer ID. I need to confirm the fencing logic on the last epoch (ie, we
> probably shouldn't allow any records to be produced with the final epoch
> since we can never properly fence that one).
>
> 3. I can agree with you that the current error handling is messy. I recall
> taking a look at your KIP a while back, but I think I mostly saw the
> section about how the errors were wrapped. Maybe I need to take another
> look. As for abortable error, the idea was that the handling would be
> simple -- if this error is seen, the transaction should be aborted -- no
> other logic about previous state or requests necessary. Is your concern
> simply about adding new errors? We were hoping to have an error that would
> have one meaning and many of the current errors have a history of meaning
> different things on different client versions. That was the main motivation
> for adding a new error.
>
> 4. This is a good point about record timestamp reordering. Timestamps don't
> affect compaction, but they do affect retention deletion. For that, kafka
> considers the largest timestamp in the segment, so I think a small amount
> of reordering (hopefully on the order of milliseconds or even seconds) will
> be ok. We take timestamps from clients so there is already a possibility
> for some drift and non-monotonically increasing timestamps.
>
> 5. Thanks for catching. The error is there, but it's actually that those
> fields should be 4+! Due to how the message generator works, I actually
> have to redefine those fields inside the `"AddPartitionsToTxnTransaction`
> block for it to build correctly. I'll fix it to be correct.
>
> 6. Correct -- we will only add the request to purgatory if the cache has no
> ongoing transaction. I can change the wording to make that clearer that we
> only place the request in purgatory if we need to contact the transaction
> coordinator.
>
> 7. We did take a look at some of the errors and it was hard to come up with
> a good one. I agree that InvalidTxnStateException is ideal except for the
> fact that it hasn't been returned on Produce requests before. The error
> handling for clients is a bit vague (which is why I opened KAFKA-14439
> <https://issues.apache.org/jira/browse/KAFKA-14439>), but the decision we
> made here was to only return errors that have been previously returned to
> producers. As for not being fatal, I think part of the theory was that in
> many cases, the producer would be disconnected. (See point 1) and this
> would just be an error to return from the server. I did plan to think about
> other cases, so let me know if you think of any as well!
>
> Lots to say! Let me know if you have further thoughts!
> Justine
>
> On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang <gu...@gmail.com>
> wrote:
>
> > Hello Justine,
> >
> > Thanks for the great write-up! I made a quick pass through it and here
> > are some thoughts (I have not been able to read through this thread so
> > pardon me if they have overlapped or subsumed by previous comments):
> >
> > First are some meta ones:
> >
> > 1. I think we need to also improve the client's experience once we
> > have this defence in place. More concretely, say a user's producer
> > code is like following:
> >
> > future = producer.send();
> > // producer.flush();
> > producer.commitTransaction();
> > future.get();
> >
> > Which resulted in the order of a) produce-request sent by producer, b)
> > end-txn-request sent by producer, c) end-txn-response sent back, d)
> > txn-marker-request sent from coordinator to partition leader, e)
> > produce-request finally received by the partition leader, before this
> > KIP e) step would be accepted causing a dangling txn; now it would be
> > rejected in step e) which is good. But from the client's point of view
> > now it becomes confusing since the `commitTransaction()` returns
> > successfully, but the "future" throws an invalid-epoch error, and they
> > are not sure if the transaction did succeed or not. In fact, it
> > "partially succeeded" with some msgs being rejected but others
> > committed successfully.
> >
> > Of course the easy way to avoid this is, always call
> > "producer.flush()" before commitTxn and that's what we do ourselves,
> > and what we recommend users do. But I suspect not everyone does it. In
> > fact I just checked the javadoc in KafkaProducer and our code snippet
> > does not include a `flush()` call. So I'm thinking maybe we can in
> > side the `commitTxn` code to enforce flushing before sending the
> > end-txn request.
> >
> > 2. I'd like to clarify a bit details on "just add partitions to the
> > transaction on the first produce request during a transaction". My
> > understanding is that the partition leader's cache has the producer id
> > / sequence / epoch for the latest txn, either on-going or is completed
> > (upon receiving the marker request from coordinator). When a produce
> > request is received, if
> >
> > * producer's epoch < cached epoch, or producer's epoch == cached epoch
> > but the latest txn is completed, leader directly reject with
> > invalid-epoch.
> > * producer's epoch > cached epoch, park the the request and send
> > add-partitions request to coordinator.
> >
> > In order to do it, does the coordinator need to bump the sequence and
> > reset epoch to 0 when the next epoch is going to overflow? If no need
> > to do so, then how we handle the (admittedly rare, but still may
> > happen) epoch overflow situation?
> >
> > 3. I'm a bit concerned about adding a generic "ABORTABLE_ERROR" given
> > we already have a pretty messy error classification and error handling
> > on the producer clients side --- I have a summary about the issues and
> > a proposal to address this in
> >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
> > -- I understand we do not want to use "UNKNOWN_PRODUCER_ID" anymore
> > and in fact we intend to deprecate it in KIP-360 and eventually remove
> > it; but I'm wondering can we still use specific error codes. E.g. what
> > about "InvalidProducerEpochException" since for new clients, the
> > actual reason this would actually be rejected is indeed because the
> > epoch on the coordinator caused the add-partitions-request from the
> > brokers to be rejected anyways?
> >
> > 4. It seems we put the producer request into purgatory before we ever
> > append the records, while other producer's records may still be
> > appended during the time; and that potentially may result in some
> > re-ordering compared with reception order. I'm not super concerned
> > about it since Kafka does not guarantee reception ordering across
> > producers anyways, but it may make the timestamps of records inside a
> > partition to be more out-of-ordered. Are we aware of any scenarios
> > such as future enhancements on log compactions that may be affected by
> > this effect?
> >
> > Below are just minor comments:
> >
> > 5. In "AddPartitionsToTxnTransaction" field of
> > "AddPartitionsToTxnRequest" RPC, the versions of those inner fields
> > are "0-3" while I thought they should be "0+" still?
> >
> > 6. Regarding "we can place the request in a purgatory of sorts and
> > check if there is any state for the transaction on the broker": i
> > think at this time when we just do the checks against the cached
> > state, we do not need to put the request to purgatory yet?
> >
> > 7. This is related to 3) above. I feel using "InvalidRecordException"
> > for older clients may also be a bit confusing, and also it is not
> > fatal -- for old clients, it better to be fatal since this indicates
> > the clients is doing something wrong and hence it should be closed.
> > And in general I'd prefer to use slightly more specific meaning error
> > codes for clients. That being said, I also feel
> > "InvalidProducerEpochException" is not suitable for old versioned
> > clients, and we'd have to pick one that old clients recognize. I'd
> > prefer "InvalidTxnStateException" but that one is supposed to be
> > returned from txn coordinators only today. I'd suggest we do a quick
> > check in the current client's code path and see if that one would be
> > handled if it's from a produce-response, and if yes, use this one;
> > otherwise, use "ProducerFencedException" which is much less meaningful
> > but it's still a fatal error.
> >
> >
> > Thanks,
> > Guozhang
> >
> >
> >
> > On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
> > <jo...@confluent.io.invalid> wrote:
> > >
> > > Yeah -- looks like we already have code to handle bumping the epoch and
> > > when the epoch is Short.MAX_VALUE, we get a new producer ID. Since this
> > is
> > > already the behavior, do we want to change it further?
> > >
> > > Justine
> > >
> > > On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <jo...@confluent.io>
> > wrote:
> > >
> > > > Hey all, just wanted to quickly update and say I've modified the KIP to
> > > > explicitly mention that AddOffsetCommitsToTxnRequest will be replaced
> > by
> > > > a coordinator-side (inter-broker) AddPartitionsToTxn implicit request.
> > This
> > > > mirrors the user partitions and will implicitly add offset partitions
> > to
> > > > transactions when we commit offsets on them. We will deprecate
> > AddOffsetCommitsToTxnRequest
> > > > for new clients.
> > > >
> > > > Also to address Artem's comments --
> > > > I'm a bit unsure if the changes here will change the previous behavior
> > for
> > > > fencing producers. In the case you mention in the first paragraph, are
> > you
> > > > saying we bump the epoch before we try to abort the transaction? I
> > think I
> > > > need to understand the scenarios you mention a bit better.
> > > >
> > > > As for the second part -- I think it makes sense to have some sort of
> > > > "sentinel" epoch to signal epoch is about to overflow (I think we sort
> > of
> > > > have this value in place in some ways) so we can codify it in the KIP.
> > I'll
> > > > look into that and try to update soon.
> > > >
> > > > Thanks,
> > > > Justine.
> > > >
> > > > On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
> > > > <al...@confluent.io.invalid> wrote:
> > > >
> > > >> It's good to know that KIP-588 addressed some of the issues.  Looking
> > at
> > > >> the code, it still looks like there are some cases that would result
> > in
> > > >> fatal error, e.g. PRODUCER_FENCED is issued by the transaction
> > coordinator
> > > >> if epoch doesn't match, and the client treats it as a fatal error
> > (code in
> > > >> TransactionManager request handling).  If we consider, for example,
> > > >> committing a transaction that returns a timeout, but actually
> > succeeds,
> > > >> trying to abort it or re-commit may result in PRODUCER_FENCED error
> > > >> (because of epoch bump).
> > > >>
> > > >> For failed commits, specifically, we need to know the actual outcome,
> > > >> because if we return an error the application may think that the
> > > >> transaction is aborted and redo the work, leading to duplicates.
> > > >>
> > > >> Re: overflowing epoch.  We could either do it on the TC and return
> > both
> > > >> producer id and epoch (e.g. change the protocol), or signal the client
> > > >> that
> > > >> it needs to get a new producer id.  Checking for max epoch could be a
> > > >> reasonable signal, the value to check should probably be present in
> > the
> > > >> KIP
> > > >> as this is effectively a part of the contract.  Also, the TC should
> > > >> probably return an error if the client didn't change producer id after
> > > >> hitting max epoch.
> > > >>
> > > >> -Artem
> > > >>
> > > >>
> > > >> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
> > > >> <jo...@confluent.io.invalid> wrote:
> > > >>
> > > >> > Thanks for the discussion Artem.
> > > >> >
> > > >> > With respect to the handling of fenced producers, we have some
> > behavior
> > > >> > already in place. As of KIP-588:
> > > >> >
> > > >> >
> > > >>
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> > > >> > ,
> > > >> > we handle timeouts more gracefully. The producer can recover.
> > > >> >
> > > >> > Produce requests can also recover from epoch fencing by aborting the
> > > >> > transaction and starting over.
> > > >> >
> > > >> > What other cases were you considering that would cause us to have a
> > > >> fenced
> > > >> > epoch but we'd want to recover?
> > > >> >
> > > >> > The first point about handling epoch overflows is fair. I think
> > there is
> > > >> > some logic we'd need to consider. (ie, if we are one away from the
> > max
> > > >> > epoch, we need to reset the producer ID.) I'm still wondering if
> > there
> > > >> is a
> > > >> > way to direct this from the response, or if everything should be
> > done on
> > > >> > the client side. Let me know if you have any thoughts here.
> > > >> >
> > > >> > Thanks,
> > > >> > Justine
> > > >> >
> > > >> > On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
> > > >> > <al...@confluent.io.invalid> wrote:
> > > >> >
> > > >> > > There are some workflows in the client that are implied by
> > protocol
> > > >> > > changes, e.g.:
> > > >> > >
> > > >> > > - for new clients, epoch changes with every transaction and can
> > > >> overflow,
> > > >> > > in old clients this condition was handled transparently, because
> > epoch
> > > >> > was
> > > >> > > bumped in InitProducerId and it would return a new producer id if
> > > >> epoch
> > > >> > > overflows, the new clients would need to implement some workflow
> > to
> > > >> > refresh
> > > >> > > producer id
> > > >> > > - how to handle fenced producers, for new clients epoch changes
> > with
> > > >> > every
> > > >> > > transaction, so in presence of failures during commits / aborts,
> > the
> > > >> > > producer could get easily fenced, old clients would pretty much
> > would
> > > >> get
> > > >> > > fenced when a new incarnation of the producer was initialized with
> > > >> > > InitProducerId so it's ok to treat as a fatal error, the new
> > clients
> > > >> > would
> > > >> > > need to implement some workflow to handle that error, otherwise
> > they
> > > >> > could
> > > >> > > get fenced by themselves
> > > >> > > - in particular (as a subset of the previous issue), what would
> > the
> > > >> > client
> > > >> > > do if it got a timeout during commit?  commit could've succeeded
> > or
> > > >> > failed
> > > >> > >
> > > >> > > Not sure if this has to be defined in the KIP as implementing
> > those
> > > >> > > probably wouldn't require protocol changes, but we have multiple
> > > >> > > implementations of Kafka clients, so probably would be good to
> > have
> > > >> some
> > > >> > > client implementation guidance.  Could also be done as a separate
> > doc.
> > > >> > >
> > > >> > > -Artem
> > > >> > >
> > > >> > > On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
> > > >> > <jolshan@confluent.io.invalid
> > > >> > > >
> > > >> > > wrote:
> > > >> > >
> > > >> > > > Hey all, I've updated the KIP to incorporate Jason's
> > suggestions.
> > > >> > > >
> > > >> > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > >> > > >
> > > >> > > >
> > > >> > > > 1. Use AddPartitionsToTxn + verify flag to check on old clients
> > > >> > > > 2. Updated AddPartitionsToTxn API to support transaction
> > batching
> > > >> > > > 3. Mention IBP bump
> > > >> > > > 4. Mention auth change on new AddPartitionsToTxn version.
> > > >> > > >
> > > >> > > > I'm planning on opening a vote soon.
> > > >> > > > Thanks,
> > > >> > > > Justine
> > > >> > > >
> > > >> > > > On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <
> > jolshan@confluent.io
> > > >> >
> > > >> > > > wrote:
> > > >> > > >
> > > >> > > > > Thanks Jason. Those changes make sense to me. I will update
> > the
> > > >> KIP.
> > > >> > > > >
> > > >> > > > >
> > > >> > > > >
> > > >> > > > > On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
> > > >> > > > <ja...@confluent.io.invalid>
> > > >> > > > > wrote:
> > > >> > > > >
> > > >> > > > >> Hey Justine,
> > > >> > > > >>
> > > >> > > > >> > I was wondering about compatibility here. When we send
> > requests
> > > >> > > > >> between brokers, we want to ensure that the receiving broker
> > > >> > > understands
> > > >> > > > >> the request (specifically the new fields). Typically this is
> > done
> > > >> > via
> > > >> > > > >> IBP/metadata version.
> > > >> > > > >> I'm trying to think if there is a way around it but I'm not
> > sure
> > > >> > there
> > > >> > > > is.
> > > >> > > > >>
> > > >> > > > >> Yes. I think we would gate usage of this behind an IBP bump.
> > Does
> > > >> > that
> > > >> > > > >> seem
> > > >> > > > >> reasonable?
> > > >> > > > >>
> > > >> > > > >> > As for the improvements -- can you clarify how the multiple
> > > >> > > > >> transactional
> > > >> > > > >> IDs would help here? Were you thinking of a case where we
> > > >> wait/batch
> > > >> > > > >> multiple produce requests together? My understanding for now
> > was
> > > >> 1
> > > >> > > > >> transactional ID and one validation per 1 produce request.
> > > >> > > > >>
> > > >> > > > >> Each call to `AddPartitionsToTxn` is essentially a write to
> > the
> > > >> > > > >> transaction
> > > >> > > > >> log and must block on replication. The more we can fit into a
> > > >> single
> > > >> > > > >> request, the more writes we can do in parallel. The
> > alternative
> > > >> is
> > > >> > to
> > > >> > > > make
> > > >> > > > >> use of more connections, but usually we prefer batching
> > since the
> > > >> > > > network
> > > >> > > > >> stack is not really optimized for high connection/request
> > loads.
> > > >> > > > >>
> > > >> > > > >> > Finally with respect to the authorizations, I think it
> > makes
> > > >> sense
> > > >> > > to
> > > >> > > > >> skip
> > > >> > > > >> topic authorizations, but I'm a bit confused by the "leader
> > ID"
> > > >> > field.
> > > >> > > > >> Wouldn't we just want to flag the request as from a broker
> > (does
> > > >> it
> > > >> > > > matter
> > > >> > > > >> which one?).
> > > >> > > > >>
> > > >> > > > >> We could also make it version-based. For the next version, we
> > > >> could
> > > >> > > > >> require
> > > >> > > > >> CLUSTER auth. So clients would not be able to use the API
> > > >> anymore,
> > > >> > > which
> > > >> > > > >> is
> > > >> > > > >> probably what we want.
> > > >> > > > >>
> > > >> > > > >> -Jason
> > > >> > > > >>
> > > >> > > > >> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> > > >> > > > >> <jo...@confluent.io.invalid>
> > > >> > > > >> wrote:
> > > >> > > > >>
> > > >> > > > >> > As a follow up, I was just thinking about the batching a
> > bit
> > > >> more.
> > > >> > > > >> > I suppose if we have one request in flight and we queue up
> > the
> > > >> > other
> > > >> > > > >> > produce requests in some sort of purgatory, we could send
> > > >> > > information
> > > >> > > > >> out
> > > >> > > > >> > for all of them rather than one by one. So that would be a
> > > >> benefit
> > > >> > > of
> > > >> > > > >> > batching partitions to add per transaction.
> > > >> > > > >> >
> > > >> > > > >> > I'll need to think a bit more on the design of this part
> > of the
> > > >> > KIP,
> > > >> > > > and
> > > >> > > > >> > will update the KIP in the next few days.
> > > >> > > > >> >
> > > >> > > > >> > Thanks,
> > > >> > > > >> > Justine
> > > >> > > > >> >
> > > >> > > > >> > On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan <
> > > >> > > jolshan@confluent.io>
> > > >> > > > >> > wrote:
> > > >> > > > >> >
> > > >> > > > >> > > Hey Jason -- thanks for the input -- I was just digging
> > a bit
> > > >> > > deeper
> > > >> > > > >> into
> > > >> > > > >> > > the design + implementation of the validation calls here
> > and
> > > >> > what
> > > >> > > > you
> > > >> > > > >> say
> > > >> > > > >> > > makes sense.
> > > >> > > > >> > >
> > > >> > > > >> > > I was wondering about compatibility here. When we send
> > > >> requests
> > > >> > > > >> > > between brokers, we want to ensure that the receiving
> > broker
> > > >> > > > >> understands
> > > >> > > > >> > > the request (specifically the new fields). Typically
> > this is
> > > >> > done
> > > >> > > > via
> > > >> > > > >> > > IBP/metadata version.
> > > >> > > > >> > > I'm trying to think if there is a way around it but I'm
> > not
> > > >> sure
> > > >> > > > there
> > > >> > > > >> > is.
> > > >> > > > >> > >
> > > >> > > > >> > > As for the improvements -- can you clarify how the
> > multiple
> > > >> > > > >> transactional
> > > >> > > > >> > > IDs would help here? Were you thinking of a case where we
> > > >> > > wait/batch
> > > >> > > > >> > > multiple produce requests together? My understanding for
> > now
> > > >> > was 1
> > > >> > > > >> > > transactional ID and one validation per 1 produce
> > request.
> > > >> > > > >> > >
> > > >> > > > >> > > Finally with respect to the authorizations, I think it
> > makes
> > > >> > sense
> > > >> > > > to
> > > >> > > > >> > skip
> > > >> > > > >> > > topic authorizations, but I'm a bit confused by the
> > "leader
> > > >> ID"
> > > >> > > > field.
> > > >> > > > >> > > Wouldn't we just want to flag the request as from a
> > broker
> > > >> (does
> > > >> > > it
> > > >> > > > >> > matter
> > > >> > > > >> > > which one?).
> > > >> > > > >> > >
> > > >> > > > >> > > I think I want to adopt these suggestions, just had a few
> > > >> > > questions
> > > >> > > > on
> > > >> > > > >> > the
> > > >> > > > >> > > details.
> > > >> > > > >> > >
> > > >> > > > >> > > Thanks,
> > > >> > > > >> > > Justine
> > > >> > > > >> > >
> > > >> > > > >> > > On Thu, Jan 5, 2023 at 5:05 PM Jason Gustafson
> > > >> > > > >> > <ja...@confluent.io.invalid>
> > > >> > > > >> > > wrote:
> > > >> > > > >> > >
> > > >> > > > >> > >> Hi Justine,
> > > >> > > > >> > >>
> > > >> > > > >> > >> Thanks for the proposal.
> > > >> > > > >> > >>
> > > >> > > > >> > >> I was thinking about the implementation a little bit.
> > In the
> > > >> > > > current
> > > >> > > > >> > >> proposal, the behavior depends on whether we have an
> > old or
> > > >> new
> > > >> > > > >> client.
> > > >> > > > >> > >> For
> > > >> > > > >> > >> old clients, we send `DescribeTransactions` and verify
> > the
> > > >> > result
> > > >> > > > and
> > > >> > > > >> > for
> > > >> > > > >> > >> new clients, we send `AddPartitionsToTxn`. We might be
> > able
> > > >> to
> > > >> > > > >> simplify
> > > >> > > > >> > >> the
> > > >> > > > >> > >> implementation if we can use the same request type. For
> > > >> > example,
> > > >> > > > >> what if
> > > >> > > > >> > >> we
> > > >> > > > >> > >> bump the protocol version for `AddPartitionsToTxn` and
> > add a
> > > >> > > > >> > >> `validateOnly`
> > > >> > > > >> > >> flag? For older versions, we can set
> > `validateOnly=true` so
> > > >> > that
> > > >> > > > the
> > > >> > > > >> > >> request only returns successfully if the partition had
> > > >> already
> > > >> > > been
> > > >> > > > >> > added.
> > > >> > > > >> > >> For new versions, we can set `validateOnly=false` and
> > the
> > > >> > > partition
> > > >> > > > >> will
> > > >> > > > >> > >> be
> > > >> > > > >> > >> added to the transaction. The other slightly annoying
> > thing
> > > >> > that
> > > >> > > > this
> > > >> > > > >> > >> would
> > > >> > > > >> > >> get around is the need to collect the transaction state
> > for
> > > >> all
> > > >> > > > >> > partitions
> > > >> > > > >> > >> even when we only care about a subset.
> > > >> > > > >> > >>
> > > >> > > > >> > >> Some additional improvements to consider:
> > > >> > > > >> > >>
> > > >> > > > >> > >> - We can give `AddPartitionsToTxn` better batch support
> > for
> > > >> > > > >> inter-broker
> > > >> > > > >> > >> usage. Currently we only allow one `TransactionalId` to
> > be
> > > >> > > > specified,
> > > >> > > > >> > but
> > > >> > > > >> > >> the broker may get some benefit being able to batch
> > across
> > > >> > > multiple
> > > >> > > > >> > >> transactions.
> > > >> > > > >> > >> - Another small improvement is skipping topic
> > authorization
> > > >> > > checks
> > > >> > > > >> for
> > > >> > > > >> > >> `AddPartitionsToTxn` when the request is from a broker.
> > > >> Perhaps
> > > >> > > we
> > > >> > > > >> can
> > > >> > > > >> > add
> > > >> > > > >> > >> a field for the `LeaderId` or something like that and
> > > >> require
> > > >> > > > CLUSTER
> > > >> > > > >> > >> permission when set.
> > > >> > > > >> > >>
> > > >> > > > >> > >> Best,
> > > >> > > > >> > >> Jason
> > > >> > > > >> > >>
> > > >> > > > >> > >>
> > > >> > > > >> > >>
> > > >> > > > >> > >> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
> > > >> > <jun@confluent.io.invalid
> > > >> > > >
> > > >> > > > >> > wrote:
> > > >> > > > >> > >>
> > > >> > > > >> > >> > Hi, Justine,
> > > >> > > > >> > >> >
> > > >> > > > >> > >> > Thanks for the explanation. It makes sense to me now.
> > > >> > > > >> > >> >
> > > >> > > > >> > >> > Jun
> > > >> > > > >> > >> >
> > > >> > > > >> > >> > On Mon, Dec 19, 2022 at 1:42 PM Justine Olshan
> > > >> > > > >> > >> > <jo...@confluent.io.invalid>
> > > >> > > > >> > >> > wrote:
> > > >> > > > >> > >> >
> > > >> > > > >> > >> > > Hi Jun,
> > > >> > > > >> > >> > >
> > > >> > > > >> > >> > > My understanding of the mechanism is that when we
> > get to
> > > >> > the
> > > >> > > > last
> > > >> > > > >> > >> epoch,
> > > >> > > > >> > >> > we
> > > >> > > > >> > >> > > increment to the fencing/last epoch and if any
> > further
> > > >> > > requests
> > > >> > > > >> come
> > > >> > > > >> > >> in
> > > >> > > > >> > >> > for
> > > >> > > > >> > >> > > this producer ID they are fenced. Then the producer
> > > >> gets a
> > > >> > > new
> > > >> > > > ID
> > > >> > > > >> > and
> > > >> > > > >> > >> > > restarts with epoch/sequence 0. The fenced epoch
> > sticks
> > > >> > > around
> > > >> > > > >> for
> > > >> > > > >> > the
> > > >> > > > >> > >> > > duration of producer.id.expiration.ms and blocks
> > any
> > > >> late
> > > >> > > > >> messages
> > > >> > > > >> > >> > there.
> > > >> > > > >> > >> > > The new ID will get to take advantage of the
> > improved
> > > >> > > semantics
> > > >> > > > >> > around
> > > >> > > > >> > >> > > non-zero start sequences. So I think we are covered.
> > > >> > > > >> > >> > >
> > > >> > > > >> > >> > > The only potential issue is overloading the cache,
> > but
> > > >> > > > hopefully
> > > >> > > > >> the
> > > >> > > > >> > >> > > improvements (lowered producer.id.expiration.ms)
> > will
> > > >> help
> > > >> > > > with
> > > >> > > > >> > that.
> > > >> > > > >> > >> > Let
> > > >> > > > >> > >> > > me know if you still have concerns.
> > > >> > > > >> > >> > >
> > > >> > > > >> > >> > > Thanks,
> > > >> > > > >> > >> > > Justine
> > > >> > > > >> > >> > >
> > > >> > > > >> > >> > > On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
> > > >> > > > >> <ju...@confluent.io.invalid>
> > > >> > > > >> > >> > wrote:
> > > >> > > > >> > >> > >
> > > >> > > > >> > >> > > > Hi, Justine,
> > > >> > > > >> > >> > > >
> > > >> > > > >> > >> > > > Thanks for the explanation.
> > > >> > > > >> > >> > > >
> > > >> > > > >> > >> > > > 70. The proposed fencing logic doesn't apply when
> > pid
> > > >> > > > changes,
> > > >> > > > >> is
> > > >> > > > >> > >> that
> > > >> > > > >> > >> > > > right? If so, I am not sure how complete we are
> > > >> > addressing
> > > >> > > > this
> > > >> > > > >> > >> issue
> > > >> > > > >> > >> > if
> > > >> > > > >> > >> > > > the pid changes more frequently.
> > > >> > > > >> > >> > > >
> > > >> > > > >> > >> > > > Thanks,
> > > >> > > > >> > >> > > >
> > > >> > > > >> > >> > > > Jun
> > > >> > > > >> > >> > > >
> > > >> > > > >> > >> > > >
> > > >> > > > >> > >> > > >
> > > >> > > > >> > >> > > > On Fri, Dec 16, 2022 at 9:16 AM Justine Olshan
> > > >> > > > >> > >> > > > <jo...@confluent.io.invalid>
> > > >> > > > >> > >> > > > wrote:
> > > >> > > > >> > >> > > >
> > > >> > > > >> > >> > > > > Hi Jun,
> > > >> > > > >> > >> > > > >
> > > >> > > > >> > >> > > > > Thanks for replying!
> > > >> > > > >> > >> > > > >
> > > >> > > > >> > >> > > > > 70.We already do the overflow mechanism, so my
> > > >> change
> > > >> > > would
> > > >> > > > >> just
> > > >> > > > >> > >> make
> > > >> > > > >> > >> > > it
> > > >> > > > >> > >> > > > > happen more often.
> > > >> > > > >> > >> > > > > I was also not suggesting a new field in the
> > log,
> > > >> but
> > > >> > in
> > > >> > > > the
> > > >> > > > >> > >> > response,
> > > >> > > > >> > >> > > > > which would be gated by the client version.
> > Sorry if
> > > >> > > > >> something
> > > >> > > > >> > >> there
> > > >> > > > >> > >> > is
> > > >> > > > >> > >> > > > > unclear. I think we are starting to diverge.
> > > >> > > > >> > >> > > > > The goal of this KIP is to not change to the
> > marker
> > > >> > > format
> > > >> > > > at
> > > >> > > > >> > all.
> > > >> > > > >> > >> > > > >
> > > >> > > > >> > >> > > > > 71. Yes, I guess I was going under the
> > assumption
> > > >> that
> > > >> > > the
> > > >> > > > >> log
> > > >> > > > >> > >> would
> > > >> > > > >> > >> > > just
> > > >> > > > >> > >> > > > > look at its last epoch and treat it as the
> > current
> > > >> > > epoch. I
> > > >> > > > >> > >> suppose
> > > >> > > > >> > >> > we
> > > >> > > > >> > >> > > > can
> > > >> > > > >> > >> > > > > have some special logic that if the last epoch
> > was
> > > >> on a
> > > >> > > > >> marker
> > > >> > > > >> > we
> > > >> > > > >> > >> > > > actually
> > > >> > > > >> > >> > > > > expect the next epoch or something like that. We
> > > >> just
> > > >> > > need
> > > >> > > > to
> > > >> > > > >> > >> > > distinguish
> > > >> > > > >> > >> > > > > based on whether we had a commit/abort marker.
> > > >> > > > >> > >> > > > >
> > > >> > > > >> > >> > > > > 72.
> > > >> > > > >> > >> > > > > > if the producer epoch hasn't been bumped on
> > the
> > > >> > > > >> > >> > > > > broker, it seems that the stucked message will
> > fail
> > > >> the
> > > >> > > > >> sequence
> > > >> > > > >> > >> > > > validation
> > > >> > > > >> > >> > > > > and will be ignored. If the producer epoch has
> > been
> > > >> > > bumped,
> > > >> > > > >> we
> > > >> > > > >> > >> ignore
> > > >> > > > >> > >> > > the
> > > >> > > > >> > >> > > > > sequence check and the stuck message could be
> > > >> appended
> > > >> > to
> > > >> > > > the
> > > >> > > > >> > log.
> > > >> > > > >> > >> > So,
> > > >> > > > >> > >> > > is
> > > >> > > > >> > >> > > > > the latter case that we want to guard?
> > > >> > > > >> > >> > > > >
> > > >> > > > >> > >> > > > > I'm not sure I follow that "the message will
> > fail
> > > >> the
> > > >> > > > >> sequence
> > > >> > > > >> > >> > > > validation".
> > > >> > > > >> > >> > > > > In some of these cases, we had an abort marker
> > (due
> > > >> to
> > > >> > an
> > > >> > > > >> error)
> > > >> > > > >> > >> and
> > > >> > > > >> > >> > > then
> > > >> > > > >> > >> > > > > the late message comes in with the correct
> > sequence
> > > >> > > number.
> > > >> > > > >> This
> > > >> > > > >> > >> is a
> > > >> > > > >> > >> > > > case
> > > >> > > > >> > >> > > > > covered by the KIP.
> > > >> > > > >> > >> > > > > The latter case is actually not something we've
> > > >> > > considered
> > > >> > > > >> > here. I
> > > >> > > > >> > >> > > think
> > > >> > > > >> > >> > > > > generally when we bump the epoch, we are
> > accepting
> > > >> that
> > > >> > > the
> > > >> > > > >> > >> sequence
> > > >> > > > >> > >> > > does
> > > >> > > > >> > >> > > > > not need to be checked anymore. My
> > understanding is
> > > >> > also
> > > >> > > > >> that we
> > > >> > > > >> > >> > don't
> > > >> > > > >> > >> > > > > typically bump epoch mid transaction (based on a
> > > >> quick
> > > >> > > look
> > > >> > > > >> at
> > > >> > > > >> > the
> > > >> > > > >> > >> > > code)
> > > >> > > > >> > >> > > > > but let me know if that is the case.
> > > >> > > > >> > >> > > > >
> > > >> > > > >> > >> > > > > Thanks,
> > > >> > > > >> > >> > > > > Justine
> > > >> > > > >> > >> > > > >
> > > >> > > > >> > >> > > > > On Thu, Dec 15, 2022 at 12:23 PM Jun Rao
> > > >> > > > >> > <jun@confluent.io.invalid
> > > >> > > > >> > >> >
> > > >> > > > >> > >> > > > wrote:
> > > >> > > > >> > >> > > > >
> > > >> > > > >> > >> > > > > > Hi, Justine,
> > > >> > > > >> > >> > > > > >
> > > >> > > > >> > >> > > > > > Thanks for the reply.
> > > >> > > > >> > >> > > > > >
> > > >> > > > >> > >> > > > > > 70. Assigning a new pid on int overflow seems
> > a
> > > >> bit
> > > >> > > > hacky.
> > > >> > > > >> If
> > > >> > > > >> > we
> > > >> > > > >> > >> > > need a
> > > >> > > > >> > >> > > > > txn
> > > >> > > > >> > >> > > > > > level id, it will be better to model this
> > > >> explicitly.
> > > >> > > > >> Adding a
> > > >> > > > >> > >> new
> > > >> > > > >> > >> > > > field
> > > >> > > > >> > >> > > > > > would require a bit more work since it
> > requires a
> > > >> new
> > > >> > > txn
> > > >> > > > >> > marker
> > > >> > > > >> > >> > > format
> > > >> > > > >> > >> > > > > in
> > > >> > > > >> > >> > > > > > the log. So, we probably need to guard it
> > with an
> > > >> IBP
> > > >> > > or
> > > >> > > > >> > >> metadata
> > > >> > > > >> > >> > > > version
> > > >> > > > >> > >> > > > > > and document the impact on downgrade once the
> > new
> > > >> > > format
> > > >> > > > is
> > > >> > > > >> > >> written
> > > >> > > > >> > >> > > to
> > > >> > > > >> > >> > > > > the
> > > >> > > > >> > >> > > > > > log.
> > > >> > > > >> > >> > > > > >
> > > >> > > > >> > >> > > > > > 71. Hmm, once the marker is written, the
> > partition
> > > >> > will
> > > >> > > > >> expect
> > > >> > > > >> > >> the
> > > >> > > > >> > >> > > next
> > > >> > > > >> > >> > > > > > append to be on the next epoch. Does that
> > cover
> > > >> the
> > > >> > > case
> > > >> > > > >> you
> > > >> > > > >> > >> > > mentioned?
> > > >> > > > >> > >> > > > > >
> > > >> > > > >> > >> > > > > > 72. Also, just to be clear on the stucked
> > message
> > > >> > issue
> > > >> > > > >> > >> described
> > > >> > > > >> > >> > in
> > > >> > > > >> > >> > > > the
> > > >> > > > >> > >> > > > > > motivation. With EoS, we also validate the
> > > >> sequence
> > > >> > id
> > > >> > > > for
> > > >> > > > >> > >> > > idempotency.
> > > >> > > > >> > >> > > > > So,
> > > >> > > > >> > >> > > > > > with the current logic, if the producer epoch
> > > >> hasn't
> > > >> > > been
> > > >> > > > >> > >> bumped on
> > > >> > > > >> > >> > > the
> > > >> > > > >> > >> > > > > > broker, it seems that the stucked message will
> > > >> fail
> > > >> > the
> > > >> > > > >> > sequence
> > > >> > > > >> > >> > > > > validation
> > > >> > > > >> > >> > > > > > and will be ignored. If the producer epoch has
> > > >> been
> > > >> > > > >> bumped, we
> > > >> > > > >> > >> > ignore
> > > >> > > > >> > >> > > > the
> > > >> > > > >> > >> > > > > > sequence check and the stuck message could be
> > > >> > appended
> > > >> > > to
> > > >> > > > >> the
> > > >> > > > >> > >> log.
> > > >> > > > >> > >> > > So,
> > > >> > > > >> > >> > > > is
> > > >> > > > >> > >> > > > > > the latter case that we want to guard?
> > > >> > > > >> > >> > > > > >
> > > >> > > > >> > >> > > > > > Thanks,
> > > >> > > > >> > >> > > > > >
> > > >> > > > >> > >> > > > > > Jun
> > > >> > > > >> > >> > > > > >
> > > >> > > > >> > >> > > > > > On Wed, Dec 14, 2022 at 10:44 AM Justine
> > Olshan
> > > >> > > > >> > >> > > > > > <jo...@confluent.io.invalid> wrote:
> > > >> > > > >> > >> > > > > >
> > > >> > > > >> > >> > > > > > > Matthias — thanks again for taking time to
> > look
> > > >> a
> > > >> > > this.
> > > >> > > > >> You
> > > >> > > > >> > >> said:
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > > > > My proposal was only focusing to avoid
> > > >> dangling
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > > > transactions if records are added without
> > > >> > registered
> > > >> > > > >> > >> partition.
> > > >> > > > >> > >> > --
> > > >> > > > >> > >> > > > > Maybe
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > > > you can add a few more details to the KIP
> > about
> > > >> > this
> > > >> > > > >> > scenario
> > > >> > > > >> > >> for
> > > >> > > > >> > >> > > > > better
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > > > documentation purpose?
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > > > I'm not sure I understand what you mean
> > here.
> > > >> The
> > > >> > > > >> motivation
> > > >> > > > >> > >> > > section
> > > >> > > > >> > >> > > > > > > describes two scenarios about how the record
> > > >> can be
> > > >> > > > added
> > > >> > > > >> > >> > without a
> > > >> > > > >> > >> > > > > > > registered partition:
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > > > > 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.
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > > > > Another way hanging transactions can
> > occur is
> > > >> > that
> > > >> > > a
> > > >> > > > >> > client
> > > >> > > > >> > >> is
> > > >> > > > >> > >> > > > buggy
> > > >> > > > >> > >> > > > > > and
> > > >> > > > >> > >> > > > > > > may somehow try to write to a partition
> > before
> > > >> it
> > > >> > > adds
> > > >> > > > >> the
> > > >> > > > >> > >> > > partition
> > > >> > > > >> > >> > > > to
> > > >> > > > >> > >> > > > > > the
> > > >> > > > >> > >> > > > > > > transaction.
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > > > For the first example of this would it be
> > > >> helpful
> > > >> > to
> > > >> > > > say
> > > >> > > > >> > that
> > > >> > > > >> > >> > this
> > > >> > > > >> > >> > > > > > message
> > > >> > > > >> > >> > > > > > > comes in after the abort, but before the
> > > >> partition
> > > >> > is
> > > >> > > > >> added
> > > >> > > > >> > to
> > > >> > > > >> > >> > the
> > > >> > > > >> > >> > > > next
> > > >> > > > >> > >> > > > > > > transaction so it becomes "hanging."
> > Perhaps the
> > > >> > next
> > > >> > > > >> > sentence
> > > >> > > > >> > >> > > > > describing
> > > >> > > > >> > >> > > > > > > the message becoming part of the next
> > > >> transaction
> > > >> > (a
> > > >> > > > >> > different
> > > >> > > > >> > >> > > case)
> > > >> > > > >> > >> > > > > was
> > > >> > > > >> > >> > > > > > > not properly differentiated.
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > > > Jun — thanks for reading the KIP.
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > > > 70. The int typing was a concern. Currently
> > we
> > > >> > have a
> > > >> > > > >> > >> mechanism
> > > >> > > > >> > >> > in
> > > >> > > > >> > >> > > > > place
> > > >> > > > >> > >> > > > > > to
> > > >> > > > >> > >> > > > > > > fence the final epoch when the epoch is
> > about to
> > > >> > > > overflow
> > > >> > > > >> > and
> > > >> > > > >> > >> > > assign
> > > >> > > > >> > >> > > > a
> > > >> > > > >> > >> > > > > > new
> > > >> > > > >> > >> > > > > > > producer ID with epoch 0. Of course, this
> > is a
> > > >> bit
> > > >> > > > tricky
> > > >> > > > >> > >> when it
> > > >> > > > >> > >> > > > comes
> > > >> > > > >> > >> > > > > > to
> > > >> > > > >> > >> > > > > > > the response back to the client.
> > > >> > > > >> > >> > > > > > > Making this a long could be another option,
> > but
> > > >> I
> > > >> > > > wonder
> > > >> > > > >> are
> > > >> > > > >> > >> > there
> > > >> > > > >> > >> > > > any
> > > >> > > > >> > >> > > > > > > implications on changing this field if the
> > > >> epoch is
> > > >> > > > >> > persisted
> > > >> > > > >> > >> to
> > > >> > > > >> > >> > > > disk?
> > > >> > > > >> > >> > > > > > I'd
> > > >> > > > >> > >> > > > > > > need to check the usages.
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > > > 71.This was something Matthias asked about
> > as
> > > >> > well. I
> > > >> > > > was
> > > >> > > > >> > >> > > > considering a
> > > >> > > > >> > >> > > > > > > possible edge case where a produce request
> > from
> > > >> a
> > > >> > new
> > > >> > > > >> > >> transaction
> > > >> > > > >> > >> > > > > somehow
> > > >> > > > >> > >> > > > > > > gets sent right after the marker is
> > written, but
> > > >> > > before
> > > >> > > > >> the
> > > >> > > > >> > >> > > producer
> > > >> > > > >> > >> > > > is
> > > >> > > > >> > >> > > > > > > alerted of the newly bumped epoch. In this
> > > >> case, we
> > > >> > > may
> > > >> > > > >> > >> include
> > > >> > > > >> > >> > > this
> > > >> > > > >> > >> > > > > > record
> > > >> > > > >> > >> > > > > > > when we don't want to. I suppose we could
> > try
> > > >> to do
> > > >> > > > >> > something
> > > >> > > > >> > >> > > client
> > > >> > > > >> > >> > > > > side
> > > >> > > > >> > >> > > > > > > to bump the epoch after sending an endTxn as
> > > >> well
> > > >> > in
> > > >> > > > this
> > > >> > > > >> > >> > scenario
> > > >> > > > >> > >> > > —
> > > >> > > > >> > >> > > > > but
> > > >> > > > >> > >> > > > > > I
> > > >> > > > >> > >> > > > > > > wonder how it would work when the server is
> > > >> > aborting
> > > >> > > > >> based
> > > >> > > > >> > on
> > > >> > > > >> > >> a
> > > >> > > > >> > >> > > > > > server-side
> > > >> > > > >> > >> > > > > > > error. I could also be missing something and
> > > >> this
> > > >> > > > >> scenario
> > > >> > > > >> > is
> > > >> > > > >> > >> > > > actually
> > > >> > > > >> > >> > > > > > not
> > > >> > > > >> > >> > > > > > > possible.
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > > > Thanks again to everyone reading and
> > commenting.
> > > >> > Let
> > > >> > > me
> > > >> > > > >> know
> > > >> > > > >> > >> > about
> > > >> > > > >> > >> > > > any
> > > >> > > > >> > >> > > > > > > further questions or comments.
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > > > Justine
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > > > On Wed, Dec 14, 2022 at 9:41 AM Jun Rao
> > > >> > > > >> > >> <jun@confluent.io.invalid
> > > >> > > > >> > >> > >
> > > >> > > > >> > >> > > > > > wrote:
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > > > > Hi, Justine,
> > > >> > > > >> > >> > > > > > > >
> > > >> > > > >> > >> > > > > > > > Thanks for the KIP. A couple of comments.
> > > >> > > > >> > >> > > > > > > >
> > > >> > > > >> > >> > > > > > > > 70. Currently, the producer epoch is an
> > int.
> > > >> I am
> > > >> > > not
> > > >> > > > >> sure
> > > >> > > > >> > >> if
> > > >> > > > >> > >> > > it's
> > > >> > > > >> > >> > > > > > enough
> > > >> > > > >> > >> > > > > > > > to accommodate all transactions in the
> > > >> lifetime
> > > >> > of
> > > >> > > a
> > > >> > > > >> > >> producer.
> > > >> > > > >> > >> > > > Should
> > > >> > > > >> > >> > > > > > we
> > > >> > > > >> > >> > > > > > > > change that to a long or add a new long
> > field
> > > >> > like
> > > >> > > > >> txnId?
> > > >> > > > >> > >> > > > > > > >
> > > >> > > > >> > >> > > > > > > > 71. "it will write the prepare commit
> > message
> > > >> > with
> > > >> > > a
> > > >> > > > >> > bumped
> > > >> > > > >> > >> > epoch
> > > >> > > > >> > >> > > > and
> > > >> > > > >> > >> > > > > > > send
> > > >> > > > >> > >> > > > > > > > WriteTxnMarkerRequests with the bumped
> > epoch."
> > > >> > Hmm,
> > > >> > > > the
> > > >> > > > >> > >> epoch
> > > >> > > > >> > >> > is
> > > >> > > > >> > >> > > > > > > associated
> > > >> > > > >> > >> > > > > > > > with the current txn right? So, it seems
> > > >> weird to
> > > >> > > > >> write a
> > > >> > > > >> > >> > commit
> > > >> > > > >> > >> > > > > > message
> > > >> > > > >> > >> > > > > > > > with a bumped epoch. Should we only bump
> > up
> > > >> the
> > > >> > > epoch
> > > >> > > > >> in
> > > >> > > > >> > >> > > > > EndTxnResponse
> > > >> > > > >> > >> > > > > > > and
> > > >> > > > >> > >> > > > > > > > rename the field to sth like
> > > >> nextProducerEpoch?
> > > >> > > > >> > >> > > > > > > >
> > > >> > > > >> > >> > > > > > > > Thanks,
> > > >> > > > >> > >> > > > > > > >
> > > >> > > > >> > >> > > > > > > > Jun
> > > >> > > > >> > >> > > > > > > >
> > > >> > > > >> > >> > > > > > > >
> > > >> > > > >> > >> > > > > > > >
> > > >> > > > >> > >> > > > > > > > On Mon, Dec 12, 2022 at 8:54 PM Matthias
> > J.
> > > >> Sax <
> > > >> > > > >> > >> > > mjsax@apache.org>
> > > >> > > > >> > >> > > > > > > wrote:
> > > >> > > > >> > >> > > > > > > >
> > > >> > > > >> > >> > > > > > > > > Thanks for the background.
> > > >> > > > >> > >> > > > > > > > >
> > > >> > > > >> > >> > > > > > > > > 20/30: SGTM. My proposal was only
> > focusing
> > > >> to
> > > >> > > avoid
> > > >> > > > >> > >> dangling
> > > >> > > > >> > >> > > > > > > > > transactions if records are added
> > without
> > > >> > > > registered
> > > >> > > > >> > >> > partition.
> > > >> > > > >> > >> > > > --
> > > >> > > > >> > >> > > > > > > Maybe
> > > >> > > > >> > >> > > > > > > > > you can add a few more details to the
> > KIP
> > > >> about
> > > >> > > > this
> > > >> > > > >> > >> scenario
> > > >> > > > >> > >> > > for
> > > >> > > > >> > >> > > > > > > better
> > > >> > > > >> > >> > > > > > > > > documentation purpose?
> > > >> > > > >> > >> > > > > > > > >
> > > >> > > > >> > >> > > > > > > > > 40: I think you hit a fair point about
> > race
> > > >> > > > >> conditions
> > > >> > > > >> > or
> > > >> > > > >> > >> > > client
> > > >> > > > >> > >> > > > > bugs
> > > >> > > > >> > >> > > > > > > > > (incorrectly not bumping the epoch). The
> > > >> > > > >> > >> complexity/confusion
> > > >> > > > >> > >> > > for
> > > >> > > > >> > >> > > > > > using
> > > >> > > > >> > >> > > > > > > > > the bumped epoch I see, is mainly for
> > > >> internal
> > > >> > > > >> > debugging,
> > > >> > > > >> > >> ie,
> > > >> > > > >> > >> > > > > > > inspecting
> > > >> > > > >> > >> > > > > > > > > log segment dumps -- it seems harder to
> > > >> reason
> > > >> > > > about
> > > >> > > > >> the
> > > >> > > > >> > >> > system
> > > >> > > > >> > >> > > > for
> > > >> > > > >> > >> > > > > > us
> > > >> > > > >> > >> > > > > > > > > humans. But if we get better
> > guarantees, it
> > > >> > would
> > > >> > > > be
> > > >> > > > >> > >> worth to
> > > >> > > > >> > >> > > use
> > > >> > > > >> > >> > > > > the
> > > >> > > > >> > >> > > > > > > > > bumped epoch.
> > > >> > > > >> > >> > > > > > > > >
> > > >> > > > >> > >> > > > > > > > > 60: as I mentioned already, I don't
> > know the
> > > >> > > broker
> > > >> > > > >> > >> internals
> > > >> > > > >> > >> > > to
> > > >> > > > >> > >> > > > > > > provide
> > > >> > > > >> > >> > > > > > > > > more input. So if nobody else chimes
> > in, we
> > > >> > > should
> > > >> > > > >> just
> > > >> > > > >> > >> move
> > > >> > > > >> > >> > > > > forward
> > > >> > > > >> > >> > > > > > > > > with your proposal.
> > > >> > > > >> > >> > > > > > > > >
> > > >> > > > >> > >> > > > > > > > >
> > > >> > > > >> > >> > > > > > > > > -Matthias
> > > >> > > > >> > >> > > > > > > > >
> > > >> > > > >> > >> > > > > > > > >
> > > >> > > > >> > >> > > > > > > > > On 12/6/22 4:22 PM, Justine Olshan
> > wrote:
> > > >> > > > >> > >> > > > > > > > > > Hi all,
> > > >> > > > >> > >> > > > > > > > > > After Artem's questions about error
> > > >> behavior,
> > > >> > > > I've
> > > >> > > > >> > >> > > re-evaluated
> > > >> > > > >> > >> > > > > the
> > > >> > > > >> > >> > > > > > > > > > unknown producer ID exception and had
> > some
> > > >> > > > >> discussions
> > > >> > > > >> > >> > > offline.
> > > >> > > > >> > >> > > > > > > > > >
> > > >> > > > >> > >> > > > > > > > > > I think generally it makes sense to
> > > >> simplify
> > > >> > > > error
> > > >> > > > >> > >> handling
> > > >> > > > >> > >> > > in
> > > >> > > > >> > >> > > > > > cases
> > > >> > > > >> > >> > > > > > > > like
> > > >> > > > >> > >> > > > > > > > > > this and the UNKNOWN_PRODUCER_ID error
> > > >> has a
> > > >> > > > pretty
> > > >> > > > >> > long
> > > >> > > > >> > >> > and
> > > >> > > > >> > >> > > > > > > > complicated
> > > >> > > > >> > >> > > > > > > > > > history. Because of this, I propose
> > > >> adding a
> > > >> > > new
> > > >> > > > >> error
> > > >> > > > >> > >> code
> > > >> > > > >> > >> > > > > > > > > ABORTABLE_ERROR
> > > >> > > > >> > >> > > > > > > > > > that when encountered by new clients
> > > >> (gated
> > > >> > by
> > > >> > > > the
> > > >> > > > >> > >> produce
> > > >> > > > >> > >> > > > > request
> > > >> > > > >> > >> > > > > > > > > version)
> > > >> > > > >> > >> > > > > > > > > > will simply abort the transaction.
> > This
> > > >> > allows
> > > >> > > > the
> > > >> > > > >> > >> server
> > > >> > > > >> > >> > to
> > > >> > > > >> > >> > > > have
> > > >> > > > >> > >> > > > > > > some
> > > >> > > > >> > >> > > > > > > > > say
> > > >> > > > >> > >> > > > > > > > > > in whether the client aborts and makes
> > > >> > handling
> > > >> > > > >> much
> > > >> > > > >> > >> > simpler.
> > > >> > > > >> > >> > > > In
> > > >> > > > >> > >> > > > > > the
> > > >> > > > >> > >> > > > > > > > > > future, we can also use this error in
> > > >> other
> > > >> > > > >> situations
> > > >> > > > >> > >> > where
> > > >> > > > >> > >> > > we
> > > >> > > > >> > >> > > > > > want
> > > >> > > > >> > >> > > > > > > to
> > > >> > > > >> > >> > > > > > > > > > abort the transactions. We can even
> > use on
> > > >> > > other
> > > >> > > > >> apis.
> > > >> > > > >> > >> > > > > > > > > >
> > > >> > > > >> > >> > > > > > > > > > I've added this to the KIP. Let me
> > know if
> > > >> > > there
> > > >> > > > >> are
> > > >> > > > >> > any
> > > >> > > > >> > >> > > > > questions
> > > >> > > > >> > >> > > > > > or
> > > >> > > > >> > >> > > > > > > > > > issues.
> > > >> > > > >> > >> > > > > > > > > >
> > > >> > > > >> > >> > > > > > > > > > Justine
> > > >> > > > >> > >> > > > > > > > > >
> > > >> > > > >> > >> > > > > > > > > > On Fri, Dec 2, 2022 at 10:22 AM
> > Justine
> > > >> > Olshan
> > > >> > > <
> > > >> > > > >> > >> > > > > > jolshan@confluent.io
> > > >> > > > >> > >> > > > > > > >
> > > >> > > > >> > >> > > > > > > > > wrote:
> > > >> > > > >> > >> > > > > > > > > >
> > > >> > > > >> > >> > > > > > > > > >> Hey Matthias,
> > > >> > > > >> > >> > > > > > > > > >>
> > > >> > > > >> > >> > > > > > > > > >>
> > > >> > > > >> > >> > > > > > > > > >> 20/30 — Maybe I also didn't express
> > > >> myself
> > > >> > > > >> clearly.
> > > >> > > > >> > For
> > > >> > > > >> > >> > > older
> > > >> > > > >> > >> > > > > > > clients
> > > >> > > > >> > >> > > > > > > > we
> > > >> > > > >> > >> > > > > > > > > >> don't have a way to distinguish
> > between a
> > > >> > > > previous
> > > >> > > > >> > and
> > > >> > > > >> > >> the
> > > >> > > > >> > >> > > > > current
> > > >> > > > >> > >> > > > > > > > > >> transaction since we don't have the
> > epoch
> > > >> > > bump.
> > > >> > > > >> This
> > > >> > > > >> > >> means
> > > >> > > > >> > >> > > > that
> > > >> > > > >> > >> > > > > a
> > > >> > > > >> > >> > > > > > > late
> > > >> > > > >> > >> > > > > > > > > >> message from the previous transaction
> > > >> may be
> > > >> > > > >> added to
> > > >> > > > >> > >> the
> > > >> > > > >> > >> > > new
> > > >> > > > >> > >> > > > > one.
> > > >> > > > >> > >> > > > > > > > With
> > > >> > > > >> > >> > > > > > > > > >> older clients — we can't guarantee
> > this
> > > >> > won't
> > > >> > > > >> happen
> > > >> > > > >> > >> if we
> > > >> > > > >> > >> > > > > already
> > > >> > > > >> > >> > > > > > > > sent
> > > >> > > > >> > >> > > > > > > > > the
> > > >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call (why we make
> > > >> changes
> > > >> > > for
> > > >> > > > >> the
> > > >> > > > >> > >> newer
> > > >> > > > >> > >> > > > > client)
> > > >> > > > >> > >> > > > > > > but
> > > >> > > > >> > >> > > > > > > > > we
> > > >> > > > >> > >> > > > > > > > > >> can at least gate some by ensuring
> > that
> > > >> the
> > > >> > > > >> partition
> > > >> > > > >> > >> has
> > > >> > > > >> > >> > > been
> > > >> > > > >> > >> > > > > > added
> > > >> > > > >> > >> > > > > > > > to
> > > >> > > > >> > >> > > > > > > > > the
> > > >> > > > >> > >> > > > > > > > > >> transaction. The rationale here is
> > that
> > > >> > there
> > > >> > > > are
> > > >> > > > >> > >> likely
> > > >> > > > >> > >> > > LESS
> > > >> > > > >> > >> > > > > late
> > > >> > > > >> > >> > > > > > > > > arrivals
> > > >> > > > >> > >> > > > > > > > > >> as time goes on, so hopefully most
> > late
> > > >> > > arrivals
> > > >> > > > >> will
> > > >> > > > >> > >> come
> > > >> > > > >> > >> > > in
> > > >> > > > >> > >> > > > > > BEFORE
> > > >> > > > >> > >> > > > > > > > the
> > > >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call. Those that
> > > >> arrive
> > > >> > > > before
> > > >> > > > >> > will
> > > >> > > > >> > >> be
> > > >> > > > >> > >> > > > > properly
> > > >> > > > >> > >> > > > > > > > gated
> > > >> > > > >> > >> > > > > > > > > >> with the describeTransactions
> > approach.
> > > >> > > > >> > >> > > > > > > > > >>
> > > >> > > > >> > >> > > > > > > > > >> If we take the approach you
> > suggested,
> > > >> ANY
> > > >> > > late
> > > >> > > > >> > arrival
> > > >> > > > >> > >> > > from a
> > > >> > > > >> > >> > > > > > > > previous
> > > >> > > > >> > >> > > > > > > > > >> transaction will be added. And we
> > don't
> > > >> want
> > > >> > > > >> that. I
> > > >> > > > >> > >> also
> > > >> > > > >> > >> > > > don't
> > > >> > > > >> > >> > > > > > see
> > > >> > > > >> > >> > > > > > > > any
> > > >> > > > >> > >> > > > > > > > > >> benefit in sending addPartitionsToTxn
> > > >> over
> > > >> > the
> > > >> > > > >> > >> > describeTxns
> > > >> > > > >> > >> > > > > call.
> > > >> > > > >> > >> > > > > > > They
> > > >> > > > >> > >> > > > > > > > > will
> > > >> > > > >> > >> > > > > > > > > >> both be one extra RPC to the Txn
> > > >> > coordinator.
> > > >> > > > >> > >> > > > > > > > > >>
> > > >> > > > >> > >> > > > > > > > > >>
> > > >> > > > >> > >> > > > > > > > > >> To be clear — newer clients will use
> > > >> > > > >> > addPartitionsToTxn
> > > >> > > > >> > >> > > > instead
> > > >> > > > >> > >> > > > > of
> > > >> > > > >> > >> > > > > > > the
> > > >> > > > >> > >> > > > > > > > > >> DescribeTxns.
> > > >> > > > >> > >> > > > > > > > > >>
> > > >> > > > >> > >> > > > > > > > > >>
> > > >> > > > >> > >> > > > > > > > > >> 40)
> > > >> > > > >> > >> > > > > > > > > >> My concern is that if we have some
> > delay
> > > >> in
> > > >> > > the
> > > >> > > > >> > client
> > > >> > > > >> > >> to
> > > >> > > > >> > >> > > bump
> > > >> > > > >> > >> > > > > the
> > > >> > > > >> > >> > > > > > > > > epoch,
> > > >> > > > >> > >> > > > > > > > > >> it could continue to send epoch 73
> > and
> > > >> those
> > > >> > > > >> records
> > > >> > > > >> > >> would
> > > >> > > > >> > >> > > not
> > > >> > > > >> > >> > > > > be
> > > >> > > > >> > >> > > > > > > > > fenced.
> > > >> > > > >> > >> > > > > > > > > >> Perhaps this is not an issue if we
> > don't
> > > >> > allow
> > > >> > > > the
> > > >> > > > >> > next
> > > >> > > > >> > >> > > > produce
> > > >> > > > >> > >> > > > > to
> > > >> > > > >> > >> > > > > > > go
> > > >> > > > >> > >> > > > > > > > > >> through before the EndTxn request
> > > >> returns.
> > > >> > I'm
> > > >> > > > >> also
> > > >> > > > >> > >> > thinking
> > > >> > > > >> > >> > > > > about
> > > >> > > > >> > >> > > > > > > > > cases of
> > > >> > > > >> > >> > > > > > > > > >> failure. I will need to think on
> > this a
> > > >> bit.
> > > >> > > > >> > >> > > > > > > > > >>
> > > >> > > > >> > >> > > > > > > > > >> I wasn't sure if it was that
> > confusing.
> > > >> But
> > > >> > if
> > > >> > > > we
> > > >> > > > >> > >> think it
> > > >> > > > >> > >> > > is,
> > > >> > > > >> > >> > > > > we
> > > >> > > > >> > >> > > > > > > can
> > > >> > > > >> > >> > > > > > > > > >> investigate other ways.
> > > >> > > > >> > >> > > > > > > > > >>
> > > >> > > > >> > >> > > > > > > > > >>
> > > >> > > > >> > >> > > > > > > > > >> 60)
> > > >> > > > >> > >> > > > > > > > > >>
> > > >> > > > >> > >> > > > > > > > > >> I'm not sure these are the same
> > > >> purgatories
> > > >> > > > since
> > > >> > > > >> one
> > > >> > > > >> > >> is a
> > > >> > > > >> > >> > > > > produce
> > > >> > > > >> > >> > > > > > > > > >> purgatory (I was planning on using a
> > > >> > callback
> > > >> > > > >> rather
> > > >> > > > >> > >> than
> > > >> > > > >> > >> > > > > > purgatory)
> > > >> > > > >> > >> > > > > > > > and
> > > >> > > > >> > >> > > > > > > > > >> the other is simply a request to
> > append
> > > >> to
> > > >> > the
> > > >> > > > >> log.
> > > >> > > > >> > Not
> > > >> > > > >> > >> > sure
> > > >> > > > >> > >> > > > we
> > > >> > > > >> > >> > > > > > have
> > > >> > > > >> > >> > > > > > > > any
> > > >> > > > >> > >> > > > > > > > > >> structure here for ordering, but my
> > > >> > > > understanding
> > > >> > > > >> is
> > > >> > > > >> > >> that
> > > >> > > > >> > >> > > the
> > > >> > > > >> > >> > > > > > broker
> > > >> > > > >> > >> > > > > > > > > could
> > > >> > > > >> > >> > > > > > > > > >> handle the write request before it
> > hears
> > > >> > back
> > > >> > > > from
> > > >> > > > >> > the
> > > >> > > > >> > >> Txn
> > > >> > > > >> > >> > > > > > > > Coordinator.
> > > >> > > > >> > >> > > > > > > > > >>
> > > >> > > > >> > >> > > > > > > > > >> Let me know if I misunderstood
> > something
> > > >> or
> > > >> > > > >> something
> > > >> > > > >> > >> was
> > > >> > > > >> > >> > > > > unclear.
> > > >> > > > >> > >> > > > > > > > > >>
> > > >> > > > >> > >> > > > > > > > > >> Justine
> > > >> > > > >> > >> > > > > > > > > >>
> > > >> > > > >> > >> > > > > > > > > >> On Thu, Dec 1, 2022 at 12:15 PM
> > Matthias
> > > >> J.
> > > >> > > Sax
> > > >> > > > <
> > > >> > > > >> > >> > > > > mjsax@apache.org
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > > > > > wrote:
> > > >> > > > >> > >> > > > > > > > > >>
> > > >> > > > >> > >> > > > > > > > > >>> Thanks for the details Justine!
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > > >>>> 20)
> > > >> > > > >> > >> > > > > > > > > >>>>
> > > >> > > > >> > >> > > > > > > > > >>>> The client side change for 2 is
> > > >> removing
> > > >> > the
> > > >> > > > >> > >> > addPartitions
> > > >> > > > >> > >> > > > to
> > > >> > > > >> > >> > > > > > > > > >>> transaction
> > > >> > > > >> > >> > > > > > > > > >>>> call. We don't need to make this
> > from
> > > >> the
> > > >> > > > >> producer
> > > >> > > > >> > to
> > > >> > > > >> > >> > the
> > > >> > > > >> > >> > > > txn
> > > >> > > > >> > >> > > > > > > > > >>> coordinator,
> > > >> > > > >> > >> > > > > > > > > >>>> only server side.
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > > >>> I think I did not express myself
> > > >> clearly. I
> > > >> > > > >> > understand
> > > >> > > > >> > >> > that
> > > >> > > > >> > >> > > > we
> > > >> > > > >> > >> > > > > > can
> > > >> > > > >> > >> > > > > > > > (and
> > > >> > > > >> > >> > > > > > > > > >>> should) change the producer to not
> > send
> > > >> the
> > > >> > > > >> > >> > `addPartitions`
> > > >> > > > >> > >> > > > > > request
> > > >> > > > >> > >> > > > > > > > any
> > > >> > > > >> > >> > > > > > > > > >>> longer. But I don't thinks it's
> > > >> requirement
> > > >> > > to
> > > >> > > > >> > change
> > > >> > > > >> > >> the
> > > >> > > > >> > >> > > > > broker?
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > > >>> What I am trying to say is: as a
> > > >> safe-guard
> > > >> > > and
> > > >> > > > >> > >> > improvement
> > > >> > > > >> > >> > > > for
> > > >> > > > >> > >> > > > > > > older
> > > >> > > > >> > >> > > > > > > > > >>> producers, the partition leader can
> > just
> > > >> > send
> > > >> > > > the
> > > >> > > > >> > >> > > > > `addPartitions`
> > > >> > > > >> > >> > > > > > > > > >>> request to the TX-coordinator in any
> > > >> case
> > > >> > --
> > > >> > > if
> > > >> > > > >> the
> > > >> > > > >> > >> old
> > > >> > > > >> > >> > > > > producer
> > > >> > > > >> > >> > > > > > > > > >>> correctly did send the
> > `addPartition`
> > > >> > request
> > > >> > > > to
> > > >> > > > >> the
> > > >> > > > >> > >> > > > > > TX-coordinator
> > > >> > > > >> > >> > > > > > > > > >>> already, the TX-coordinator can just
> > > >> > "ignore"
> > > >> > > > is
> > > >> > > > >> as
> > > >> > > > >> > >> > > > idempotent.
> > > >> > > > >> > >> > > > > > > > > However,
> > > >> > > > >> > >> > > > > > > > > >>> if the old producer has a bug and
> > did
> > > >> > forget
> > > >> > > to
> > > >> > > > >> sent
> > > >> > > > >> > >> the
> > > >> > > > >> > >> > > > > > > > `addPartition`
> > > >> > > > >> > >> > > > > > > > > >>> request, we would now ensure that
> > the
> > > >> > > partition
> > > >> > > > >> is
> > > >> > > > >> > >> indeed
> > > >> > > > >> > >> > > > added
> > > >> > > > >> > >> > > > > > to
> > > >> > > > >> > >> > > > > > > > the
> > > >> > > > >> > >> > > > > > > > > >>> TX and thus fix a potential
> > producer bug
> > > >> > > (even
> > > >> > > > >> if we
> > > >> > > > >> > >> > don't
> > > >> > > > >> > >> > > > get
> > > >> > > > >> > >> > > > > > the
> > > >> > > > >> > >> > > > > > > > > >>> fencing via the bump epoch). -- It
> > > >> seems to
> > > >> > > be
> > > >> > > > a
> > > >> > > > >> > good
> > > >> > > > >> > >> > > > > > improvement?
> > > >> > > > >> > >> > > > > > > Or
> > > >> > > > >> > >> > > > > > > > > is
> > > >> > > > >> > >> > > > > > > > > >>> there a reason to not do this?
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > > >>>> 30)
> > > >> > > > >> > >> > > > > > > > > >>>>
> > > >> > > > >> > >> > > > > > > > > >>>> Transaction is ongoing = partition
> > was
> > > >> > added
> > > >> > > > to
> > > >> > > > >> > >> > > transaction
> > > >> > > > >> > >> > > > > via
> > > >> > > > >> > >> > > > > > > > > >>>> addPartitionsToTxn. We check this
> > with
> > > >> the
> > > >> > > > >> > >> > > > > DescribeTransactions
> > > >> > > > >> > >> > > > > > > > call.
> > > >> > > > >> > >> > > > > > > > > >>> Let
> > > >> > > > >> > >> > > > > > > > > >>>> me know if this wasn't sufficiently
> > > >> > > explained
> > > >> > > > >> here:
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > > >>> If we do what I propose in (20), we
> > > >> don't
> > > >> > > > really
> > > >> > > > >> > need
> > > >> > > > >> > >> to
> > > >> > > > >> > >> > > make
> > > >> > > > >> > >> > > > > > this
> > > >> > > > >> > >> > > > > > > > > >>> `DescribeTransaction` call, as the
> > > >> > partition
> > > >> > > > >> leader
> > > >> > > > >> > >> adds
> > > >> > > > >> > >> > > the
> > > >> > > > >> > >> > > > > > > > partition
> > > >> > > > >> > >> > > > > > > > > >>> for older clients and we get this
> > check
> > > >> for
> > > >> > > > free.
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > > >>>> 40)
> > > >> > > > >> > >> > > > > > > > > >>>>
> > > >> > > > >> > >> > > > > > > > > >>>> The idea here is that if any
> > messages
> > > >> > > somehow
> > > >> > > > >> come
> > > >> > > > >> > in
> > > >> > > > >> > >> > > before
> > > >> > > > >> > >> > > > > we
> > > >> > > > >> > >> > > > > > > get
> > > >> > > > >> > >> > > > > > > > > the
> > > >> > > > >> > >> > > > > > > > > >>> new
> > > >> > > > >> > >> > > > > > > > > >>>> epoch to the producer, they will be
> > > >> > fenced.
> > > >> > > > >> > However,
> > > >> > > > >> > >> if
> > > >> > > > >> > >> > we
> > > >> > > > >> > >> > > > > don't
> > > >> > > > >> > >> > > > > > > > think
> > > >> > > > >> > >> > > > > > > > > >>> this
> > > >> > > > >> > >> > > > > > > > > >>>> is necessary, it can be discussed
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > > >>> I agree that we should have epoch
> > > >> fencing.
> > > >> > My
> > > >> > > > >> > >> question is
> > > >> > > > >> > >> > > > > > > different:
> > > >> > > > >> > >> > > > > > > > > >>> Assume we are at epoch 73, and we
> > have
> > > >> an
> > > >> > > > ongoing
> > > >> > > > >> > >> > > > transaction,
> > > >> > > > >> > >> > > > > > that
> > > >> > > > >> > >> > > > > > > > is
> > > >> > > > >> > >> > > > > > > > > >>> committed. It seems natural to
> > write the
> > > >> > > > "prepare
> > > >> > > > >> > >> commit"
> > > >> > > > >> > >> > > > > marker
> > > >> > > > >> > >> > > > > > > and
> > > >> > > > >> > >> > > > > > > > > the
> > > >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` both with
> > epoch
> > > >> 73,
> > > >> > > too,
> > > >> > > > >> as
> > > >> > > > >> > it
> > > >> > > > >> > >> > > belongs
> > > >> > > > >> > >> > > > > to
> > > >> > > > >> > >> > > > > > > the
> > > >> > > > >> > >> > > > > > > > > >>> current transaction. Of course, we
> > now
> > > >> also
> > > >> > > > bump
> > > >> > > > >> the
> > > >> > > > >> > >> > epoch
> > > >> > > > >> > >> > > > and
> > > >> > > > >> > >> > > > > > > expect
> > > >> > > > >> > >> > > > > > > > > >>> the next requests to have epoch 74,
> > and
> > > >> > would
> > > >> > > > >> reject
> > > >> > > > >> > >> an
> > > >> > > > >> > >> > > > request
> > > >> > > > >> > >> > > > > > > with
> > > >> > > > >> > >> > > > > > > > > >>> epoch 73, as the corresponding TX
> > for
> > > >> epoch
> > > >> > > 73
> > > >> > > > >> was
> > > >> > > > >> > >> > already
> > > >> > > > >> > >> > > > > > > committed.
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > > >>> It seems you propose to write the
> > > >> "prepare
> > > >> > > > commit
> > > >> > > > >> > >> marker"
> > > >> > > > >> > >> > > and
> > > >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` with epoch 74
> > > >> > though,
> > > >> > > > what
> > > >> > > > >> > >> would
> > > >> > > > >> > >> > > work,
> > > >> > > > >> > >> > > > > but
> > > >> > > > >> > >> > > > > > > it
> > > >> > > > >> > >> > > > > > > > > >>> seems confusing. Is there a reason
> > why
> > > >> we
> > > >> > > would
> > > >> > > > >> use
> > > >> > > > >> > >> the
> > > >> > > > >> > >> > > > bumped
> > > >> > > > >> > >> > > > > > > epoch
> > > >> > > > >> > >> > > > > > > > 74
> > > >> > > > >> > >> > > > > > > > > >>> instead of the current epoch 73?
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > > >>>> 60)
> > > >> > > > >> > >> > > > > > > > > >>>>
> > > >> > > > >> > >> > > > > > > > > >>>> When we are checking if the
> > > >> transaction is
> > > >> > > > >> ongoing,
> > > >> > > > >> > >> we
> > > >> > > > >> > >> > > need
> > > >> > > > >> > >> > > > to
> > > >> > > > >> > >> > > > > > > make
> > > >> > > > >> > >> > > > > > > > a
> > > >> > > > >> > >> > > > > > > > > >>> round
> > > >> > > > >> > >> > > > > > > > > >>>> trip from the leader partition to
> > the
> > > >> > > > >> transaction
> > > >> > > > >> > >> > > > coordinator.
> > > >> > > > >> > >> > > > > > In
> > > >> > > > >> > >> > > > > > > > the
> > > >> > > > >> > >> > > > > > > > > >>> time
> > > >> > > > >> > >> > > > > > > > > >>>> we are waiting for this message to
> > come
> > > >> > > back,
> > > >> > > > in
> > > >> > > > >> > >> theory
> > > >> > > > >> > >> > we
> > > >> > > > >> > >> > > > > could
> > > >> > > > >> > >> > > > > > > > have
> > > >> > > > >> > >> > > > > > > > > >>> sent
> > > >> > > > >> > >> > > > > > > > > >>>> a commit/abort call that would
> > make the
> > > >> > > > original
> > > >> > > > >> > >> result
> > > >> > > > >> > >> > of
> > > >> > > > >> > >> > > > the
> > > >> > > > >> > >> > > > > > > check
> > > >> > > > >> > >> > > > > > > > > >>> out of
> > > >> > > > >> > >> > > > > > > > > >>>> date. That is why we can check the
> > > >> leader
> > > >> > > > state
> > > >> > > > >> > >> before
> > > >> > > > >> > >> > we
> > > >> > > > >> > >> > > > > write
> > > >> > > > >> > >> > > > > > to
> > > >> > > > >> > >> > > > > > > > the
> > > >> > > > >> > >> > > > > > > > > >>> log.
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > > >>> Thanks. Got it.
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > > >>> However, is this really an issue?
> > We put
> > > >> > the
> > > >> > > > >> produce
> > > >> > > > >> > >> > > request
> > > >> > > > >> > >> > > > in
> > > >> > > > >> > >> > > > > > > > > >>> purgatory, so how could we process
> > the
> > > >> > > > >> > >> > > > `WriteTxnMarkerRequest`
> > > >> > > > >> > >> > > > > > > first?
> > > >> > > > >> > >> > > > > > > > > >>> Don't we need to put the
> > > >> > > > `WriteTxnMarkerRequest`
> > > >> > > > >> > into
> > > >> > > > >> > >> > > > > purgatory,
> > > >> > > > >> > >> > > > > > > too,
> > > >> > > > >> > >> > > > > > > > > >>> for this case, and process both
> > request
> > > >> > > > in-order?
> > > >> > > > >> > >> (Again,
> > > >> > > > >> > >> > > my
> > > >> > > > >> > >> > > > > > broker
> > > >> > > > >> > >> > > > > > > > > >>> knowledge is limited and maybe we
> > don't
> > > >> > > > maintain
> > > >> > > > >> > >> request
> > > >> > > > >> > >> > > > order
> > > >> > > > >> > >> > > > > > for
> > > >> > > > >> > >> > > > > > > > this
> > > >> > > > >> > >> > > > > > > > > >>> case, what seems to be an issue
> > IMHO,
> > > >> and I
> > > >> > > am
> > > >> > > > >> > >> wondering
> > > >> > > > >> > >> > if
> > > >> > > > >> > >> > > > > > > changing
> > > >> > > > >> > >> > > > > > > > > >>> request handling to preserve order
> > for
> > > >> this
> > > >> > > > case
> > > >> > > > >> > >> might be
> > > >> > > > >> > >> > > the
> > > >> > > > >> > >> > > > > > > cleaner
> > > >> > > > >> > >> > > > > > > > > >>> solution?)
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > > >>> -Matthias
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > > >>> On 11/30/22 3:28 PM, Artem Livshits
> > > >> wrote:
> > > >> > > > >> > >> > > > > > > > > >>>> Hi Justine,
> > > >> > > > >> > >> > > > > > > > > >>>>
> > > >> > > > >> > >> > > > > > > > > >>>> I think the interesting part is
> > not in
> > > >> > this
> > > >> > > > >> logic
> > > >> > > > >> > >> > (because
> > > >> > > > >> > >> > > > it
> > > >> > > > >> > >> > > > > > > tries
> > > >> > > > >> > >> > > > > > > > to
> > > >> > > > >> > >> > > > > > > > > >>>> figure out when
> > UNKNOWN_PRODUCER_ID is
> > > >> > > > retriable
> > > >> > > > >> > and
> > > >> > > > >> > >> if
> > > >> > > > >> > >> > > it's
> > > >> > > > >> > >> > > > > > > > > retryable,
> > > >> > > > >> > >> > > > > > > > > >>>> it's definitely not fatal), but
> > what
> > > >> > happens
> > > >> > > > >> when
> > > >> > > > >> > >> this
> > > >> > > > >> > >> > > logic
> > > >> > > > >> > >> > > > > > > doesn't
> > > >> > > > >> > >> > > > > > > > > >>> return
> > > >> > > > >> > >> > > > > > > > > >>>> 'true' and falls through.  In the
> > old
> > > >> > > clients
> > > >> > > > it
> > > >> > > > >> > >> seems
> > > >> > > > >> > >> > to
> > > >> > > > >> > >> > > be
> > > >> > > > >> > >> > > > > > > fatal,
> > > >> > > > >> > >> > > > > > > > if
> > > >> > > > >> > >> > > > > > > > > >>> we
> > > >> > > > >> > >> > > > > > > > > >>>> keep the behavior in the new
> > clients,
> > > >> I'd
> > > >> > > > >> expect it
> > > >> > > > >> > >> > would
> > > >> > > > >> > >> > > be
> > > >> > > > >> > >> > > > > > fatal
> > > >> > > > >> > >> > > > > > > > as
> > > >> > > > >> > >> > > > > > > > > >>> well.
> > > >> > > > >> > >> > > > > > > > > >>>>
> > > >> > > > >> > >> > > > > > > > > >>>> -Artem
> > > >> > > > >> > >> > > > > > > > > >>>>
> > > >> > > > >> > >> > > > > > > > > >>>> On Tue, Nov 29, 2022 at 11:57 AM
> > > >> Justine
> > > >> > > > Olshan
> > > >> > > > >> > >> > > > > > > > > >>>> <jo...@confluent.io.invalid>
> > wrote:
> > > >> > > > >> > >> > > > > > > > > >>>>
> > > >> > > > >> > >> > > > > > > > > >>>>> Hi Artem and Jeff,
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>> Thanks for taking a look and
> > sorry for
> > > >> > the
> > > >> > > > slow
> > > >> > > > >> > >> > response.
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>> You both mentioned the change to
> > > >> handle
> > > >> > > > >> > >> > > UNKNOWN_PRODUCER_ID
> > > >> > > > >> > >> > > > > > > errors.
> > > >> > > > >> > >> > > > > > > > > To
> > > >> > > > >> > >> > > > > > > > > >>> be
> > > >> > > > >> > >> > > > > > > > > >>>>> clear — this error code will only
> > be
> > > >> sent
> > > >> > > > again
> > > >> > > > >> > when
> > > >> > > > >> > >> > the
> > > >> > > > >> > >> > > > > > client's
> > > >> > > > >> > >> > > > > > > > > >>> request
> > > >> > > > >> > >> > > > > > > > > >>>>> version is high enough to ensure
> > we
> > > >> > handle
> > > >> > > it
> > > >> > > > >> > >> > correctly.
> > > >> > > > >> > >> > > > > > > > > >>>>> The current (Java) client handles
> > > >> this by
> > > >> > > the
> > > >> > > > >> > >> following
> > > >> > > > >> > >> > > > > > (somewhat
> > > >> > > > >> > >> > > > > > > > > long)
> > > >> > > > >> > >> > > > > > > > > >>>>> code snippet:
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>> // An UNKNOWN_PRODUCER_ID means
> > that
> > > >> we
> > > >> > > have
> > > >> > > > >> lost
> > > >> > > > >> > >> the
> > > >> > > > >> > >> > > > > producer
> > > >> > > > >> > >> > > > > > > > state
> > > >> > > > >> > >> > > > > > > > > >>> on the
> > > >> > > > >> > >> > > > > > > > > >>>>> broker. Depending on the log start
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>> // offset, we may want to retry
> > > >> these, as
> > > >> > > > >> > described
> > > >> > > > >> > >> for
> > > >> > > > >> > >> > > > each
> > > >> > > > >> > >> > > > > > case
> > > >> > > > >> > >> > > > > > > > > >>> below. If
> > > >> > > > >> > >> > > > > > > > > >>>>> none of those apply, then for the
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>> // idempotent producer, we will
> > > >> locally
> > > >> > > bump
> > > >> > > > >> the
> > > >> > > > >> > >> epoch
> > > >> > > > >> > >> > > and
> > > >> > > > >> > >> > > > > > reset
> > > >> > > > >> > >> > > > > > > > the
> > > >> > > > >> > >> > > > > > > > > >>>>> sequence numbers of in-flight
> > batches
> > > >> > from
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>> // sequence 0, then retry the
> > failed
> > > >> > batch,
> > > >> > > > >> which
> > > >> > > > >> > >> > should
> > > >> > > > >> > >> > > > now
> > > >> > > > >> > >> > > > > > > > succeed.
> > > >> > > > >> > >> > > > > > > > > >>> For
> > > >> > > > >> > >> > > > > > > > > >>>>> the transactional producer, allow
> > the
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>> // batch to fail. When processing
> > the
> > > >> > > failed
> > > >> > > > >> > batch,
> > > >> > > > >> > >> we
> > > >> > > > >> > >> > > will
> > > >> > > > >> > >> > > > > > > > > transition
> > > >> > > > >> > >> > > > > > > > > >>> to
> > > >> > > > >> > >> > > > > > > > > >>>>> an abortable error and set a flag
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>> // indicating that we need to
> > bump the
> > > >> > > epoch
> > > >> > > > >> (if
> > > >> > > > >> > >> > > supported
> > > >> > > > >> > >> > > > by
> > > >> > > > >> > >> > > > > > the
> > > >> > > > >> > >> > > > > > > > > >>> broker).
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>> if (error ==
> > > >> > Errors.*UNKNOWN_PRODUCER_ID*)
> > > >> > > {
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>       if (response.logStartOffset
> > ==
> > > >> -1)
> > > >> > {
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           // We don't know the log
> > > >> start
> > > >> > > > offset
> > > >> > > > >> > with
> > > >> > > > >> > >> > this
> > > >> > > > >> > >> > > > > > > response.
> > > >> > > > >> > >> > > > > > > > > We
> > > >> > > > >> > >> > > > > > > > > >>> should
> > > >> > > > >> > >> > > > > > > > > >>>>> just retry the request until we
> > get
> > > >> it.
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           // The
> > UNKNOWN_PRODUCER_ID
> > > >> > error
> > > >> > > > code
> > > >> > > > >> > was
> > > >> > > > >> > >> > added
> > > >> > > > >> > >> > > > > along
> > > >> > > > >> > >> > > > > > > > with
> > > >> > > > >> > >> > > > > > > > > >>> the new
> > > >> > > > >> > >> > > > > > > > > >>>>> ProduceResponse which includes the
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           // logStartOffset. So
> > the
> > > >> '-1'
> > > >> > > > >> sentinel
> > > >> > > > >> > is
> > > >> > > > >> > >> > not
> > > >> > > > >> > >> > > > for
> > > >> > > > >> > >> > > > > > > > backward
> > > >> > > > >> > >> > > > > > > > > >>>>> compatibility. Instead, it is
> > possible
> > > >> > for
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           // a broker to not know
> > the
> > > >> > > > >> > >> logStartOffset at
> > > >> > > > >> > >> > > > when
> > > >> > > > >> > >> > > > > it
> > > >> > > > >> > >> > > > > > > is
> > > >> > > > >> > >> > > > > > > > > >>> returning
> > > >> > > > >> > >> > > > > > > > > >>>>> the response because the partition
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           // may have moved away
> > from
> > > >> the
> > > >> > > > >> broker
> > > >> > > > >> > >> from
> > > >> > > > >> > >> > the
> > > >> > > > >> > >> > > > > time
> > > >> > > > >> > >> > > > > > > the
> > > >> > > > >> > >> > > > > > > > > >>> error was
> > > >> > > > >> > >> > > > > > > > > >>>>> initially raised to the time the
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           // response was being
> > > >> > > constructed.
> > > >> > > > In
> > > >> > > > >> > >> these
> > > >> > > > >> > >> > > > cases,
> > > >> > > > >> > >> > > > > we
> > > >> > > > >> > >> > > > > > > > > should
> > > >> > > > >> > >> > > > > > > > > >>> just
> > > >> > > > >> > >> > > > > > > > > >>>>> retry the request: we are
> > guaranteed
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           // to eventually get a
> > > >> > > > logStartOffset
> > > >> > > > >> > once
> > > >> > > > >> > >> > > things
> > > >> > > > >> > >> > > > > > > settle
> > > >> > > > >> > >> > > > > > > > > down.
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>       if
> > > >> (batch.sequenceHasBeenReset()) {
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           // When the first
> > inflight
> > > >> > batch
> > > >> > > > >> fails
> > > >> > > > >> > >> due to
> > > >> > > > >> > >> > > the
> > > >> > > > >> > >> > > > > > > > > truncation
> > > >> > > > >> > >> > > > > > > > > >>> case,
> > > >> > > > >> > >> > > > > > > > > >>>>> then the sequences of all the
> > other
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           // in flight batches
> > would
> > > >> have
> > > >> > > > been
> > > >> > > > >> > >> > restarted
> > > >> > > > >> > >> > > > from
> > > >> > > > >> > >> > > > > > the
> > > >> > > > >> > >> > > > > > > > > >>> beginning.
> > > >> > > > >> > >> > > > > > > > > >>>>> However, when those responses
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           // come back from the
> > > >> broker,
> > > >> > > they
> > > >> > > > >> would
> > > >> > > > >> > >> also
> > > >> > > > >> > >> > > > come
> > > >> > > > >> > >> > > > > > with
> > > >> > > > >> > >> > > > > > > > an
> > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID error. In this
> > > >> case,
> > > >> > we
> > > >> > > > >> should
> > > >> > > > >> > >> not
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           // reset the sequence
> > > >> numbers
> > > >> > to
> > > >> > > > the
> > > >> > > > >> > >> > beginning.
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>       } else if
> > > >> > > > >> > >> > > > > (lastAckedOffset(batch.topicPartition).orElse(
> > > >> > > > >> > >> > > > > > > > > >>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > > >> > > > >> > >> > > > response.logStartOffset) {
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           // The head of the log
> > has
> > > >> been
> > > >> > > > >> removed,
> > > >> > > > >> > >> > > probably
> > > >> > > > >> > >> > > > > due
> > > >> > > > >> > >> > > > > > > to
> > > >> > > > >> > >> > > > > > > > > the
> > > >> > > > >> > >> > > > > > > > > >>>>> retention time elapsing. In this
> > case,
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           // we expect to lose the
> > > >> > producer
> > > >> > > > >> state.
> > > >> > > > >> > >> For
> > > >> > > > >> > >> > > the
> > > >> > > > >> > >> > > > > > > > > transactional
> > > >> > > > >> > >> > > > > > > > > >>>>> producer, reset the sequences of
> > all
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           // inflight batches to
> > be
> > > >> from
> > > >> > > the
> > > >> > > > >> > >> beginning
> > > >> > > > >> > >> > > and
> > > >> > > > >> > >> > > > > > retry
> > > >> > > > >> > >> > > > > > > > > them,
> > > >> > > > >> > >> > > > > > > > > >>> so
> > > >> > > > >> > >> > > > > > > > > >>>>> that the transaction does not
> > need to
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           // be aborted. For the
> > > >> > idempotent
> > > >> > > > >> > >> producer,
> > > >> > > > >> > >> > > bump
> > > >> > > > >> > >> > > > > the
> > > >> > > > >> > >> > > > > > > > epoch
> > > >> > > > >> > >> > > > > > > > > to
> > > >> > > > >> > >> > > > > > > > > >>> avoid
> > > >> > > > >> > >> > > > > > > > > >>>>> reusing (sequence, epoch) pairs
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           if (isTransactional()) {
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > >
> > > >> > > > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > > >> > > > >> > >> > > > > > > > > >>>>> this.producerIdAndEpoch);
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           } else {
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > >
> > > >> requestEpochBumpForPartition(batch.topicPartition);
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           }
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>       if (!isTransactional()) {
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           // For the idempotent
> > > >> producer,
> > > >> > > > >> always
> > > >> > > > >> > >> retry
> > > >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > >> > > > >> > >> > > > > > > > > >>>>> errors. If the batch has the
> > current
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           // producer ID and
> > epoch,
> > > >> > > request a
> > > >> > > > >> bump
> > > >> > > > >> > >> of
> > > >> > > > >> > >> > the
> > > >> > > > >> > >> > > > > > epoch.
> > > >> > > > >> > >> > > > > > > > > >>> Otherwise
> > > >> > > > >> > >> > > > > > > > > >>>>> just retry the produce.
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > >
> > requestEpochBumpForPartition(batch.topicPartition);
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>       }
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>> }
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>> I was considering keeping this
> > > >> behavior —
> > > >> > > but
> > > >> > > > >> am
> > > >> > > > >> > >> open
> > > >> > > > >> > >> > to
> > > >> > > > >> > >> > > > > > > > simplifying
> > > >> > > > >> > >> > > > > > > > > >>> it.
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>> We are leaving changes to older
> > > >> clients
> > > >> > off
> > > >> > > > the
> > > >> > > > >> > >> table
> > > >> > > > >> > >> > > here
> > > >> > > > >> > >> > > > > > since
> > > >> > > > >> > >> > > > > > > it
> > > >> > > > >> > >> > > > > > > > > >>> caused
> > > >> > > > >> > >> > > > > > > > > >>>>> many issues for clients in the
> > past.
> > > >> > > > Previously
> > > >> > > > >> > this
> > > >> > > > >> > >> > was
> > > >> > > > >> > >> > > a
> > > >> > > > >> > >> > > > > > fatal
> > > >> > > > >> > >> > > > > > > > > error
> > > >> > > > >> > >> > > > > > > > > >>> and
> > > >> > > > >> > >> > > > > > > > > >>>>> we didn't have the mechanisms in
> > > >> place to
> > > >> > > > >> detect
> > > >> > > > >> > >> when
> > > >> > > > >> > >> > > this
> > > >> > > > >> > >> > > > > was
> > > >> > > > >> > >> > > > > > a
> > > >> > > > >> > >> > > > > > > > > >>> legitimate
> > > >> > > > >> > >> > > > > > > > > >>>>> case vs some bug or gap in the
> > > >> protocol.
> > > >> > > > >> Ensuring
> > > >> > > > >> > >> each
> > > >> > > > >> > >> > > > > > > transaction
> > > >> > > > >> > >> > > > > > > > > has
> > > >> > > > >> > >> > > > > > > > > >>> its
> > > >> > > > >> > >> > > > > > > > > >>>>> own epoch should close this gap.
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>> And to address Jeff's second
> > point:
> > > >> > > > >> > >> > > > > > > > > >>>>> *does the typical produce request
> > path
> > > >> > > append
> > > >> > > > >> > >> records
> > > >> > > > >> > >> > to
> > > >> > > > >> > >> > > > > local
> > > >> > > > >> > >> > > > > > > log
> > > >> > > > >> > >> > > > > > > > > >>> along*
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>> *with the currentTxnFirstOffset
> > > >> > > information?
> > > >> > > > I
> > > >> > > > >> > would
> > > >> > > > >> > >> > like
> > > >> > > > >> > >> > > > to
> > > >> > > > >> > >> > > > > > > > > >>> understand*
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>> *when the field is written to
> > disk.*
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>> Yes, the first produce request
> > > >> populates
> > > >> > > this
> > > >> > > > >> > field
> > > >> > > > >> > >> and
> > > >> > > > >> > >> > > > > writes
> > > >> > > > >> > >> > > > > > > the
> > > >> > > > >> > >> > > > > > > > > >>> offset
> > > >> > > > >> > >> > > > > > > > > >>>>> as part of the record batch and
> > also
> > > >> to
> > > >> > the
> > > >> > > > >> > producer
> > > >> > > > >> > >> > > state
> > > >> > > > >> > >> > > > > > > > snapshot.
> > > >> > > > >> > >> > > > > > > > > >>> When
> > > >> > > > >> > >> > > > > > > > > >>>>> we reload the records on restart
> > > >> and/or
> > > >> > > > >> > >> reassignment,
> > > >> > > > >> > >> > we
> > > >> > > > >> > >> > > > > > > repopulate
> > > >> > > > >> > >> > > > > > > > > >>> this
> > > >> > > > >> > >> > > > > > > > > >>>>> field with the snapshot from disk
> > > >> along
> > > >> > > with
> > > >> > > > >> the
> > > >> > > > >> > >> rest
> > > >> > > > >> > >> > of
> > > >> > > > >> > >> > > > the
> > > >> > > > >> > >> > > > > > > > producer
> > > >> > > > >> > >> > > > > > > > > >>>>> state.
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>> Let me know if there are further
> > > >> comments
> > > >> > > > >> and/or
> > > >> > > > >> > >> > > questions.
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>> Thanks,
> > > >> > > > >> > >> > > > > > > > > >>>>> Justine
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>> On Tue, Nov 22, 2022 at 9:00 PM
> > Jeff
> > > >> Kim
> > > >> > > > >> > >> > > > > > > > > <jeff.kim@confluent.io.invalid
> > > >> > > > >> > >> > > > > > > > > >>>>
> > > >> > > > >> > >> > > > > > > > > >>>>> wrote:
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>> Hi Justine,
> > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>> Thanks for the KIP! I have two
> > > >> > questions:
> > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>> 1) For new clients, we can once
> > again
> > > >> > > return
> > > >> > > > >> an
> > > >> > > > >> > >> error
> > > >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > >> > > > >> > >> > > > > > > > > >>>>>> for sequences
> > > >> > > > >> > >> > > > > > > > > >>>>>> that are non-zero when there is
> > no
> > > >> > > producer
> > > >> > > > >> state
> > > >> > > > >> > >> > > present
> > > >> > > > >> > >> > > > on
> > > >> > > > >> > >> > > > > > the
> > > >> > > > >> > >> > > > > > > > > >>> server.
> > > >> > > > >> > >> > > > > > > > > >>>>>> This will indicate we missed the
> > 0
> > > >> > > sequence
> > > >> > > > >> and
> > > >> > > > >> > we
> > > >> > > > >> > >> > don't
> > > >> > > > >> > >> > > > yet
> > > >> > > > >> > >> > > > > > > want
> > > >> > > > >> > >> > > > > > > > to
> > > >> > > > >> > >> > > > > > > > > >>>>> write
> > > >> > > > >> > >> > > > > > > > > >>>>>> to the log.
> > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>> I would like to understand the
> > > >> current
> > > >> > > > >> behavior
> > > >> > > > >> > to
> > > >> > > > >> > >> > > handle
> > > >> > > > >> > >> > > > > > older
> > > >> > > > >> > >> > > > > > > > > >>> clients,
> > > >> > > > >> > >> > > > > > > > > >>>>>> and if there are any changes we
> > are
> > > >> > > making.
> > > >> > > > >> Maybe
> > > >> > > > >> > >> I'm
> > > >> > > > >> > >> > > > > missing
> > > >> > > > >> > >> > > > > > > > > >>> something,
> > > >> > > > >> > >> > > > > > > > > >>>>>> but we would want to identify
> > > >> whether we
> > > >> > > > >> missed
> > > >> > > > >> > >> the 0
> > > >> > > > >> > >> > > > > sequence
> > > >> > > > >> > >> > > > > > > for
> > > >> > > > >> > >> > > > > > > > > >>> older
> > > >> > > > >> > >> > > > > > > > > >>>>>> clients, no?
> > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>> 2) Upon returning from the
> > > >> transaction
> > > >> > > > >> > >> coordinator, we
> > > >> > > > >> > >> > > can
> > > >> > > > >> > >> > > > > set
> > > >> > > > >> > >> > > > > > > the
> > > >> > > > >> > >> > > > > > > > > >>>>>> transaction
> > > >> > > > >> > >> > > > > > > > > >>>>>> as ongoing on the leader by
> > > >> populating
> > > >> > > > >> > >> > > > currentTxnFirstOffset
> > > >> > > > >> > >> > > > > > > > > >>>>>> through the typical produce
> > request
> > > >> > > > handling.
> > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>> does the typical produce request
> > path
> > > >> > > append
> > > >> > > > >> > >> records
> > > >> > > > >> > >> > to
> > > >> > > > >> > >> > > > > local
> > > >> > > > >> > >> > > > > > > log
> > > >> > > > >> > >> > > > > > > > > >>> along
> > > >> > > > >> > >> > > > > > > > > >>>>>> with the currentTxnFirstOffset
> > > >> > > information?
> > > >> > > > I
> > > >> > > > >> > would
> > > >> > > > >> > >> > like
> > > >> > > > >> > >> > > > to
> > > >> > > > >> > >> > > > > > > > > understand
> > > >> > > > >> > >> > > > > > > > > >>>>>> when the field is written to
> > disk.
> > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>> Thanks,
> > > >> > > > >> > >> > > > > > > > > >>>>>> Jeff
> > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>> On Tue, Nov 22, 2022 at 4:44 PM
> > Artem
> > > >> > > > Livshits
> > > >> > > > >> > >> > > > > > > > > >>>>>> <al...@confluent.io.invalid>
> > > >> wrote:
> > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>> Hi Justine,
> > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>> Thank you for the KIP.  I have
> > one
> > > >> > > > question.
> > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>> 5) For new clients, we can once
> > > >> again
> > > >> > > > return
> > > >> > > > >> an
> > > >> > > > >> > >> error
> > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>> I believe we had problems in the
> > > >> past
> > > >> > > with
> > > >> > > > >> > >> returning
> > > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > >> > > > >> > >> > > > > > > > > >>>>>>> because it was considered fatal
> > and
> > > >> > > > required
> > > >> > > > >> > >> client
> > > >> > > > >> > >> > > > > restart.
> > > >> > > > >> > >> > > > > > > It
> > > >> > > > >> > >> > > > > > > > > >>> would
> > > >> > > > >> > >> > > > > > > > > >>>>> be
> > > >> > > > >> > >> > > > > > > > > >>>>>>> good to spell out the new client
> > > >> > behavior
> > > >> > > > >> when
> > > >> > > > >> > it
> > > >> > > > >> > >> > > > receives
> > > >> > > > >> > >> > > > > > the
> > > >> > > > >> > >> > > > > > > > > error.
> > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>> -Artem
> > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>> On Tue, Nov 22, 2022 at 10:00 AM
> > > >> > Justine
> > > >> > > > >> Olshan
> > > >> > > > >> > >> > > > > > > > > >>>>>>> <jo...@confluent.io.invalid>
> > > >> wrote:
> > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks for taking a look
> > Matthias.
> > > >> > I've
> > > >> > > > >> tried
> > > >> > > > >> > to
> > > >> > > > >> > >> > > answer
> > > >> > > > >> > >> > > > > your
> > > >> > > > >> > >> > > > > > > > > >>>>> questions
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> below:
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> 10)
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> Right — so the hanging
> > transaction
> > > >> > only
> > > >> > > > >> occurs
> > > >> > > > >> > >> when
> > > >> > > > >> > >> > we
> > > >> > > > >> > >> > > > > have
> > > >> > > > >> > >> > > > > > a
> > > >> > > > >> > >> > > > > > > > late
> > > >> > > > >> > >> > > > > > > > > >>>>>>> message
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> come in and the partition is
> > never
> > > >> > added
> > > >> > > > to
> > > >> > > > >> a
> > > >> > > > >> > >> > > > transaction
> > > >> > > > >> > >> > > > > > > again.
> > > >> > > > >> > >> > > > > > > > > If
> > > >> > > > >> > >> > > > > > > > > >>>>> we
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> never add the partition to a
> > > >> > > transaction,
> > > >> > > > we
> > > >> > > > >> > will
> > > >> > > > >> > >> > > never
> > > >> > > > >> > >> > > > > > write
> > > >> > > > >> > >> > > > > > > a
> > > >> > > > >> > >> > > > > > > > > >>>>> marker
> > > >> > > > >> > >> > > > > > > > > >>>>>>> and
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> never advance the LSO.
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> If we do end up adding the
> > > >> partition
> > > >> > to
> > > >> > > > the
> > > >> > > > >> > >> > > transaction
> > > >> > > > >> > >> > > > (I
> > > >> > > > >> > >> > > > > > > > suppose
> > > >> > > > >> > >> > > > > > > > > >>>>> this
> > > >> > > > >> > >> > > > > > > > > >>>>>>> can
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> happen before or after the late
> > > >> > message
> > > >> > > > >> comes
> > > >> > > > >> > in)
> > > >> > > > >> > >> > then
> > > >> > > > >> > >> > > > we
> > > >> > > > >> > >> > > > > > will
> > > >> > > > >> > >> > > > > > > > > >>>>> include
> > > >> > > > >> > >> > > > > > > > > >>>>>>> the
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> late message in the next
> > > >> (incorrect)
> > > >> > > > >> > transaction.
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> So perhaps it is clearer to
> > make
> > > >> the
> > > >> > > > >> > distinction
> > > >> > > > >> > >> > > between
> > > >> > > > >> > >> > > > > > > > messages
> > > >> > > > >> > >> > > > > > > > > >>>>> that
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> eventually get added to the
> > > >> > transaction
> > > >> > > > (but
> > > >> > > > >> > the
> > > >> > > > >> > >> > wrong
> > > >> > > > >> > >> > > > > one)
> > > >> > > > >> > >> > > > > > or
> > > >> > > > >> > >> > > > > > > > > >>>>> messages
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> that never get added and become
> > > >> > hanging.
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> 20)
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> The client side change for 2 is
> > > >> > removing
> > > >> > > > the
> > > >> > > > >> > >> > > > addPartitions
> > > >> > > > >> > >> > > > > > to
> > > >> > > > >> > >> > > > > > > > > >>>>>> transaction
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> call. We don't need to make
> > this
> > > >> from
> > > >> > > the
> > > >> > > > >> > >> producer
> > > >> > > > >> > >> > to
> > > >> > > > >> > >> > > > the
> > > >> > > > >> > >> > > > > > txn
> > > >> > > > >> > >> > > > > > > > > >>>>>>> coordinator,
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> only server side.
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> In my opinion, the issue with
> > the
> > > >> > > > >> > >> addPartitionsToTxn
> > > >> > > > >> > >> > > > call
> > > >> > > > >> > >> > > > > > for
> > > >> > > > >> > >> > > > > > > > > older
> > > >> > > > >> > >> > > > > > > > > >>>>>>> clients
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> is that we don't have the epoch
> > > >> bump,
> > > >> > so
> > > >> > > > we
> > > >> > > > >> > don't
> > > >> > > > >> > >> > know
> > > >> > > > >> > >> > > > if
> > > >> > > > >> > >> > > > > > the
> > > >> > > > >> > >> > > > > > > > > >>> message
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> belongs to the previous
> > > >> transaction or
> > > >> > > > this
> > > >> > > > >> > one.
> > > >> > > > >> > >> We
> > > >> > > > >> > >> > > need
> > > >> > > > >> > >> > > > > to
> > > >> > > > >> > >> > > > > > > > check
> > > >> > > > >> > >> > > > > > > > > if
> > > >> > > > >> > >> > > > > > > > > >>>>>> the
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> partition has been added to
> > this
> > > >> > > > >> transaction.
> > > >> > > > >> > Of
> > > >> > > > >> > >> > > course,
> > > >> > > > >> > >> > > > > > this
> > > >> > > > >> > >> > > > > > > > > means
> > > >> > > > >> > >> > > > > > > > > >>>>> we
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> won't completely cover the case
> > > >> where
> > > >> > we
> > > >> > > > >> have a
> > > >> > > > >> > >> > really
> > > >> > > > >> > >> > > > > late
> > > >> > > > >> > >> > > > > > > > > message
> > > >> > > > >> > >> > > > > > > > > >>>>> and
> > > >> > > > >> > >> > > > > > > > > >>>>>>> we
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> have added the partition to
> > the new
> > > >> > > > >> > transaction,
> > > >> > > > >> > >> but
> > > >> > > > >> > >> > > > > that's
> > > >> > > > >> > >> > > > > > > > > >>>>>> unfortunately
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> something we will need the new
> > > >> clients
> > > >> > > to
> > > >> > > > >> > cover.
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> 30)
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> Transaction is ongoing =
> > partition
> > > >> was
> > > >> > > > >> added to
> > > >> > > > >> > >> > > > > transaction
> > > >> > > > >> > >> > > > > > > via
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> addPartitionsToTxn. We check
> > this
> > > >> with
> > > >> > > the
> > > >> > > > >> > >> > > > > > > DescribeTransactions
> > > >> > > > >> > >> > > > > > > > > >>> call.
> > > >> > > > >> > >> > > > > > > > > >>>>>> Let
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> me know if this wasn't
> > sufficiently
> > > >> > > > >> explained
> > > >> > > > >> > >> here:
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > >
> > > >> > > > >> > >> > > > > > > >
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > >
> > > >> > > > >> > >> > > > >
> > > >> > > > >> > >> > > >
> > > >> > > > >> > >> > >
> > > >> > > > >> > >> >
> > > >> > > > >> > >>
> > > >> > > > >> >
> > > >> > > > >>
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> 40)
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> The idea here is that if any
> > > >> messages
> > > >> > > > >> somehow
> > > >> > > > >> > >> come
> > > >> > > > >> > >> > in
> > > >> > > > >> > >> > > > > before
> > > >> > > > >> > >> > > > > > > we
> > > >> > > > >> > >> > > > > > > > > get
> > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > >> > > > >> > >> > > > > > > > > >>>>>>> new
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> epoch to the producer, they
> > will be
> > > >> > > > fenced.
> > > >> > > > >> > >> However,
> > > >> > > > >> > >> > > if
> > > >> > > > >> > >> > > > we
> > > >> > > > >> > >> > > > > > > don't
> > > >> > > > >> > >> > > > > > > > > >>>>> think
> > > >> > > > >> > >> > > > > > > > > >>>>>>> this
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> is necessary, it can be
> > discussed
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> 50)
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> It should be synchronous
> > because
> > > >> if we
> > > >> > > > have
> > > >> > > > >> an
> > > >> > > > >> > >> event
> > > >> > > > >> > >> > > > (ie,
> > > >> > > > >> > >> > > > > an
> > > >> > > > >> > >> > > > > > > > > error)
> > > >> > > > >> > >> > > > > > > > > >>>>>> that
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> causes us to need to abort the
> > > >> > > > transaction,
> > > >> > > > >> we
> > > >> > > > >> > >> need
> > > >> > > > >> > >> > to
> > > >> > > > >> > >> > > > > know
> > > >> > > > >> > >> > > > > > > > which
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> partitions to send transaction
> > > >> markers
> > > >> > > to.
> > > >> > > > >> We
> > > >> > > > >> > >> know
> > > >> > > > >> > >> > the
> > > >> > > > >> > >> > > > > > > > partitions
> > > >> > > > >> > >> > > > > > > > > >>>>>> because
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> we added them to the
> > coordinator
> > > >> via
> > > >> > the
> > > >> > > > >> > >> > > > > addPartitionsToTxn
> > > >> > > > >> > >> > > > > > > > call.
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> Previously we have had
> > asynchronous
> > > >> > > calls
> > > >> > > > in
> > > >> > > > >> > the
> > > >> > > > >> > >> > past
> > > >> > > > >> > >> > > > (ie,
> > > >> > > > >> > >> > > > > > > > writing
> > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> commit markers when the
> > > >> transaction is
> > > >> > > > >> > completed)
> > > >> > > > >> > >> > but
> > > >> > > > >> > >> > > > > often
> > > >> > > > >> > >> > > > > > > this
> > > >> > > > >> > >> > > > > > > > > >>> just
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> causes confusion as we need to
> > wait
> > > >> > for
> > > >> > > > some
> > > >> > > > >> > >> > > operations
> > > >> > > > >> > >> > > > to
> > > >> > > > >> > >> > > > > > > > > complete.
> > > >> > > > >> > >> > > > > > > > > >>>>> In
> > > >> > > > >> > >> > > > > > > > > >>>>>>> the
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> writing commit markers case,
> > > >> clients
> > > >> > > often
> > > >> > > > >> see
> > > >> > > > >> > >> > > > > > > > > >>>>> CONCURRENT_TRANSACTIONs
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> error messages and that can be
> > > >> > > confusing.
> > > >> > > > >> For
> > > >> > > > >> > >> that
> > > >> > > > >> > >> > > > reason,
> > > >> > > > >> > >> > > > > > it
> > > >> > > > >> > >> > > > > > > > may
> > > >> > > > >> > >> > > > > > > > > be
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> simpler to just have
> > synchronous
> > > >> > calls —
> > > >> > > > >> > >> especially
> > > >> > > > >> > >> > if
> > > >> > > > >> > >> > > > we
> > > >> > > > >> > >> > > > > > need
> > > >> > > > >> > >> > > > > > > > to
> > > >> > > > >> > >> > > > > > > > > >>>>> block
> > > >> > > > >> > >> > > > > > > > > >>>>>>> on
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> some operation's completion
> > anyway
> > > >> > > before
> > > >> > > > we
> > > >> > > > >> > can
> > > >> > > > >> > >> > start
> > > >> > > > >> > >> > > > the
> > > >> > > > >> > >> > > > > > > next
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> transaction. And yes, I meant
> > > >> > > > coordinator. I
> > > >> > > > >> > will
> > > >> > > > >> > >> > fix
> > > >> > > > >> > >> > > > > that.
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> 60)
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> When we are checking if the
> > > >> > transaction
> > > >> > > is
> > > >> > > > >> > >> ongoing,
> > > >> > > > >> > >> > we
> > > >> > > > >> > >> > > > > need
> > > >> > > > >> > >> > > > > > to
> > > >> > > > >> > >> > > > > > > > > make
> > > >> > > > >> > >> > > > > > > > > >>> a
> > > >> > > > >> > >> > > > > > > > > >>>>>>> round
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> trip from the leader partition
> > to
> > > >> the
> > > >> > > > >> > transaction
> > > >> > > > >> > >> > > > > > coordinator.
> > > >> > > > >> > >> > > > > > > > In
> > > >> > > > >> > >> > > > > > > > > >>> the
> > > >> > > > >> > >> > > > > > > > > >>>>>>> time
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> we are waiting for this
> > message to
> > > >> > come
> > > >> > > > >> back,
> > > >> > > > >> > in
> > > >> > > > >> > >> > > theory
> > > >> > > > >> > >> > > > we
> > > >> > > > >> > >> > > > > > > could
> > > >> > > > >> > >> > > > > > > > > >>> have
> > > >> > > > >> > >> > > > > > > > > >>>>>>> sent
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> a commit/abort call that would
> > make
> > > >> > the
> > > >> > > > >> > original
> > > >> > > > >> > >> > > result
> > > >> > > > >> > >> > > > of
> > > >> > > > >> > >> > > > > > the
> > > >> > > > >> > >> > > > > > > > > check
> > > >> > > > >> > >> > > > > > > > > >>>>>> out
> > > >> > > > >> > >> > > > > > > > > >>>>>>> of
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> date. That is why we can check
> > the
> > > >> > > leader
> > > >> > > > >> state
> > > >> > > > >> > >> > before
> > > >> > > > >> > >> > > > we
> > > >> > > > >> > >> > > > > > > write
> > > >> > > > >> > >> > > > > > > > to
> > > >> > > > >> > >> > > > > > > > > >>>>> the
> > > >> > > > >> > >> > > > > > > > > >>>>>>> log.
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> I'm happy to update the KIP if
> > > >> some of
> > > >> > > > these
> > > >> > > > >> > >> things
> > > >> > > > >> > >> > > were
> > > >> > > > >> > >> > > > > not
> > > >> > > > >> > >> > > > > > > > > clear.
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks,
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> Justine
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> On Mon, Nov 21, 2022 at 7:11 PM
> > > >> > Matthias
> > > >> > > > J.
> > > >> > > > >> > Sax <
> > > >> > > > >> > >> > > > > > > > mjsax@apache.org
> > > >> > > > >> > >> > > > > > > > > >
> > > >> > > > >> > >> > > > > > > > > >>>>>>> wrote:
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Thanks for the KIP.
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Couple of clarification
> > questions
> > > >> (I
> > > >> > am
> > > >> > > > >> not a
> > > >> > > > >> > >> > broker
> > > >> > > > >> > >> > > > > expert
> > > >> > > > >> > >> > > > > > > do
> > > >> > > > >> > >> > > > > > > > > >>>>> maybe
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> some question are obvious for
> > > >> others,
> > > >> > > but
> > > >> > > > >> not
> > > >> > > > >> > >> for
> > > >> > > > >> > >> > me
> > > >> > > > >> > >> > > > with
> > > >> > > > >> > >> > > > > > my
> > > >> > > > >> > >> > > > > > > > lack
> > > >> > > > >> > >> > > > > > > > > >>>>> of
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> broker knowledge).
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (10)
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> 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.
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> What happens if the message
> > come
> > > >> in
> > > >> > > > before
> > > >> > > > >> the
> > > >> > > > >> > >> next
> > > >> > > > >> > >> > > > > > > > > >>>>>> addPartitionsToTxn
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> request? It seems the broker
> > > >> hosting
> > > >> > > the
> > > >> > > > >> data
> > > >> > > > >> > >> > > > partitions
> > > >> > > > >> > >> > > > > > > won't
> > > >> > > > >> > >> > > > > > > > > know
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> anything about it and append
> > it to
> > > >> > the
> > > >> > > > >> > >> partition,
> > > >> > > > >> > >> > > too?
> > > >> > > > >> > >> > > > > What
> > > >> > > > >> > >> > > > > > > is
> > > >> > > > >> > >> > > > > > > > > the
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> difference between both cases?
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Also, it seems a TX would only
> > > >> hang,
> > > >> > if
> > > >> > > > >> there
> > > >> > > > >> > >> is no
> > > >> > > > >> > >> > > > > > following
> > > >> > > > >> > >> > > > > > > > TX
> > > >> > > > >> > >> > > > > > > > > >>>>> that
> > > >> > > > >> > >> > > > > > > > > >>>>>>> is
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> either committer or aborted?
> > Thus,
> > > >> > for
> > > >> > > > the
> > > >> > > > >> > case
> > > >> > > > >> > >> > > above,
> > > >> > > > >> > >> > > > > the
> > > >> > > > >> > >> > > > > > TX
> > > >> > > > >> > >> > > > > > > > > might
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> actually not hang (of course,
> > we
> > > >> > might
> > > >> > > > get
> > > >> > > > >> an
> > > >> > > > >> > >> EOS
> > > >> > > > >> > >> > > > > violation
> > > >> > > > >> > >> > > > > > > if
> > > >> > > > >> > >> > > > > > > > > the
> > > >> > > > >> > >> > > > > > > > > >>>>>>> first
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> TX was aborted and the second
> > > >> > > committed,
> > > >> > > > or
> > > >> > > > >> > the
> > > >> > > > >> > >> > other
> > > >> > > > >> > >> > > > way
> > > >> > > > >> > >> > > > > > > > > around).
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (20)
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Of course, 1 and 2 require
> > > >> > client-side
> > > >> > > > >> > >> changes, so
> > > >> > > > >> > >> > > for
> > > >> > > > >> > >> > > > > > older
> > > >> > > > >> > >> > > > > > > > > >>>>>> clients,
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> those approaches won’t apply.
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> For (1) I understand why a
> > client
> > > >> > > change
> > > >> > > > is
> > > >> > > > >> > >> > > necessary,
> > > >> > > > >> > >> > > > > but
> > > >> > > > >> > >> > > > > > > not
> > > >> > > > >> > >> > > > > > > > > sure
> > > >> > > > >> > >> > > > > > > > > >>>>>> why
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> we need a client change for
> > (2).
> > > >> Can
> > > >> > > you
> > > >> > > > >> > >> elaborate?
> > > >> > > > >> > >> > > --
> > > >> > > > >> > >> > > > > > Later
> > > >> > > > >> > >> > > > > > > > you
> > > >> > > > >> > >> > > > > > > > > >>>>>>> explain
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> that we should send a
> > > >> > > > >> > >> DescribeTransactionRequest,
> > > >> > > > >> > >> > > but I
> > > >> > > > >> > >> > > > > am
> > > >> > > > >> > >> > > > > > > not
> > > >> > > > >> > >> > > > > > > > > sure
> > > >> > > > >> > >> > > > > > > > > >>>>>>> why?
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Can't we not just do an
> > implicit
> > > >> > > > >> > >> AddPartiitonToTx,
> > > >> > > > >> > >> > > too?
> > > >> > > > >> > >> > > > > If
> > > >> > > > >> > >> > > > > > > the
> > > >> > > > >> > >> > > > > > > > > old
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> producer correctly registered
> > the
> > > >> > > > partition
> > > >> > > > >> > >> > already,
> > > >> > > > >> > >> > > > the
> > > >> > > > >> > >> > > > > > > > > >>>>>> TX-coordinator
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> can just ignore it as it's an
> > > >> > > idempotent
> > > >> > > > >> > >> operation?
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (30)
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> To cover older clients, we
> > will
> > > >> > > ensure a
> > > >> > > > >> > >> > transaction
> > > >> > > > >> > >> > > > is
> > > >> > > > >> > >> > > > > > > > ongoing
> > > >> > > > >> > >> > > > > > > > > >>>>>>> before
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> we write to a transaction
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Not sure what you mean by
> > this?
> > > >> Can
> > > >> > you
> > > >> > > > >> > >> elaborate?
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (40)
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> [the TX-coordinator] will
> > write
> > > >> the
> > > >> > > > >> prepare
> > > >> > > > >> > >> commit
> > > >> > > > >> > >> > > > > message
> > > >> > > > >> > >> > > > > > > > with
> > > >> > > > >> > >> > > > > > > > > a
> > > >> > > > >> > >> > > > > > > > > >>>>>>>> bumped
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> epoch and send
> > > >> WriteTxnMarkerRequests
> > > >> > > > with
> > > >> > > > >> the
> > > >> > > > >> > >> > bumped
> > > >> > > > >> > >> > > > > > epoch.
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Why do we use the bumped
> > epoch for
> > > >> > > both?
> > > >> > > > It
> > > >> > > > >> > >> seems
> > > >> > > > >> > >> > > more
> > > >> > > > >> > >> > > > > > > > intuitive
> > > >> > > > >> > >> > > > > > > > > to
> > > >> > > > >> > >> > > > > > > > > >>>>>> use
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> the current epoch, and only
> > return
> > > >> > the
> > > >> > > > >> bumped
> > > >> > > > >> > >> epoch
> > > >> > > > >> > >> > > to
> > > >> > > > >> > >> > > > > the
> > > >> > > > >> > >> > > > > > > > > >>>>> producer?
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (50) "Implicit
> > > >> > > AddPartitionToTransaction"
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Why does the implicitly sent
> > > >> request
> > > >> > > need
> > > >> > > > >> to
> > > >> > > > >> > be
> > > >> > > > >> > >> > > > > > synchronous?
> > > >> > > > >> > >> > > > > > > > The
> > > >> > > > >> > >> > > > > > > > > >>>>> KIP
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> also says
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> in case we need to abort and
> > > >> need to
> > > >> > > > know
> > > >> > > > >> > which
> > > >> > > > >> > >> > > > > partitions
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> What do you mean by this?
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> we don’t want to write to it
> > > >> before
> > > >> > we
> > > >> > > > >> store
> > > >> > > > >> > in
> > > >> > > > >> > >> > the
> > > >> > > > >> > >> > > > > > > > transaction
> > > >> > > > >> > >> > > > > > > > > >>>>>>> manager
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> Do you mean TX-coordinator
> > > >> instead of
> > > >> > > > >> > "manager"?
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> (60)
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> For older clients and ensuring
> > > >> that
> > > >> > the
> > > >> > > > TX
> > > >> > > > >> is
> > > >> > > > >> > >> > > ongoing,
> > > >> > > > >> > >> > > > > you
> > > >> > > > >> > >> > > > > > > > > >>>>> describe a
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> race condition. I am not sure
> > if I
> > > >> > can
> > > >> > > > >> follow
> > > >> > > > >> > >> here.
> > > >> > > > >> > >> > > Can
> > > >> > > > >> > >> > > > > you
> > > >> > > > >> > >> > > > > > > > > >>>>>> elaborate?
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> -Matthias
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> On 11/18/22 1:21 PM, Justine
> > > >> Olshan
> > > >> > > > wrote:
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Hey all!
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> I'd like to start a
> > discussion
> > > >> on my
> > > >> > > > >> proposal
> > > >> > > > >> > >> to
> > > >> > > > >> > >> > add
> > > >> > > > >> > >> > > > > some
> > > >> > > > >> > >> > > > > > > > > >>>>>> server-side
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> checks on transactions to
> > avoid
> > > >> > > hanging
> > > >> > > > >> > >> > > transactions.
> > > >> > > > >> > >> > > > I
> > > >> > > > >> > >> > > > > > know
> > > >> > > > >> > >> > > > > > > > > this
> > > >> > > > >> > >> > > > > > > > > >>>>>> has
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> been
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> an issue for some time, so I
> > > >> really
> > > >> > > hope
> > > >> > > > >> this
> > > >> > > > >> > >> KIP
> > > >> > > > >> > >> > > will
> > > >> > > > >> > >> > > > > be
> > > >> > > > >> > >> > > > > > > > > helpful
> > > >> > > > >> > >> > > > > > > > > >>>>>> for
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>> many
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> users of EOS.
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> The KIP includes changes that
> > > >> will
> > > >> > be
> > > >> > > > >> > >> compatible
> > > >> > > > >> > >> > > with
> > > >> > > > >> > >> > > > > old
> > > >> > > > >> > >> > > > > > > > > clients
> > > >> > > > >> > >> > > > > > > > > >>>>>> and
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> changes to improve
> > performance
> > > >> and
> > > >> > > > >> > correctness
> > > >> > > > >> > >> on
> > > >> > > > >> > >> > > new
> > > >> > > > >> > >> > > > > > > clients.
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Please take a look and leave
> > any
> > > >> > > > comments
> > > >> > > > >> you
> > > >> > > > >> > >> may
> > > >> > > > >> > >> > > > have!
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> KIP:
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > >
> > > >> > > > >> > >> > > > > > > >
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > >
> > > >> > > > >> > >> > > > >
> > > >> > > > >> > >> > > >
> > > >> > > > >> > >> > >
> > > >> > > > >> > >> >
> > > >> > > > >> > >>
> > > >> > > > >> >
> > > >> > > > >>
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> JIRA:
> > > >> > > > >> > >> > > > https://issues.apache.org/jira/browse/KAFKA-14402
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Thanks!
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Justine
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>>
> > > >> > > > >> > >> > > > > > > > > >>>>
> > > >> > > > >> > >> > > > > > > > > >>>
> > > >> > > > >> > >> > > > > > > > > >>
> > > >> > > > >> > >> > > > > > > > > >
> > > >> > > > >> > >> > > > > > > > >
> > > >> > > > >> > >> > > > > > > >
> > > >> > > > >> > >> > > > > > >
> > > >> > > > >> > >> > > > > >
> > > >> > > > >> > >> > > > >
> > > >> > > > >> > >> > > >
> > > >> > > > >> > >> > >
> > > >> > > > >> > >> >
> > > >> > > > >> > >>
> > > >> > > > >> > >
> > > >> > > > >> >
> > > >> > > > >>
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > > >
> >

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
Hey Guozhang. Thanks for taking a look and for the detailed comments! I'll
do my best to address below.

1. I see what you are saying here, but I think I need to look through the
sequence of events you mention. Typically we've seen this issue in a few
cases.

 One is when we have a producer disconnect when trying to produce.
Typically in these cases, we abort the transaction. We've seen that after
the markers are written, the disconnection can sometimes cause the request
to get flushed to the broker. In this case, we don't need client handling
because the producer we are responding to is gone. We just needed to make
sure we didn't write to the log on the broker side. I'm trying to think of
a case where we do have the client to return to. I'd think the same client
couldn't progress to committing the transaction unless the produce request
returned right? Of course, there is the incorrectly written clients case.
I'll think on this a bit more and let you know if I come up with another
scenario when we would return to an active client when the transaction is
no longer ongoing.

I was not aware that we checked the result of a send after we commit
though. I'll need to look into that a bit more.

2. There were some questions about this in the discussion. The plan is to
handle overflow with the mechanism we currently have in the producer. If we
try to bump and the epoch will overflow, we actually allocate a new
producer ID. I need to confirm the fencing logic on the last epoch (ie, we
probably shouldn't allow any records to be produced with the final epoch
since we can never properly fence that one).

3. I can agree with you that the current error handling is messy. I recall
taking a look at your KIP a while back, but I think I mostly saw the
section about how the errors were wrapped. Maybe I need to take another
look. As for abortable error, the idea was that the handling would be
simple -- if this error is seen, the transaction should be aborted -- no
other logic about previous state or requests necessary. Is your concern
simply about adding new errors? We were hoping to have an error that would
have one meaning and many of the current errors have a history of meaning
different things on different client versions. That was the main motivation
for adding a new error.

4. This is a good point about record timestamp reordering. Timestamps don't
affect compaction, but they do affect retention deletion. For that, kafka
considers the largest timestamp in the segment, so I think a small amount
of reordering (hopefully on the order of milliseconds or even seconds) will
be ok. We take timestamps from clients so there is already a possibility
for some drift and non-monotonically increasing timestamps.

5. Thanks for catching. The error is there, but it's actually that those
fields should be 4+! Due to how the message generator works, I actually
have to redefine those fields inside the `"AddPartitionsToTxnTransaction`
block for it to build correctly. I'll fix it to be correct.

6. Correct -- we will only add the request to purgatory if the cache has no
ongoing transaction. I can change the wording to make that clearer that we
only place the request in purgatory if we need to contact the transaction
coordinator.

7. We did take a look at some of the errors and it was hard to come up with
a good one. I agree that InvalidTxnStateException is ideal except for the
fact that it hasn't been returned on Produce requests before. The error
handling for clients is a bit vague (which is why I opened KAFKA-14439
<https://issues.apache.org/jira/browse/KAFKA-14439>), but the decision we
made here was to only return errors that have been previously returned to
producers. As for not being fatal, I think part of the theory was that in
many cases, the producer would be disconnected. (See point 1) and this
would just be an error to return from the server. I did plan to think about
other cases, so let me know if you think of any as well!

Lots to say! Let me know if you have further thoughts!
Justine

On Fri, Jan 20, 2023 at 11:21 AM Guozhang Wang <gu...@gmail.com>
wrote:

> Hello Justine,
>
> Thanks for the great write-up! I made a quick pass through it and here
> are some thoughts (I have not been able to read through this thread so
> pardon me if they have overlapped or subsumed by previous comments):
>
> First are some meta ones:
>
> 1. I think we need to also improve the client's experience once we
> have this defence in place. More concretely, say a user's producer
> code is like following:
>
> future = producer.send();
> // producer.flush();
> producer.commitTransaction();
> future.get();
>
> Which resulted in the order of a) produce-request sent by producer, b)
> end-txn-request sent by producer, c) end-txn-response sent back, d)
> txn-marker-request sent from coordinator to partition leader, e)
> produce-request finally received by the partition leader, before this
> KIP e) step would be accepted causing a dangling txn; now it would be
> rejected in step e) which is good. But from the client's point of view
> now it becomes confusing since the `commitTransaction()` returns
> successfully, but the "future" throws an invalid-epoch error, and they
> are not sure if the transaction did succeed or not. In fact, it
> "partially succeeded" with some msgs being rejected but others
> committed successfully.
>
> Of course the easy way to avoid this is, always call
> "producer.flush()" before commitTxn and that's what we do ourselves,
> and what we recommend users do. But I suspect not everyone does it. In
> fact I just checked the javadoc in KafkaProducer and our code snippet
> does not include a `flush()` call. So I'm thinking maybe we can in
> side the `commitTxn` code to enforce flushing before sending the
> end-txn request.
>
> 2. I'd like to clarify a bit details on "just add partitions to the
> transaction on the first produce request during a transaction". My
> understanding is that the partition leader's cache has the producer id
> / sequence / epoch for the latest txn, either on-going or is completed
> (upon receiving the marker request from coordinator). When a produce
> request is received, if
>
> * producer's epoch < cached epoch, or producer's epoch == cached epoch
> but the latest txn is completed, leader directly reject with
> invalid-epoch.
> * producer's epoch > cached epoch, park the the request and send
> add-partitions request to coordinator.
>
> In order to do it, does the coordinator need to bump the sequence and
> reset epoch to 0 when the next epoch is going to overflow? If no need
> to do so, then how we handle the (admittedly rare, but still may
> happen) epoch overflow situation?
>
> 3. I'm a bit concerned about adding a generic "ABORTABLE_ERROR" given
> we already have a pretty messy error classification and error handling
> on the producer clients side --- I have a summary about the issues and
> a proposal to address this in
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
> -- I understand we do not want to use "UNKNOWN_PRODUCER_ID" anymore
> and in fact we intend to deprecate it in KIP-360 and eventually remove
> it; but I'm wondering can we still use specific error codes. E.g. what
> about "InvalidProducerEpochException" since for new clients, the
> actual reason this would actually be rejected is indeed because the
> epoch on the coordinator caused the add-partitions-request from the
> brokers to be rejected anyways?
>
> 4. It seems we put the producer request into purgatory before we ever
> append the records, while other producer's records may still be
> appended during the time; and that potentially may result in some
> re-ordering compared with reception order. I'm not super concerned
> about it since Kafka does not guarantee reception ordering across
> producers anyways, but it may make the timestamps of records inside a
> partition to be more out-of-ordered. Are we aware of any scenarios
> such as future enhancements on log compactions that may be affected by
> this effect?
>
> Below are just minor comments:
>
> 5. In "AddPartitionsToTxnTransaction" field of
> "AddPartitionsToTxnRequest" RPC, the versions of those inner fields
> are "0-3" while I thought they should be "0+" still?
>
> 6. Regarding "we can place the request in a purgatory of sorts and
> check if there is any state for the transaction on the broker": i
> think at this time when we just do the checks against the cached
> state, we do not need to put the request to purgatory yet?
>
> 7. This is related to 3) above. I feel using "InvalidRecordException"
> for older clients may also be a bit confusing, and also it is not
> fatal -- for old clients, it better to be fatal since this indicates
> the clients is doing something wrong and hence it should be closed.
> And in general I'd prefer to use slightly more specific meaning error
> codes for clients. That being said, I also feel
> "InvalidProducerEpochException" is not suitable for old versioned
> clients, and we'd have to pick one that old clients recognize. I'd
> prefer "InvalidTxnStateException" but that one is supposed to be
> returned from txn coordinators only today. I'd suggest we do a quick
> check in the current client's code path and see if that one would be
> handled if it's from a produce-response, and if yes, use this one;
> otherwise, use "ProducerFencedException" which is much less meaningful
> but it's still a fatal error.
>
>
> Thanks,
> Guozhang
>
>
>
> On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
> <jo...@confluent.io.invalid> wrote:
> >
> > Yeah -- looks like we already have code to handle bumping the epoch and
> > when the epoch is Short.MAX_VALUE, we get a new producer ID. Since this
> is
> > already the behavior, do we want to change it further?
> >
> > Justine
> >
> > On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <jo...@confluent.io>
> wrote:
> >
> > > Hey all, just wanted to quickly update and say I've modified the KIP to
> > > explicitly mention that AddOffsetCommitsToTxnRequest will be replaced
> by
> > > a coordinator-side (inter-broker) AddPartitionsToTxn implicit request.
> This
> > > mirrors the user partitions and will implicitly add offset partitions
> to
> > > transactions when we commit offsets on them. We will deprecate
> AddOffsetCommitsToTxnRequest
> > > for new clients.
> > >
> > > Also to address Artem's comments --
> > > I'm a bit unsure if the changes here will change the previous behavior
> for
> > > fencing producers. In the case you mention in the first paragraph, are
> you
> > > saying we bump the epoch before we try to abort the transaction? I
> think I
> > > need to understand the scenarios you mention a bit better.
> > >
> > > As for the second part -- I think it makes sense to have some sort of
> > > "sentinel" epoch to signal epoch is about to overflow (I think we sort
> of
> > > have this value in place in some ways) so we can codify it in the KIP.
> I'll
> > > look into that and try to update soon.
> > >
> > > Thanks,
> > > Justine.
> > >
> > > On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
> > > <al...@confluent.io.invalid> wrote:
> > >
> > >> It's good to know that KIP-588 addressed some of the issues.  Looking
> at
> > >> the code, it still looks like there are some cases that would result
> in
> > >> fatal error, e.g. PRODUCER_FENCED is issued by the transaction
> coordinator
> > >> if epoch doesn't match, and the client treats it as a fatal error
> (code in
> > >> TransactionManager request handling).  If we consider, for example,
> > >> committing a transaction that returns a timeout, but actually
> succeeds,
> > >> trying to abort it or re-commit may result in PRODUCER_FENCED error
> > >> (because of epoch bump).
> > >>
> > >> For failed commits, specifically, we need to know the actual outcome,
> > >> because if we return an error the application may think that the
> > >> transaction is aborted and redo the work, leading to duplicates.
> > >>
> > >> Re: overflowing epoch.  We could either do it on the TC and return
> both
> > >> producer id and epoch (e.g. change the protocol), or signal the client
> > >> that
> > >> it needs to get a new producer id.  Checking for max epoch could be a
> > >> reasonable signal, the value to check should probably be present in
> the
> > >> KIP
> > >> as this is effectively a part of the contract.  Also, the TC should
> > >> probably return an error if the client didn't change producer id after
> > >> hitting max epoch.
> > >>
> > >> -Artem
> > >>
> > >>
> > >> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
> > >> <jo...@confluent.io.invalid> wrote:
> > >>
> > >> > Thanks for the discussion Artem.
> > >> >
> > >> > With respect to the handling of fenced producers, we have some
> behavior
> > >> > already in place. As of KIP-588:
> > >> >
> > >> >
> > >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> > >> > ,
> > >> > we handle timeouts more gracefully. The producer can recover.
> > >> >
> > >> > Produce requests can also recover from epoch fencing by aborting the
> > >> > transaction and starting over.
> > >> >
> > >> > What other cases were you considering that would cause us to have a
> > >> fenced
> > >> > epoch but we'd want to recover?
> > >> >
> > >> > The first point about handling epoch overflows is fair. I think
> there is
> > >> > some logic we'd need to consider. (ie, if we are one away from the
> max
> > >> > epoch, we need to reset the producer ID.) I'm still wondering if
> there
> > >> is a
> > >> > way to direct this from the response, or if everything should be
> done on
> > >> > the client side. Let me know if you have any thoughts here.
> > >> >
> > >> > Thanks,
> > >> > Justine
> > >> >
> > >> > On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
> > >> > <al...@confluent.io.invalid> wrote:
> > >> >
> > >> > > There are some workflows in the client that are implied by
> protocol
> > >> > > changes, e.g.:
> > >> > >
> > >> > > - for new clients, epoch changes with every transaction and can
> > >> overflow,
> > >> > > in old clients this condition was handled transparently, because
> epoch
> > >> > was
> > >> > > bumped in InitProducerId and it would return a new producer id if
> > >> epoch
> > >> > > overflows, the new clients would need to implement some workflow
> to
> > >> > refresh
> > >> > > producer id
> > >> > > - how to handle fenced producers, for new clients epoch changes
> with
> > >> > every
> > >> > > transaction, so in presence of failures during commits / aborts,
> the
> > >> > > producer could get easily fenced, old clients would pretty much
> would
> > >> get
> > >> > > fenced when a new incarnation of the producer was initialized with
> > >> > > InitProducerId so it's ok to treat as a fatal error, the new
> clients
> > >> > would
> > >> > > need to implement some workflow to handle that error, otherwise
> they
> > >> > could
> > >> > > get fenced by themselves
> > >> > > - in particular (as a subset of the previous issue), what would
> the
> > >> > client
> > >> > > do if it got a timeout during commit?  commit could've succeeded
> or
> > >> > failed
> > >> > >
> > >> > > Not sure if this has to be defined in the KIP as implementing
> those
> > >> > > probably wouldn't require protocol changes, but we have multiple
> > >> > > implementations of Kafka clients, so probably would be good to
> have
> > >> some
> > >> > > client implementation guidance.  Could also be done as a separate
> doc.
> > >> > >
> > >> > > -Artem
> > >> > >
> > >> > > On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
> > >> > <jolshan@confluent.io.invalid
> > >> > > >
> > >> > > wrote:
> > >> > >
> > >> > > > Hey all, I've updated the KIP to incorporate Jason's
> suggestions.
> > >> > > >
> > >> > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > >> > > >
> > >> > > >
> > >> > > > 1. Use AddPartitionsToTxn + verify flag to check on old clients
> > >> > > > 2. Updated AddPartitionsToTxn API to support transaction
> batching
> > >> > > > 3. Mention IBP bump
> > >> > > > 4. Mention auth change on new AddPartitionsToTxn version.
> > >> > > >
> > >> > > > I'm planning on opening a vote soon.
> > >> > > > Thanks,
> > >> > > > Justine
> > >> > > >
> > >> > > > On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <
> jolshan@confluent.io
> > >> >
> > >> > > > wrote:
> > >> > > >
> > >> > > > > Thanks Jason. Those changes make sense to me. I will update
> the
> > >> KIP.
> > >> > > > >
> > >> > > > >
> > >> > > > >
> > >> > > > > On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
> > >> > > > <ja...@confluent.io.invalid>
> > >> > > > > wrote:
> > >> > > > >
> > >> > > > >> Hey Justine,
> > >> > > > >>
> > >> > > > >> > I was wondering about compatibility here. When we send
> requests
> > >> > > > >> between brokers, we want to ensure that the receiving broker
> > >> > > understands
> > >> > > > >> the request (specifically the new fields). Typically this is
> done
> > >> > via
> > >> > > > >> IBP/metadata version.
> > >> > > > >> I'm trying to think if there is a way around it but I'm not
> sure
> > >> > there
> > >> > > > is.
> > >> > > > >>
> > >> > > > >> Yes. I think we would gate usage of this behind an IBP bump.
> Does
> > >> > that
> > >> > > > >> seem
> > >> > > > >> reasonable?
> > >> > > > >>
> > >> > > > >> > As for the improvements -- can you clarify how the multiple
> > >> > > > >> transactional
> > >> > > > >> IDs would help here? Were you thinking of a case where we
> > >> wait/batch
> > >> > > > >> multiple produce requests together? My understanding for now
> was
> > >> 1
> > >> > > > >> transactional ID and one validation per 1 produce request.
> > >> > > > >>
> > >> > > > >> Each call to `AddPartitionsToTxn` is essentially a write to
> the
> > >> > > > >> transaction
> > >> > > > >> log and must block on replication. The more we can fit into a
> > >> single
> > >> > > > >> request, the more writes we can do in parallel. The
> alternative
> > >> is
> > >> > to
> > >> > > > make
> > >> > > > >> use of more connections, but usually we prefer batching
> since the
> > >> > > > network
> > >> > > > >> stack is not really optimized for high connection/request
> loads.
> > >> > > > >>
> > >> > > > >> > Finally with respect to the authorizations, I think it
> makes
> > >> sense
> > >> > > to
> > >> > > > >> skip
> > >> > > > >> topic authorizations, but I'm a bit confused by the "leader
> ID"
> > >> > field.
> > >> > > > >> Wouldn't we just want to flag the request as from a broker
> (does
> > >> it
> > >> > > > matter
> > >> > > > >> which one?).
> > >> > > > >>
> > >> > > > >> We could also make it version-based. For the next version, we
> > >> could
> > >> > > > >> require
> > >> > > > >> CLUSTER auth. So clients would not be able to use the API
> > >> anymore,
> > >> > > which
> > >> > > > >> is
> > >> > > > >> probably what we want.
> > >> > > > >>
> > >> > > > >> -Jason
> > >> > > > >>
> > >> > > > >> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> > >> > > > >> <jo...@confluent.io.invalid>
> > >> > > > >> wrote:
> > >> > > > >>
> > >> > > > >> > As a follow up, I was just thinking about the batching a
> bit
> > >> more.
> > >> > > > >> > I suppose if we have one request in flight and we queue up
> the
> > >> > other
> > >> > > > >> > produce requests in some sort of purgatory, we could send
> > >> > > information
> > >> > > > >> out
> > >> > > > >> > for all of them rather than one by one. So that would be a
> > >> benefit
> > >> > > of
> > >> > > > >> > batching partitions to add per transaction.
> > >> > > > >> >
> > >> > > > >> > I'll need to think a bit more on the design of this part
> of the
> > >> > KIP,
> > >> > > > and
> > >> > > > >> > will update the KIP in the next few days.
> > >> > > > >> >
> > >> > > > >> > Thanks,
> > >> > > > >> > Justine
> > >> > > > >> >
> > >> > > > >> > On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan <
> > >> > > jolshan@confluent.io>
> > >> > > > >> > wrote:
> > >> > > > >> >
> > >> > > > >> > > Hey Jason -- thanks for the input -- I was just digging
> a bit
> > >> > > deeper
> > >> > > > >> into
> > >> > > > >> > > the design + implementation of the validation calls here
> and
> > >> > what
> > >> > > > you
> > >> > > > >> say
> > >> > > > >> > > makes sense.
> > >> > > > >> > >
> > >> > > > >> > > I was wondering about compatibility here. When we send
> > >> requests
> > >> > > > >> > > between brokers, we want to ensure that the receiving
> broker
> > >> > > > >> understands
> > >> > > > >> > > the request (specifically the new fields). Typically
> this is
> > >> > done
> > >> > > > via
> > >> > > > >> > > IBP/metadata version.
> > >> > > > >> > > I'm trying to think if there is a way around it but I'm
> not
> > >> sure
> > >> > > > there
> > >> > > > >> > is.
> > >> > > > >> > >
> > >> > > > >> > > As for the improvements -- can you clarify how the
> multiple
> > >> > > > >> transactional
> > >> > > > >> > > IDs would help here? Were you thinking of a case where we
> > >> > > wait/batch
> > >> > > > >> > > multiple produce requests together? My understanding for
> now
> > >> > was 1
> > >> > > > >> > > transactional ID and one validation per 1 produce
> request.
> > >> > > > >> > >
> > >> > > > >> > > Finally with respect to the authorizations, I think it
> makes
> > >> > sense
> > >> > > > to
> > >> > > > >> > skip
> > >> > > > >> > > topic authorizations, but I'm a bit confused by the
> "leader
> > >> ID"
> > >> > > > field.
> > >> > > > >> > > Wouldn't we just want to flag the request as from a
> broker
> > >> (does
> > >> > > it
> > >> > > > >> > matter
> > >> > > > >> > > which one?).
> > >> > > > >> > >
> > >> > > > >> > > I think I want to adopt these suggestions, just had a few
> > >> > > questions
> > >> > > > on
> > >> > > > >> > the
> > >> > > > >> > > details.
> > >> > > > >> > >
> > >> > > > >> > > Thanks,
> > >> > > > >> > > Justine
> > >> > > > >> > >
> > >> > > > >> > > On Thu, Jan 5, 2023 at 5:05 PM Jason Gustafson
> > >> > > > >> > <ja...@confluent.io.invalid>
> > >> > > > >> > > wrote:
> > >> > > > >> > >
> > >> > > > >> > >> Hi Justine,
> > >> > > > >> > >>
> > >> > > > >> > >> Thanks for the proposal.
> > >> > > > >> > >>
> > >> > > > >> > >> I was thinking about the implementation a little bit.
> In the
> > >> > > > current
> > >> > > > >> > >> proposal, the behavior depends on whether we have an
> old or
> > >> new
> > >> > > > >> client.
> > >> > > > >> > >> For
> > >> > > > >> > >> old clients, we send `DescribeTransactions` and verify
> the
> > >> > result
> > >> > > > and
> > >> > > > >> > for
> > >> > > > >> > >> new clients, we send `AddPartitionsToTxn`. We might be
> able
> > >> to
> > >> > > > >> simplify
> > >> > > > >> > >> the
> > >> > > > >> > >> implementation if we can use the same request type. For
> > >> > example,
> > >> > > > >> what if
> > >> > > > >> > >> we
> > >> > > > >> > >> bump the protocol version for `AddPartitionsToTxn` and
> add a
> > >> > > > >> > >> `validateOnly`
> > >> > > > >> > >> flag? For older versions, we can set
> `validateOnly=true` so
> > >> > that
> > >> > > > the
> > >> > > > >> > >> request only returns successfully if the partition had
> > >> already
> > >> > > been
> > >> > > > >> > added.
> > >> > > > >> > >> For new versions, we can set `validateOnly=false` and
> the
> > >> > > partition
> > >> > > > >> will
> > >> > > > >> > >> be
> > >> > > > >> > >> added to the transaction. The other slightly annoying
> thing
> > >> > that
> > >> > > > this
> > >> > > > >> > >> would
> > >> > > > >> > >> get around is the need to collect the transaction state
> for
> > >> all
> > >> > > > >> > partitions
> > >> > > > >> > >> even when we only care about a subset.
> > >> > > > >> > >>
> > >> > > > >> > >> Some additional improvements to consider:
> > >> > > > >> > >>
> > >> > > > >> > >> - We can give `AddPartitionsToTxn` better batch support
> for
> > >> > > > >> inter-broker
> > >> > > > >> > >> usage. Currently we only allow one `TransactionalId` to
> be
> > >> > > > specified,
> > >> > > > >> > but
> > >> > > > >> > >> the broker may get some benefit being able to batch
> across
> > >> > > multiple
> > >> > > > >> > >> transactions.
> > >> > > > >> > >> - Another small improvement is skipping topic
> authorization
> > >> > > checks
> > >> > > > >> for
> > >> > > > >> > >> `AddPartitionsToTxn` when the request is from a broker.
> > >> Perhaps
> > >> > > we
> > >> > > > >> can
> > >> > > > >> > add
> > >> > > > >> > >> a field for the `LeaderId` or something like that and
> > >> require
> > >> > > > CLUSTER
> > >> > > > >> > >> permission when set.
> > >> > > > >> > >>
> > >> > > > >> > >> Best,
> > >> > > > >> > >> Jason
> > >> > > > >> > >>
> > >> > > > >> > >>
> > >> > > > >> > >>
> > >> > > > >> > >> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
> > >> > <jun@confluent.io.invalid
> > >> > > >
> > >> > > > >> > wrote:
> > >> > > > >> > >>
> > >> > > > >> > >> > Hi, Justine,
> > >> > > > >> > >> >
> > >> > > > >> > >> > Thanks for the explanation. It makes sense to me now.
> > >> > > > >> > >> >
> > >> > > > >> > >> > Jun
> > >> > > > >> > >> >
> > >> > > > >> > >> > On Mon, Dec 19, 2022 at 1:42 PM Justine Olshan
> > >> > > > >> > >> > <jo...@confluent.io.invalid>
> > >> > > > >> > >> > wrote:
> > >> > > > >> > >> >
> > >> > > > >> > >> > > Hi Jun,
> > >> > > > >> > >> > >
> > >> > > > >> > >> > > My understanding of the mechanism is that when we
> get to
> > >> > the
> > >> > > > last
> > >> > > > >> > >> epoch,
> > >> > > > >> > >> > we
> > >> > > > >> > >> > > increment to the fencing/last epoch and if any
> further
> > >> > > requests
> > >> > > > >> come
> > >> > > > >> > >> in
> > >> > > > >> > >> > for
> > >> > > > >> > >> > > this producer ID they are fenced. Then the producer
> > >> gets a
> > >> > > new
> > >> > > > ID
> > >> > > > >> > and
> > >> > > > >> > >> > > restarts with epoch/sequence 0. The fenced epoch
> sticks
> > >> > > around
> > >> > > > >> for
> > >> > > > >> > the
> > >> > > > >> > >> > > duration of producer.id.expiration.ms and blocks
> any
> > >> late
> > >> > > > >> messages
> > >> > > > >> > >> > there.
> > >> > > > >> > >> > > The new ID will get to take advantage of the
> improved
> > >> > > semantics
> > >> > > > >> > around
> > >> > > > >> > >> > > non-zero start sequences. So I think we are covered.
> > >> > > > >> > >> > >
> > >> > > > >> > >> > > The only potential issue is overloading the cache,
> but
> > >> > > > hopefully
> > >> > > > >> the
> > >> > > > >> > >> > > improvements (lowered producer.id.expiration.ms)
> will
> > >> help
> > >> > > > with
> > >> > > > >> > that.
> > >> > > > >> > >> > Let
> > >> > > > >> > >> > > me know if you still have concerns.
> > >> > > > >> > >> > >
> > >> > > > >> > >> > > Thanks,
> > >> > > > >> > >> > > Justine
> > >> > > > >> > >> > >
> > >> > > > >> > >> > > On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
> > >> > > > >> <ju...@confluent.io.invalid>
> > >> > > > >> > >> > wrote:
> > >> > > > >> > >> > >
> > >> > > > >> > >> > > > Hi, Justine,
> > >> > > > >> > >> > > >
> > >> > > > >> > >> > > > Thanks for the explanation.
> > >> > > > >> > >> > > >
> > >> > > > >> > >> > > > 70. The proposed fencing logic doesn't apply when
> pid
> > >> > > > changes,
> > >> > > > >> is
> > >> > > > >> > >> that
> > >> > > > >> > >> > > > right? If so, I am not sure how complete we are
> > >> > addressing
> > >> > > > this
> > >> > > > >> > >> issue
> > >> > > > >> > >> > if
> > >> > > > >> > >> > > > the pid changes more frequently.
> > >> > > > >> > >> > > >
> > >> > > > >> > >> > > > Thanks,
> > >> > > > >> > >> > > >
> > >> > > > >> > >> > > > Jun
> > >> > > > >> > >> > > >
> > >> > > > >> > >> > > >
> > >> > > > >> > >> > > >
> > >> > > > >> > >> > > > On Fri, Dec 16, 2022 at 9:16 AM Justine Olshan
> > >> > > > >> > >> > > > <jo...@confluent.io.invalid>
> > >> > > > >> > >> > > > wrote:
> > >> > > > >> > >> > > >
> > >> > > > >> > >> > > > > Hi Jun,
> > >> > > > >> > >> > > > >
> > >> > > > >> > >> > > > > Thanks for replying!
> > >> > > > >> > >> > > > >
> > >> > > > >> > >> > > > > 70.We already do the overflow mechanism, so my
> > >> change
> > >> > > would
> > >> > > > >> just
> > >> > > > >> > >> make
> > >> > > > >> > >> > > it
> > >> > > > >> > >> > > > > happen more often.
> > >> > > > >> > >> > > > > I was also not suggesting a new field in the
> log,
> > >> but
> > >> > in
> > >> > > > the
> > >> > > > >> > >> > response,
> > >> > > > >> > >> > > > > which would be gated by the client version.
> Sorry if
> > >> > > > >> something
> > >> > > > >> > >> there
> > >> > > > >> > >> > is
> > >> > > > >> > >> > > > > unclear. I think we are starting to diverge.
> > >> > > > >> > >> > > > > The goal of this KIP is to not change to the
> marker
> > >> > > format
> > >> > > > at
> > >> > > > >> > all.
> > >> > > > >> > >> > > > >
> > >> > > > >> > >> > > > > 71. Yes, I guess I was going under the
> assumption
> > >> that
> > >> > > the
> > >> > > > >> log
> > >> > > > >> > >> would
> > >> > > > >> > >> > > just
> > >> > > > >> > >> > > > > look at its last epoch and treat it as the
> current
> > >> > > epoch. I
> > >> > > > >> > >> suppose
> > >> > > > >> > >> > we
> > >> > > > >> > >> > > > can
> > >> > > > >> > >> > > > > have some special logic that if the last epoch
> was
> > >> on a
> > >> > > > >> marker
> > >> > > > >> > we
> > >> > > > >> > >> > > > actually
> > >> > > > >> > >> > > > > expect the next epoch or something like that. We
> > >> just
> > >> > > need
> > >> > > > to
> > >> > > > >> > >> > > distinguish
> > >> > > > >> > >> > > > > based on whether we had a commit/abort marker.
> > >> > > > >> > >> > > > >
> > >> > > > >> > >> > > > > 72.
> > >> > > > >> > >> > > > > > if the producer epoch hasn't been bumped on
> the
> > >> > > > >> > >> > > > > broker, it seems that the stucked message will
> fail
> > >> the
> > >> > > > >> sequence
> > >> > > > >> > >> > > > validation
> > >> > > > >> > >> > > > > and will be ignored. If the producer epoch has
> been
> > >> > > bumped,
> > >> > > > >> we
> > >> > > > >> > >> ignore
> > >> > > > >> > >> > > the
> > >> > > > >> > >> > > > > sequence check and the stuck message could be
> > >> appended
> > >> > to
> > >> > > > the
> > >> > > > >> > log.
> > >> > > > >> > >> > So,
> > >> > > > >> > >> > > is
> > >> > > > >> > >> > > > > the latter case that we want to guard?
> > >> > > > >> > >> > > > >
> > >> > > > >> > >> > > > > I'm not sure I follow that "the message will
> fail
> > >> the
> > >> > > > >> sequence
> > >> > > > >> > >> > > > validation".
> > >> > > > >> > >> > > > > In some of these cases, we had an abort marker
> (due
> > >> to
> > >> > an
> > >> > > > >> error)
> > >> > > > >> > >> and
> > >> > > > >> > >> > > then
> > >> > > > >> > >> > > > > the late message comes in with the correct
> sequence
> > >> > > number.
> > >> > > > >> This
> > >> > > > >> > >> is a
> > >> > > > >> > >> > > > case
> > >> > > > >> > >> > > > > covered by the KIP.
> > >> > > > >> > >> > > > > The latter case is actually not something we've
> > >> > > considered
> > >> > > > >> > here. I
> > >> > > > >> > >> > > think
> > >> > > > >> > >> > > > > generally when we bump the epoch, we are
> accepting
> > >> that
> > >> > > the
> > >> > > > >> > >> sequence
> > >> > > > >> > >> > > does
> > >> > > > >> > >> > > > > not need to be checked anymore. My
> understanding is
> > >> > also
> > >> > > > >> that we
> > >> > > > >> > >> > don't
> > >> > > > >> > >> > > > > typically bump epoch mid transaction (based on a
> > >> quick
> > >> > > look
> > >> > > > >> at
> > >> > > > >> > the
> > >> > > > >> > >> > > code)
> > >> > > > >> > >> > > > > but let me know if that is the case.
> > >> > > > >> > >> > > > >
> > >> > > > >> > >> > > > > Thanks,
> > >> > > > >> > >> > > > > Justine
> > >> > > > >> > >> > > > >
> > >> > > > >> > >> > > > > On Thu, Dec 15, 2022 at 12:23 PM Jun Rao
> > >> > > > >> > <jun@confluent.io.invalid
> > >> > > > >> > >> >
> > >> > > > >> > >> > > > wrote:
> > >> > > > >> > >> > > > >
> > >> > > > >> > >> > > > > > Hi, Justine,
> > >> > > > >> > >> > > > > >
> > >> > > > >> > >> > > > > > Thanks for the reply.
> > >> > > > >> > >> > > > > >
> > >> > > > >> > >> > > > > > 70. Assigning a new pid on int overflow seems
> a
> > >> bit
> > >> > > > hacky.
> > >> > > > >> If
> > >> > > > >> > we
> > >> > > > >> > >> > > need a
> > >> > > > >> > >> > > > > txn
> > >> > > > >> > >> > > > > > level id, it will be better to model this
> > >> explicitly.
> > >> > > > >> Adding a
> > >> > > > >> > >> new
> > >> > > > >> > >> > > > field
> > >> > > > >> > >> > > > > > would require a bit more work since it
> requires a
> > >> new
> > >> > > txn
> > >> > > > >> > marker
> > >> > > > >> > >> > > format
> > >> > > > >> > >> > > > > in
> > >> > > > >> > >> > > > > > the log. So, we probably need to guard it
> with an
> > >> IBP
> > >> > > or
> > >> > > > >> > >> metadata
> > >> > > > >> > >> > > > version
> > >> > > > >> > >> > > > > > and document the impact on downgrade once the
> new
> > >> > > format
> > >> > > > is
> > >> > > > >> > >> written
> > >> > > > >> > >> > > to
> > >> > > > >> > >> > > > > the
> > >> > > > >> > >> > > > > > log.
> > >> > > > >> > >> > > > > >
> > >> > > > >> > >> > > > > > 71. Hmm, once the marker is written, the
> partition
> > >> > will
> > >> > > > >> expect
> > >> > > > >> > >> the
> > >> > > > >> > >> > > next
> > >> > > > >> > >> > > > > > append to be on the next epoch. Does that
> cover
> > >> the
> > >> > > case
> > >> > > > >> you
> > >> > > > >> > >> > > mentioned?
> > >> > > > >> > >> > > > > >
> > >> > > > >> > >> > > > > > 72. Also, just to be clear on the stucked
> message
> > >> > issue
> > >> > > > >> > >> described
> > >> > > > >> > >> > in
> > >> > > > >> > >> > > > the
> > >> > > > >> > >> > > > > > motivation. With EoS, we also validate the
> > >> sequence
> > >> > id
> > >> > > > for
> > >> > > > >> > >> > > idempotency.
> > >> > > > >> > >> > > > > So,
> > >> > > > >> > >> > > > > > with the current logic, if the producer epoch
> > >> hasn't
> > >> > > been
> > >> > > > >> > >> bumped on
> > >> > > > >> > >> > > the
> > >> > > > >> > >> > > > > > broker, it seems that the stucked message will
> > >> fail
> > >> > the
> > >> > > > >> > sequence
> > >> > > > >> > >> > > > > validation
> > >> > > > >> > >> > > > > > and will be ignored. If the producer epoch has
> > >> been
> > >> > > > >> bumped, we
> > >> > > > >> > >> > ignore
> > >> > > > >> > >> > > > the
> > >> > > > >> > >> > > > > > sequence check and the stuck message could be
> > >> > appended
> > >> > > to
> > >> > > > >> the
> > >> > > > >> > >> log.
> > >> > > > >> > >> > > So,
> > >> > > > >> > >> > > > is
> > >> > > > >> > >> > > > > > the latter case that we want to guard?
> > >> > > > >> > >> > > > > >
> > >> > > > >> > >> > > > > > Thanks,
> > >> > > > >> > >> > > > > >
> > >> > > > >> > >> > > > > > Jun
> > >> > > > >> > >> > > > > >
> > >> > > > >> > >> > > > > > On Wed, Dec 14, 2022 at 10:44 AM Justine
> Olshan
> > >> > > > >> > >> > > > > > <jo...@confluent.io.invalid> wrote:
> > >> > > > >> > >> > > > > >
> > >> > > > >> > >> > > > > > > Matthias — thanks again for taking time to
> look
> > >> a
> > >> > > this.
> > >> > > > >> You
> > >> > > > >> > >> said:
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > > > > My proposal was only focusing to avoid
> > >> dangling
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > > > transactions if records are added without
> > >> > registered
> > >> > > > >> > >> partition.
> > >> > > > >> > >> > --
> > >> > > > >> > >> > > > > Maybe
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > > > you can add a few more details to the KIP
> about
> > >> > this
> > >> > > > >> > scenario
> > >> > > > >> > >> for
> > >> > > > >> > >> > > > > better
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > > > documentation purpose?
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > > > I'm not sure I understand what you mean
> here.
> > >> The
> > >> > > > >> motivation
> > >> > > > >> > >> > > section
> > >> > > > >> > >> > > > > > > describes two scenarios about how the record
> > >> can be
> > >> > > > added
> > >> > > > >> > >> > without a
> > >> > > > >> > >> > > > > > > registered partition:
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > > > > 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.
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > > > > Another way hanging transactions can
> occur is
> > >> > that
> > >> > > a
> > >> > > > >> > client
> > >> > > > >> > >> is
> > >> > > > >> > >> > > > buggy
> > >> > > > >> > >> > > > > > and
> > >> > > > >> > >> > > > > > > may somehow try to write to a partition
> before
> > >> it
> > >> > > adds
> > >> > > > >> the
> > >> > > > >> > >> > > partition
> > >> > > > >> > >> > > > to
> > >> > > > >> > >> > > > > > the
> > >> > > > >> > >> > > > > > > transaction.
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > > > For the first example of this would it be
> > >> helpful
> > >> > to
> > >> > > > say
> > >> > > > >> > that
> > >> > > > >> > >> > this
> > >> > > > >> > >> > > > > > message
> > >> > > > >> > >> > > > > > > comes in after the abort, but before the
> > >> partition
> > >> > is
> > >> > > > >> added
> > >> > > > >> > to
> > >> > > > >> > >> > the
> > >> > > > >> > >> > > > next
> > >> > > > >> > >> > > > > > > transaction so it becomes "hanging."
> Perhaps the
> > >> > next
> > >> > > > >> > sentence
> > >> > > > >> > >> > > > > describing
> > >> > > > >> > >> > > > > > > the message becoming part of the next
> > >> transaction
> > >> > (a
> > >> > > > >> > different
> > >> > > > >> > >> > > case)
> > >> > > > >> > >> > > > > was
> > >> > > > >> > >> > > > > > > not properly differentiated.
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > > > Jun — thanks for reading the KIP.
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > > > 70. The int typing was a concern. Currently
> we
> > >> > have a
> > >> > > > >> > >> mechanism
> > >> > > > >> > >> > in
> > >> > > > >> > >> > > > > place
> > >> > > > >> > >> > > > > > to
> > >> > > > >> > >> > > > > > > fence the final epoch when the epoch is
> about to
> > >> > > > overflow
> > >> > > > >> > and
> > >> > > > >> > >> > > assign
> > >> > > > >> > >> > > > a
> > >> > > > >> > >> > > > > > new
> > >> > > > >> > >> > > > > > > producer ID with epoch 0. Of course, this
> is a
> > >> bit
> > >> > > > tricky
> > >> > > > >> > >> when it
> > >> > > > >> > >> > > > comes
> > >> > > > >> > >> > > > > > to
> > >> > > > >> > >> > > > > > > the response back to the client.
> > >> > > > >> > >> > > > > > > Making this a long could be another option,
> but
> > >> I
> > >> > > > wonder
> > >> > > > >> are
> > >> > > > >> > >> > there
> > >> > > > >> > >> > > > any
> > >> > > > >> > >> > > > > > > implications on changing this field if the
> > >> epoch is
> > >> > > > >> > persisted
> > >> > > > >> > >> to
> > >> > > > >> > >> > > > disk?
> > >> > > > >> > >> > > > > > I'd
> > >> > > > >> > >> > > > > > > need to check the usages.
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > > > 71.This was something Matthias asked about
> as
> > >> > well. I
> > >> > > > was
> > >> > > > >> > >> > > > considering a
> > >> > > > >> > >> > > > > > > possible edge case where a produce request
> from
> > >> a
> > >> > new
> > >> > > > >> > >> transaction
> > >> > > > >> > >> > > > > somehow
> > >> > > > >> > >> > > > > > > gets sent right after the marker is
> written, but
> > >> > > before
> > >> > > > >> the
> > >> > > > >> > >> > > producer
> > >> > > > >> > >> > > > is
> > >> > > > >> > >> > > > > > > alerted of the newly bumped epoch. In this
> > >> case, we
> > >> > > may
> > >> > > > >> > >> include
> > >> > > > >> > >> > > this
> > >> > > > >> > >> > > > > > record
> > >> > > > >> > >> > > > > > > when we don't want to. I suppose we could
> try
> > >> to do
> > >> > > > >> > something
> > >> > > > >> > >> > > client
> > >> > > > >> > >> > > > > side
> > >> > > > >> > >> > > > > > > to bump the epoch after sending an endTxn as
> > >> well
> > >> > in
> > >> > > > this
> > >> > > > >> > >> > scenario
> > >> > > > >> > >> > > —
> > >> > > > >> > >> > > > > but
> > >> > > > >> > >> > > > > > I
> > >> > > > >> > >> > > > > > > wonder how it would work when the server is
> > >> > aborting
> > >> > > > >> based
> > >> > > > >> > on
> > >> > > > >> > >> a
> > >> > > > >> > >> > > > > > server-side
> > >> > > > >> > >> > > > > > > error. I could also be missing something and
> > >> this
> > >> > > > >> scenario
> > >> > > > >> > is
> > >> > > > >> > >> > > > actually
> > >> > > > >> > >> > > > > > not
> > >> > > > >> > >> > > > > > > possible.
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > > > Thanks again to everyone reading and
> commenting.
> > >> > Let
> > >> > > me
> > >> > > > >> know
> > >> > > > >> > >> > about
> > >> > > > >> > >> > > > any
> > >> > > > >> > >> > > > > > > further questions or comments.
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > > > Justine
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > > > On Wed, Dec 14, 2022 at 9:41 AM Jun Rao
> > >> > > > >> > >> <jun@confluent.io.invalid
> > >> > > > >> > >> > >
> > >> > > > >> > >> > > > > > wrote:
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > > > > Hi, Justine,
> > >> > > > >> > >> > > > > > > >
> > >> > > > >> > >> > > > > > > > Thanks for the KIP. A couple of comments.
> > >> > > > >> > >> > > > > > > >
> > >> > > > >> > >> > > > > > > > 70. Currently, the producer epoch is an
> int.
> > >> I am
> > >> > > not
> > >> > > > >> sure
> > >> > > > >> > >> if
> > >> > > > >> > >> > > it's
> > >> > > > >> > >> > > > > > enough
> > >> > > > >> > >> > > > > > > > to accommodate all transactions in the
> > >> lifetime
> > >> > of
> > >> > > a
> > >> > > > >> > >> producer.
> > >> > > > >> > >> > > > Should
> > >> > > > >> > >> > > > > > we
> > >> > > > >> > >> > > > > > > > change that to a long or add a new long
> field
> > >> > like
> > >> > > > >> txnId?
> > >> > > > >> > >> > > > > > > >
> > >> > > > >> > >> > > > > > > > 71. "it will write the prepare commit
> message
> > >> > with
> > >> > > a
> > >> > > > >> > bumped
> > >> > > > >> > >> > epoch
> > >> > > > >> > >> > > > and
> > >> > > > >> > >> > > > > > > send
> > >> > > > >> > >> > > > > > > > WriteTxnMarkerRequests with the bumped
> epoch."
> > >> > Hmm,
> > >> > > > the
> > >> > > > >> > >> epoch
> > >> > > > >> > >> > is
> > >> > > > >> > >> > > > > > > associated
> > >> > > > >> > >> > > > > > > > with the current txn right? So, it seems
> > >> weird to
> > >> > > > >> write a
> > >> > > > >> > >> > commit
> > >> > > > >> > >> > > > > > message
> > >> > > > >> > >> > > > > > > > with a bumped epoch. Should we only bump
> up
> > >> the
> > >> > > epoch
> > >> > > > >> in
> > >> > > > >> > >> > > > > EndTxnResponse
> > >> > > > >> > >> > > > > > > and
> > >> > > > >> > >> > > > > > > > rename the field to sth like
> > >> nextProducerEpoch?
> > >> > > > >> > >> > > > > > > >
> > >> > > > >> > >> > > > > > > > Thanks,
> > >> > > > >> > >> > > > > > > >
> > >> > > > >> > >> > > > > > > > Jun
> > >> > > > >> > >> > > > > > > >
> > >> > > > >> > >> > > > > > > >
> > >> > > > >> > >> > > > > > > >
> > >> > > > >> > >> > > > > > > > On Mon, Dec 12, 2022 at 8:54 PM Matthias
> J.
> > >> Sax <
> > >> > > > >> > >> > > mjsax@apache.org>
> > >> > > > >> > >> > > > > > > wrote:
> > >> > > > >> > >> > > > > > > >
> > >> > > > >> > >> > > > > > > > > Thanks for the background.
> > >> > > > >> > >> > > > > > > > >
> > >> > > > >> > >> > > > > > > > > 20/30: SGTM. My proposal was only
> focusing
> > >> to
> > >> > > avoid
> > >> > > > >> > >> dangling
> > >> > > > >> > >> > > > > > > > > transactions if records are added
> without
> > >> > > > registered
> > >> > > > >> > >> > partition.
> > >> > > > >> > >> > > > --
> > >> > > > >> > >> > > > > > > Maybe
> > >> > > > >> > >> > > > > > > > > you can add a few more details to the
> KIP
> > >> about
> > >> > > > this
> > >> > > > >> > >> scenario
> > >> > > > >> > >> > > for
> > >> > > > >> > >> > > > > > > better
> > >> > > > >> > >> > > > > > > > > documentation purpose?
> > >> > > > >> > >> > > > > > > > >
> > >> > > > >> > >> > > > > > > > > 40: I think you hit a fair point about
> race
> > >> > > > >> conditions
> > >> > > > >> > or
> > >> > > > >> > >> > > client
> > >> > > > >> > >> > > > > bugs
> > >> > > > >> > >> > > > > > > > > (incorrectly not bumping the epoch). The
> > >> > > > >> > >> complexity/confusion
> > >> > > > >> > >> > > for
> > >> > > > >> > >> > > > > > using
> > >> > > > >> > >> > > > > > > > > the bumped epoch I see, is mainly for
> > >> internal
> > >> > > > >> > debugging,
> > >> > > > >> > >> ie,
> > >> > > > >> > >> > > > > > > inspecting
> > >> > > > >> > >> > > > > > > > > log segment dumps -- it seems harder to
> > >> reason
> > >> > > > about
> > >> > > > >> the
> > >> > > > >> > >> > system
> > >> > > > >> > >> > > > for
> > >> > > > >> > >> > > > > > us
> > >> > > > >> > >> > > > > > > > > humans. But if we get better
> guarantees, it
> > >> > would
> > >> > > > be
> > >> > > > >> > >> worth to
> > >> > > > >> > >> > > use
> > >> > > > >> > >> > > > > the
> > >> > > > >> > >> > > > > > > > > bumped epoch.
> > >> > > > >> > >> > > > > > > > >
> > >> > > > >> > >> > > > > > > > > 60: as I mentioned already, I don't
> know the
> > >> > > broker
> > >> > > > >> > >> internals
> > >> > > > >> > >> > > to
> > >> > > > >> > >> > > > > > > provide
> > >> > > > >> > >> > > > > > > > > more input. So if nobody else chimes
> in, we
> > >> > > should
> > >> > > > >> just
> > >> > > > >> > >> move
> > >> > > > >> > >> > > > > forward
> > >> > > > >> > >> > > > > > > > > with your proposal.
> > >> > > > >> > >> > > > > > > > >
> > >> > > > >> > >> > > > > > > > >
> > >> > > > >> > >> > > > > > > > > -Matthias
> > >> > > > >> > >> > > > > > > > >
> > >> > > > >> > >> > > > > > > > >
> > >> > > > >> > >> > > > > > > > > On 12/6/22 4:22 PM, Justine Olshan
> wrote:
> > >> > > > >> > >> > > > > > > > > > Hi all,
> > >> > > > >> > >> > > > > > > > > > After Artem's questions about error
> > >> behavior,
> > >> > > > I've
> > >> > > > >> > >> > > re-evaluated
> > >> > > > >> > >> > > > > the
> > >> > > > >> > >> > > > > > > > > > unknown producer ID exception and had
> some
> > >> > > > >> discussions
> > >> > > > >> > >> > > offline.
> > >> > > > >> > >> > > > > > > > > >
> > >> > > > >> > >> > > > > > > > > > I think generally it makes sense to
> > >> simplify
> > >> > > > error
> > >> > > > >> > >> handling
> > >> > > > >> > >> > > in
> > >> > > > >> > >> > > > > > cases
> > >> > > > >> > >> > > > > > > > like
> > >> > > > >> > >> > > > > > > > > > this and the UNKNOWN_PRODUCER_ID error
> > >> has a
> > >> > > > pretty
> > >> > > > >> > long
> > >> > > > >> > >> > and
> > >> > > > >> > >> > > > > > > > complicated
> > >> > > > >> > >> > > > > > > > > > history. Because of this, I propose
> > >> adding a
> > >> > > new
> > >> > > > >> error
> > >> > > > >> > >> code
> > >> > > > >> > >> > > > > > > > > ABORTABLE_ERROR
> > >> > > > >> > >> > > > > > > > > > that when encountered by new clients
> > >> (gated
> > >> > by
> > >> > > > the
> > >> > > > >> > >> produce
> > >> > > > >> > >> > > > > request
> > >> > > > >> > >> > > > > > > > > version)
> > >> > > > >> > >> > > > > > > > > > will simply abort the transaction.
> This
> > >> > allows
> > >> > > > the
> > >> > > > >> > >> server
> > >> > > > >> > >> > to
> > >> > > > >> > >> > > > have
> > >> > > > >> > >> > > > > > > some
> > >> > > > >> > >> > > > > > > > > say
> > >> > > > >> > >> > > > > > > > > > in whether the client aborts and makes
> > >> > handling
> > >> > > > >> much
> > >> > > > >> > >> > simpler.
> > >> > > > >> > >> > > > In
> > >> > > > >> > >> > > > > > the
> > >> > > > >> > >> > > > > > > > > > future, we can also use this error in
> > >> other
> > >> > > > >> situations
> > >> > > > >> > >> > where
> > >> > > > >> > >> > > we
> > >> > > > >> > >> > > > > > want
> > >> > > > >> > >> > > > > > > to
> > >> > > > >> > >> > > > > > > > > > abort the transactions. We can even
> use on
> > >> > > other
> > >> > > > >> apis.
> > >> > > > >> > >> > > > > > > > > >
> > >> > > > >> > >> > > > > > > > > > I've added this to the KIP. Let me
> know if
> > >> > > there
> > >> > > > >> are
> > >> > > > >> > any
> > >> > > > >> > >> > > > > questions
> > >> > > > >> > >> > > > > > or
> > >> > > > >> > >> > > > > > > > > > issues.
> > >> > > > >> > >> > > > > > > > > >
> > >> > > > >> > >> > > > > > > > > > Justine
> > >> > > > >> > >> > > > > > > > > >
> > >> > > > >> > >> > > > > > > > > > On Fri, Dec 2, 2022 at 10:22 AM
> Justine
> > >> > Olshan
> > >> > > <
> > >> > > > >> > >> > > > > > jolshan@confluent.io
> > >> > > > >> > >> > > > > > > >
> > >> > > > >> > >> > > > > > > > > wrote:
> > >> > > > >> > >> > > > > > > > > >
> > >> > > > >> > >> > > > > > > > > >> Hey Matthias,
> > >> > > > >> > >> > > > > > > > > >>
> > >> > > > >> > >> > > > > > > > > >>
> > >> > > > >> > >> > > > > > > > > >> 20/30 — Maybe I also didn't express
> > >> myself
> > >> > > > >> clearly.
> > >> > > > >> > For
> > >> > > > >> > >> > > older
> > >> > > > >> > >> > > > > > > clients
> > >> > > > >> > >> > > > > > > > we
> > >> > > > >> > >> > > > > > > > > >> don't have a way to distinguish
> between a
> > >> > > > previous
> > >> > > > >> > and
> > >> > > > >> > >> the
> > >> > > > >> > >> > > > > current
> > >> > > > >> > >> > > > > > > > > >> transaction since we don't have the
> epoch
> > >> > > bump.
> > >> > > > >> This
> > >> > > > >> > >> means
> > >> > > > >> > >> > > > that
> > >> > > > >> > >> > > > > a
> > >> > > > >> > >> > > > > > > late
> > >> > > > >> > >> > > > > > > > > >> message from the previous transaction
> > >> may be
> > >> > > > >> added to
> > >> > > > >> > >> the
> > >> > > > >> > >> > > new
> > >> > > > >> > >> > > > > one.
> > >> > > > >> > >> > > > > > > > With
> > >> > > > >> > >> > > > > > > > > >> older clients — we can't guarantee
> this
> > >> > won't
> > >> > > > >> happen
> > >> > > > >> > >> if we
> > >> > > > >> > >> > > > > already
> > >> > > > >> > >> > > > > > > > sent
> > >> > > > >> > >> > > > > > > > > the
> > >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call (why we make
> > >> changes
> > >> > > for
> > >> > > > >> the
> > >> > > > >> > >> newer
> > >> > > > >> > >> > > > > client)
> > >> > > > >> > >> > > > > > > but
> > >> > > > >> > >> > > > > > > > > we
> > >> > > > >> > >> > > > > > > > > >> can at least gate some by ensuring
> that
> > >> the
> > >> > > > >> partition
> > >> > > > >> > >> has
> > >> > > > >> > >> > > been
> > >> > > > >> > >> > > > > > added
> > >> > > > >> > >> > > > > > > > to
> > >> > > > >> > >> > > > > > > > > the
> > >> > > > >> > >> > > > > > > > > >> transaction. The rationale here is
> that
> > >> > there
> > >> > > > are
> > >> > > > >> > >> likely
> > >> > > > >> > >> > > LESS
> > >> > > > >> > >> > > > > late
> > >> > > > >> > >> > > > > > > > > arrivals
> > >> > > > >> > >> > > > > > > > > >> as time goes on, so hopefully most
> late
> > >> > > arrivals
> > >> > > > >> will
> > >> > > > >> > >> come
> > >> > > > >> > >> > > in
> > >> > > > >> > >> > > > > > BEFORE
> > >> > > > >> > >> > > > > > > > the
> > >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call. Those that
> > >> arrive
> > >> > > > before
> > >> > > > >> > will
> > >> > > > >> > >> be
> > >> > > > >> > >> > > > > properly
> > >> > > > >> > >> > > > > > > > gated
> > >> > > > >> > >> > > > > > > > > >> with the describeTransactions
> approach.
> > >> > > > >> > >> > > > > > > > > >>
> > >> > > > >> > >> > > > > > > > > >> If we take the approach you
> suggested,
> > >> ANY
> > >> > > late
> > >> > > > >> > arrival
> > >> > > > >> > >> > > from a
> > >> > > > >> > >> > > > > > > > previous
> > >> > > > >> > >> > > > > > > > > >> transaction will be added. And we
> don't
> > >> want
> > >> > > > >> that. I
> > >> > > > >> > >> also
> > >> > > > >> > >> > > > don't
> > >> > > > >> > >> > > > > > see
> > >> > > > >> > >> > > > > > > > any
> > >> > > > >> > >> > > > > > > > > >> benefit in sending addPartitionsToTxn
> > >> over
> > >> > the
> > >> > > > >> > >> > describeTxns
> > >> > > > >> > >> > > > > call.
> > >> > > > >> > >> > > > > > > They
> > >> > > > >> > >> > > > > > > > > will
> > >> > > > >> > >> > > > > > > > > >> both be one extra RPC to the Txn
> > >> > coordinator.
> > >> > > > >> > >> > > > > > > > > >>
> > >> > > > >> > >> > > > > > > > > >>
> > >> > > > >> > >> > > > > > > > > >> To be clear — newer clients will use
> > >> > > > >> > addPartitionsToTxn
> > >> > > > >> > >> > > > instead
> > >> > > > >> > >> > > > > of
> > >> > > > >> > >> > > > > > > the
> > >> > > > >> > >> > > > > > > > > >> DescribeTxns.
> > >> > > > >> > >> > > > > > > > > >>
> > >> > > > >> > >> > > > > > > > > >>
> > >> > > > >> > >> > > > > > > > > >> 40)
> > >> > > > >> > >> > > > > > > > > >> My concern is that if we have some
> delay
> > >> in
> > >> > > the
> > >> > > > >> > client
> > >> > > > >> > >> to
> > >> > > > >> > >> > > bump
> > >> > > > >> > >> > > > > the
> > >> > > > >> > >> > > > > > > > > epoch,
> > >> > > > >> > >> > > > > > > > > >> it could continue to send epoch 73
> and
> > >> those
> > >> > > > >> records
> > >> > > > >> > >> would
> > >> > > > >> > >> > > not
> > >> > > > >> > >> > > > > be
> > >> > > > >> > >> > > > > > > > > fenced.
> > >> > > > >> > >> > > > > > > > > >> Perhaps this is not an issue if we
> don't
> > >> > allow
> > >> > > > the
> > >> > > > >> > next
> > >> > > > >> > >> > > > produce
> > >> > > > >> > >> > > > > to
> > >> > > > >> > >> > > > > > > go
> > >> > > > >> > >> > > > > > > > > >> through before the EndTxn request
> > >> returns.
> > >> > I'm
> > >> > > > >> also
> > >> > > > >> > >> > thinking
> > >> > > > >> > >> > > > > about
> > >> > > > >> > >> > > > > > > > > cases of
> > >> > > > >> > >> > > > > > > > > >> failure. I will need to think on
> this a
> > >> bit.
> > >> > > > >> > >> > > > > > > > > >>
> > >> > > > >> > >> > > > > > > > > >> I wasn't sure if it was that
> confusing.
> > >> But
> > >> > if
> > >> > > > we
> > >> > > > >> > >> think it
> > >> > > > >> > >> > > is,
> > >> > > > >> > >> > > > > we
> > >> > > > >> > >> > > > > > > can
> > >> > > > >> > >> > > > > > > > > >> investigate other ways.
> > >> > > > >> > >> > > > > > > > > >>
> > >> > > > >> > >> > > > > > > > > >>
> > >> > > > >> > >> > > > > > > > > >> 60)
> > >> > > > >> > >> > > > > > > > > >>
> > >> > > > >> > >> > > > > > > > > >> I'm not sure these are the same
> > >> purgatories
> > >> > > > since
> > >> > > > >> one
> > >> > > > >> > >> is a
> > >> > > > >> > >> > > > > produce
> > >> > > > >> > >> > > > > > > > > >> purgatory (I was planning on using a
> > >> > callback
> > >> > > > >> rather
> > >> > > > >> > >> than
> > >> > > > >> > >> > > > > > purgatory)
> > >> > > > >> > >> > > > > > > > and
> > >> > > > >> > >> > > > > > > > > >> the other is simply a request to
> append
> > >> to
> > >> > the
> > >> > > > >> log.
> > >> > > > >> > Not
> > >> > > > >> > >> > sure
> > >> > > > >> > >> > > > we
> > >> > > > >> > >> > > > > > have
> > >> > > > >> > >> > > > > > > > any
> > >> > > > >> > >> > > > > > > > > >> structure here for ordering, but my
> > >> > > > understanding
> > >> > > > >> is
> > >> > > > >> > >> that
> > >> > > > >> > >> > > the
> > >> > > > >> > >> > > > > > broker
> > >> > > > >> > >> > > > > > > > > could
> > >> > > > >> > >> > > > > > > > > >> handle the write request before it
> hears
> > >> > back
> > >> > > > from
> > >> > > > >> > the
> > >> > > > >> > >> Txn
> > >> > > > >> > >> > > > > > > > Coordinator.
> > >> > > > >> > >> > > > > > > > > >>
> > >> > > > >> > >> > > > > > > > > >> Let me know if I misunderstood
> something
> > >> or
> > >> > > > >> something
> > >> > > > >> > >> was
> > >> > > > >> > >> > > > > unclear.
> > >> > > > >> > >> > > > > > > > > >>
> > >> > > > >> > >> > > > > > > > > >> Justine
> > >> > > > >> > >> > > > > > > > > >>
> > >> > > > >> > >> > > > > > > > > >> On Thu, Dec 1, 2022 at 12:15 PM
> Matthias
> > >> J.
> > >> > > Sax
> > >> > > > <
> > >> > > > >> > >> > > > > mjsax@apache.org
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > > > > > wrote:
> > >> > > > >> > >> > > > > > > > > >>
> > >> > > > >> > >> > > > > > > > > >>> Thanks for the details Justine!
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > > >>>> 20)
> > >> > > > >> > >> > > > > > > > > >>>>
> > >> > > > >> > >> > > > > > > > > >>>> The client side change for 2 is
> > >> removing
> > >> > the
> > >> > > > >> > >> > addPartitions
> > >> > > > >> > >> > > > to
> > >> > > > >> > >> > > > > > > > > >>> transaction
> > >> > > > >> > >> > > > > > > > > >>>> call. We don't need to make this
> from
> > >> the
> > >> > > > >> producer
> > >> > > > >> > to
> > >> > > > >> > >> > the
> > >> > > > >> > >> > > > txn
> > >> > > > >> > >> > > > > > > > > >>> coordinator,
> > >> > > > >> > >> > > > > > > > > >>>> only server side.
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > > >>> I think I did not express myself
> > >> clearly. I
> > >> > > > >> > understand
> > >> > > > >> > >> > that
> > >> > > > >> > >> > > > we
> > >> > > > >> > >> > > > > > can
> > >> > > > >> > >> > > > > > > > (and
> > >> > > > >> > >> > > > > > > > > >>> should) change the producer to not
> send
> > >> the
> > >> > > > >> > >> > `addPartitions`
> > >> > > > >> > >> > > > > > request
> > >> > > > >> > >> > > > > > > > any
> > >> > > > >> > >> > > > > > > > > >>> longer. But I don't thinks it's
> > >> requirement
> > >> > > to
> > >> > > > >> > change
> > >> > > > >> > >> the
> > >> > > > >> > >> > > > > broker?
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > > >>> What I am trying to say is: as a
> > >> safe-guard
> > >> > > and
> > >> > > > >> > >> > improvement
> > >> > > > >> > >> > > > for
> > >> > > > >> > >> > > > > > > older
> > >> > > > >> > >> > > > > > > > > >>> producers, the partition leader can
> just
> > >> > send
> > >> > > > the
> > >> > > > >> > >> > > > > `addPartitions`
> > >> > > > >> > >> > > > > > > > > >>> request to the TX-coordinator in any
> > >> case
> > >> > --
> > >> > > if
> > >> > > > >> the
> > >> > > > >> > >> old
> > >> > > > >> > >> > > > > producer
> > >> > > > >> > >> > > > > > > > > >>> correctly did send the
> `addPartition`
> > >> > request
> > >> > > > to
> > >> > > > >> the
> > >> > > > >> > >> > > > > > TX-coordinator
> > >> > > > >> > >> > > > > > > > > >>> already, the TX-coordinator can just
> > >> > "ignore"
> > >> > > > is
> > >> > > > >> as
> > >> > > > >> > >> > > > idempotent.
> > >> > > > >> > >> > > > > > > > > However,
> > >> > > > >> > >> > > > > > > > > >>> if the old producer has a bug and
> did
> > >> > forget
> > >> > > to
> > >> > > > >> sent
> > >> > > > >> > >> the
> > >> > > > >> > >> > > > > > > > `addPartition`
> > >> > > > >> > >> > > > > > > > > >>> request, we would now ensure that
> the
> > >> > > partition
> > >> > > > >> is
> > >> > > > >> > >> indeed
> > >> > > > >> > >> > > > added
> > >> > > > >> > >> > > > > > to
> > >> > > > >> > >> > > > > > > > the
> > >> > > > >> > >> > > > > > > > > >>> TX and thus fix a potential
> producer bug
> > >> > > (even
> > >> > > > >> if we
> > >> > > > >> > >> > don't
> > >> > > > >> > >> > > > get
> > >> > > > >> > >> > > > > > the
> > >> > > > >> > >> > > > > > > > > >>> fencing via the bump epoch). -- It
> > >> seems to
> > >> > > be
> > >> > > > a
> > >> > > > >> > good
> > >> > > > >> > >> > > > > > improvement?
> > >> > > > >> > >> > > > > > > Or
> > >> > > > >> > >> > > > > > > > > is
> > >> > > > >> > >> > > > > > > > > >>> there a reason to not do this?
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > > >>>> 30)
> > >> > > > >> > >> > > > > > > > > >>>>
> > >> > > > >> > >> > > > > > > > > >>>> Transaction is ongoing = partition
> was
> > >> > added
> > >> > > > to
> > >> > > > >> > >> > > transaction
> > >> > > > >> > >> > > > > via
> > >> > > > >> > >> > > > > > > > > >>>> addPartitionsToTxn. We check this
> with
> > >> the
> > >> > > > >> > >> > > > > DescribeTransactions
> > >> > > > >> > >> > > > > > > > call.
> > >> > > > >> > >> > > > > > > > > >>> Let
> > >> > > > >> > >> > > > > > > > > >>>> me know if this wasn't sufficiently
> > >> > > explained
> > >> > > > >> here:
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > > >>> If we do what I propose in (20), we
> > >> don't
> > >> > > > really
> > >> > > > >> > need
> > >> > > > >> > >> to
> > >> > > > >> > >> > > make
> > >> > > > >> > >> > > > > > this
> > >> > > > >> > >> > > > > > > > > >>> `DescribeTransaction` call, as the
> > >> > partition
> > >> > > > >> leader
> > >> > > > >> > >> adds
> > >> > > > >> > >> > > the
> > >> > > > >> > >> > > > > > > > partition
> > >> > > > >> > >> > > > > > > > > >>> for older clients and we get this
> check
> > >> for
> > >> > > > free.
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > > >>>> 40)
> > >> > > > >> > >> > > > > > > > > >>>>
> > >> > > > >> > >> > > > > > > > > >>>> The idea here is that if any
> messages
> > >> > > somehow
> > >> > > > >> come
> > >> > > > >> > in
> > >> > > > >> > >> > > before
> > >> > > > >> > >> > > > > we
> > >> > > > >> > >> > > > > > > get
> > >> > > > >> > >> > > > > > > > > the
> > >> > > > >> > >> > > > > > > > > >>> new
> > >> > > > >> > >> > > > > > > > > >>>> epoch to the producer, they will be
> > >> > fenced.
> > >> > > > >> > However,
> > >> > > > >> > >> if
> > >> > > > >> > >> > we
> > >> > > > >> > >> > > > > don't
> > >> > > > >> > >> > > > > > > > think
> > >> > > > >> > >> > > > > > > > > >>> this
> > >> > > > >> > >> > > > > > > > > >>>> is necessary, it can be discussed
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > > >>> I agree that we should have epoch
> > >> fencing.
> > >> > My
> > >> > > > >> > >> question is
> > >> > > > >> > >> > > > > > > different:
> > >> > > > >> > >> > > > > > > > > >>> Assume we are at epoch 73, and we
> have
> > >> an
> > >> > > > ongoing
> > >> > > > >> > >> > > > transaction,
> > >> > > > >> > >> > > > > > that
> > >> > > > >> > >> > > > > > > > is
> > >> > > > >> > >> > > > > > > > > >>> committed. It seems natural to
> write the
> > >> > > > "prepare
> > >> > > > >> > >> commit"
> > >> > > > >> > >> > > > > marker
> > >> > > > >> > >> > > > > > > and
> > >> > > > >> > >> > > > > > > > > the
> > >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` both with
> epoch
> > >> 73,
> > >> > > too,
> > >> > > > >> as
> > >> > > > >> > it
> > >> > > > >> > >> > > belongs
> > >> > > > >> > >> > > > > to
> > >> > > > >> > >> > > > > > > the
> > >> > > > >> > >> > > > > > > > > >>> current transaction. Of course, we
> now
> > >> also
> > >> > > > bump
> > >> > > > >> the
> > >> > > > >> > >> > epoch
> > >> > > > >> > >> > > > and
> > >> > > > >> > >> > > > > > > expect
> > >> > > > >> > >> > > > > > > > > >>> the next requests to have epoch 74,
> and
> > >> > would
> > >> > > > >> reject
> > >> > > > >> > >> an
> > >> > > > >> > >> > > > request
> > >> > > > >> > >> > > > > > > with
> > >> > > > >> > >> > > > > > > > > >>> epoch 73, as the corresponding TX
> for
> > >> epoch
> > >> > > 73
> > >> > > > >> was
> > >> > > > >> > >> > already
> > >> > > > >> > >> > > > > > > committed.
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > > >>> It seems you propose to write the
> > >> "prepare
> > >> > > > commit
> > >> > > > >> > >> marker"
> > >> > > > >> > >> > > and
> > >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` with epoch 74
> > >> > though,
> > >> > > > what
> > >> > > > >> > >> would
> > >> > > > >> > >> > > work,
> > >> > > > >> > >> > > > > but
> > >> > > > >> > >> > > > > > > it
> > >> > > > >> > >> > > > > > > > > >>> seems confusing. Is there a reason
> why
> > >> we
> > >> > > would
> > >> > > > >> use
> > >> > > > >> > >> the
> > >> > > > >> > >> > > > bumped
> > >> > > > >> > >> > > > > > > epoch
> > >> > > > >> > >> > > > > > > > 74
> > >> > > > >> > >> > > > > > > > > >>> instead of the current epoch 73?
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > > >>>> 60)
> > >> > > > >> > >> > > > > > > > > >>>>
> > >> > > > >> > >> > > > > > > > > >>>> When we are checking if the
> > >> transaction is
> > >> > > > >> ongoing,
> > >> > > > >> > >> we
> > >> > > > >> > >> > > need
> > >> > > > >> > >> > > > to
> > >> > > > >> > >> > > > > > > make
> > >> > > > >> > >> > > > > > > > a
> > >> > > > >> > >> > > > > > > > > >>> round
> > >> > > > >> > >> > > > > > > > > >>>> trip from the leader partition to
> the
> > >> > > > >> transaction
> > >> > > > >> > >> > > > coordinator.
> > >> > > > >> > >> > > > > > In
> > >> > > > >> > >> > > > > > > > the
> > >> > > > >> > >> > > > > > > > > >>> time
> > >> > > > >> > >> > > > > > > > > >>>> we are waiting for this message to
> come
> > >> > > back,
> > >> > > > in
> > >> > > > >> > >> theory
> > >> > > > >> > >> > we
> > >> > > > >> > >> > > > > could
> > >> > > > >> > >> > > > > > > > have
> > >> > > > >> > >> > > > > > > > > >>> sent
> > >> > > > >> > >> > > > > > > > > >>>> a commit/abort call that would
> make the
> > >> > > > original
> > >> > > > >> > >> result
> > >> > > > >> > >> > of
> > >> > > > >> > >> > > > the
> > >> > > > >> > >> > > > > > > check
> > >> > > > >> > >> > > > > > > > > >>> out of
> > >> > > > >> > >> > > > > > > > > >>>> date. That is why we can check the
> > >> leader
> > >> > > > state
> > >> > > > >> > >> before
> > >> > > > >> > >> > we
> > >> > > > >> > >> > > > > write
> > >> > > > >> > >> > > > > > to
> > >> > > > >> > >> > > > > > > > the
> > >> > > > >> > >> > > > > > > > > >>> log.
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > > >>> Thanks. Got it.
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > > >>> However, is this really an issue?
> We put
> > >> > the
> > >> > > > >> produce
> > >> > > > >> > >> > > request
> > >> > > > >> > >> > > > in
> > >> > > > >> > >> > > > > > > > > >>> purgatory, so how could we process
> the
> > >> > > > >> > >> > > > `WriteTxnMarkerRequest`
> > >> > > > >> > >> > > > > > > first?
> > >> > > > >> > >> > > > > > > > > >>> Don't we need to put the
> > >> > > > `WriteTxnMarkerRequest`
> > >> > > > >> > into
> > >> > > > >> > >> > > > > purgatory,
> > >> > > > >> > >> > > > > > > too,
> > >> > > > >> > >> > > > > > > > > >>> for this case, and process both
> request
> > >> > > > in-order?
> > >> > > > >> > >> (Again,
> > >> > > > >> > >> > > my
> > >> > > > >> > >> > > > > > broker
> > >> > > > >> > >> > > > > > > > > >>> knowledge is limited and maybe we
> don't
> > >> > > > maintain
> > >> > > > >> > >> request
> > >> > > > >> > >> > > > order
> > >> > > > >> > >> > > > > > for
> > >> > > > >> > >> > > > > > > > this
> > >> > > > >> > >> > > > > > > > > >>> case, what seems to be an issue
> IMHO,
> > >> and I
> > >> > > am
> > >> > > > >> > >> wondering
> > >> > > > >> > >> > if
> > >> > > > >> > >> > > > > > > changing
> > >> > > > >> > >> > > > > > > > > >>> request handling to preserve order
> for
> > >> this
> > >> > > > case
> > >> > > > >> > >> might be
> > >> > > > >> > >> > > the
> > >> > > > >> > >> > > > > > > cleaner
> > >> > > > >> > >> > > > > > > > > >>> solution?)
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > > >>> -Matthias
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > > >>> On 11/30/22 3:28 PM, Artem Livshits
> > >> wrote:
> > >> > > > >> > >> > > > > > > > > >>>> Hi Justine,
> > >> > > > >> > >> > > > > > > > > >>>>
> > >> > > > >> > >> > > > > > > > > >>>> I think the interesting part is
> not in
> > >> > this
> > >> > > > >> logic
> > >> > > > >> > >> > (because
> > >> > > > >> > >> > > > it
> > >> > > > >> > >> > > > > > > tries
> > >> > > > >> > >> > > > > > > > to
> > >> > > > >> > >> > > > > > > > > >>>> figure out when
> UNKNOWN_PRODUCER_ID is
> > >> > > > retriable
> > >> > > > >> > and
> > >> > > > >> > >> if
> > >> > > > >> > >> > > it's
> > >> > > > >> > >> > > > > > > > > retryable,
> > >> > > > >> > >> > > > > > > > > >>>> it's definitely not fatal), but
> what
> > >> > happens
> > >> > > > >> when
> > >> > > > >> > >> this
> > >> > > > >> > >> > > logic
> > >> > > > >> > >> > > > > > > doesn't
> > >> > > > >> > >> > > > > > > > > >>> return
> > >> > > > >> > >> > > > > > > > > >>>> 'true' and falls through.  In the
> old
> > >> > > clients
> > >> > > > it
> > >> > > > >> > >> seems
> > >> > > > >> > >> > to
> > >> > > > >> > >> > > be
> > >> > > > >> > >> > > > > > > fatal,
> > >> > > > >> > >> > > > > > > > if
> > >> > > > >> > >> > > > > > > > > >>> we
> > >> > > > >> > >> > > > > > > > > >>>> keep the behavior in the new
> clients,
> > >> I'd
> > >> > > > >> expect it
> > >> > > > >> > >> > would
> > >> > > > >> > >> > > be
> > >> > > > >> > >> > > > > > fatal
> > >> > > > >> > >> > > > > > > > as
> > >> > > > >> > >> > > > > > > > > >>> well.
> > >> > > > >> > >> > > > > > > > > >>>>
> > >> > > > >> > >> > > > > > > > > >>>> -Artem
> > >> > > > >> > >> > > > > > > > > >>>>
> > >> > > > >> > >> > > > > > > > > >>>> On Tue, Nov 29, 2022 at 11:57 AM
> > >> Justine
> > >> > > > Olshan
> > >> > > > >> > >> > > > > > > > > >>>> <jo...@confluent.io.invalid>
> wrote:
> > >> > > > >> > >> > > > > > > > > >>>>
> > >> > > > >> > >> > > > > > > > > >>>>> Hi Artem and Jeff,
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>> Thanks for taking a look and
> sorry for
> > >> > the
> > >> > > > slow
> > >> > > > >> > >> > response.
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>> You both mentioned the change to
> > >> handle
> > >> > > > >> > >> > > UNKNOWN_PRODUCER_ID
> > >> > > > >> > >> > > > > > > errors.
> > >> > > > >> > >> > > > > > > > > To
> > >> > > > >> > >> > > > > > > > > >>> be
> > >> > > > >> > >> > > > > > > > > >>>>> clear — this error code will only
> be
> > >> sent
> > >> > > > again
> > >> > > > >> > when
> > >> > > > >> > >> > the
> > >> > > > >> > >> > > > > > client's
> > >> > > > >> > >> > > > > > > > > >>> request
> > >> > > > >> > >> > > > > > > > > >>>>> version is high enough to ensure
> we
> > >> > handle
> > >> > > it
> > >> > > > >> > >> > correctly.
> > >> > > > >> > >> > > > > > > > > >>>>> The current (Java) client handles
> > >> this by
> > >> > > the
> > >> > > > >> > >> following
> > >> > > > >> > >> > > > > > (somewhat
> > >> > > > >> > >> > > > > > > > > long)
> > >> > > > >> > >> > > > > > > > > >>>>> code snippet:
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>> // An UNKNOWN_PRODUCER_ID means
> that
> > >> we
> > >> > > have
> > >> > > > >> lost
> > >> > > > >> > >> the
> > >> > > > >> > >> > > > > producer
> > >> > > > >> > >> > > > > > > > state
> > >> > > > >> > >> > > > > > > > > >>> on the
> > >> > > > >> > >> > > > > > > > > >>>>> broker. Depending on the log start
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>> // offset, we may want to retry
> > >> these, as
> > >> > > > >> > described
> > >> > > > >> > >> for
> > >> > > > >> > >> > > > each
> > >> > > > >> > >> > > > > > case
> > >> > > > >> > >> > > > > > > > > >>> below. If
> > >> > > > >> > >> > > > > > > > > >>>>> none of those apply, then for the
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>> // idempotent producer, we will
> > >> locally
> > >> > > bump
> > >> > > > >> the
> > >> > > > >> > >> epoch
> > >> > > > >> > >> > > and
> > >> > > > >> > >> > > > > > reset
> > >> > > > >> > >> > > > > > > > the
> > >> > > > >> > >> > > > > > > > > >>>>> sequence numbers of in-flight
> batches
> > >> > from
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>> // sequence 0, then retry the
> failed
> > >> > batch,
> > >> > > > >> which
> > >> > > > >> > >> > should
> > >> > > > >> > >> > > > now
> > >> > > > >> > >> > > > > > > > succeed.
> > >> > > > >> > >> > > > > > > > > >>> For
> > >> > > > >> > >> > > > > > > > > >>>>> the transactional producer, allow
> the
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>> // batch to fail. When processing
> the
> > >> > > failed
> > >> > > > >> > batch,
> > >> > > > >> > >> we
> > >> > > > >> > >> > > will
> > >> > > > >> > >> > > > > > > > > transition
> > >> > > > >> > >> > > > > > > > > >>> to
> > >> > > > >> > >> > > > > > > > > >>>>> an abortable error and set a flag
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>> // indicating that we need to
> bump the
> > >> > > epoch
> > >> > > > >> (if
> > >> > > > >> > >> > > supported
> > >> > > > >> > >> > > > by
> > >> > > > >> > >> > > > > > the
> > >> > > > >> > >> > > > > > > > > >>> broker).
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>> if (error ==
> > >> > Errors.*UNKNOWN_PRODUCER_ID*)
> > >> > > {
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>       if (response.logStartOffset
> ==
> > >> -1)
> > >> > {
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           // We don't know the log
> > >> start
> > >> > > > offset
> > >> > > > >> > with
> > >> > > > >> > >> > this
> > >> > > > >> > >> > > > > > > response.
> > >> > > > >> > >> > > > > > > > > We
> > >> > > > >> > >> > > > > > > > > >>> should
> > >> > > > >> > >> > > > > > > > > >>>>> just retry the request until we
> get
> > >> it.
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           // The
> UNKNOWN_PRODUCER_ID
> > >> > error
> > >> > > > code
> > >> > > > >> > was
> > >> > > > >> > >> > added
> > >> > > > >> > >> > > > > along
> > >> > > > >> > >> > > > > > > > with
> > >> > > > >> > >> > > > > > > > > >>> the new
> > >> > > > >> > >> > > > > > > > > >>>>> ProduceResponse which includes the
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           // logStartOffset. So
> the
> > >> '-1'
> > >> > > > >> sentinel
> > >> > > > >> > is
> > >> > > > >> > >> > not
> > >> > > > >> > >> > > > for
> > >> > > > >> > >> > > > > > > > backward
> > >> > > > >> > >> > > > > > > > > >>>>> compatibility. Instead, it is
> possible
> > >> > for
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           // a broker to not know
> the
> > >> > > > >> > >> logStartOffset at
> > >> > > > >> > >> > > > when
> > >> > > > >> > >> > > > > it
> > >> > > > >> > >> > > > > > > is
> > >> > > > >> > >> > > > > > > > > >>> returning
> > >> > > > >> > >> > > > > > > > > >>>>> the response because the partition
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           // may have moved away
> from
> > >> the
> > >> > > > >> broker
> > >> > > > >> > >> from
> > >> > > > >> > >> > the
> > >> > > > >> > >> > > > > time
> > >> > > > >> > >> > > > > > > the
> > >> > > > >> > >> > > > > > > > > >>> error was
> > >> > > > >> > >> > > > > > > > > >>>>> initially raised to the time the
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           // response was being
> > >> > > constructed.
> > >> > > > In
> > >> > > > >> > >> these
> > >> > > > >> > >> > > > cases,
> > >> > > > >> > >> > > > > we
> > >> > > > >> > >> > > > > > > > > should
> > >> > > > >> > >> > > > > > > > > >>> just
> > >> > > > >> > >> > > > > > > > > >>>>> retry the request: we are
> guaranteed
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           // to eventually get a
> > >> > > > logStartOffset
> > >> > > > >> > once
> > >> > > > >> > >> > > things
> > >> > > > >> > >> > > > > > > settle
> > >> > > > >> > >> > > > > > > > > down.
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>       }
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>       if
> > >> (batch.sequenceHasBeenReset()) {
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           // When the first
> inflight
> > >> > batch
> > >> > > > >> fails
> > >> > > > >> > >> due to
> > >> > > > >> > >> > > the
> > >> > > > >> > >> > > > > > > > > truncation
> > >> > > > >> > >> > > > > > > > > >>> case,
> > >> > > > >> > >> > > > > > > > > >>>>> then the sequences of all the
> other
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           // in flight batches
> would
> > >> have
> > >> > > > been
> > >> > > > >> > >> > restarted
> > >> > > > >> > >> > > > from
> > >> > > > >> > >> > > > > > the
> > >> > > > >> > >> > > > > > > > > >>> beginning.
> > >> > > > >> > >> > > > > > > > > >>>>> However, when those responses
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           // come back from the
> > >> broker,
> > >> > > they
> > >> > > > >> would
> > >> > > > >> > >> also
> > >> > > > >> > >> > > > come
> > >> > > > >> > >> > > > > > with
> > >> > > > >> > >> > > > > > > > an
> > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID error. In this
> > >> case,
> > >> > we
> > >> > > > >> should
> > >> > > > >> > >> not
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           // reset the sequence
> > >> numbers
> > >> > to
> > >> > > > the
> > >> > > > >> > >> > beginning.
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>       } else if
> > >> > > > >> > >> > > > > (lastAckedOffset(batch.topicPartition).orElse(
> > >> > > > >> > >> > > > > > > > > >>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > >> > > > >> > >> > > > response.logStartOffset) {
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           // The head of the log
> has
> > >> been
> > >> > > > >> removed,
> > >> > > > >> > >> > > probably
> > >> > > > >> > >> > > > > due
> > >> > > > >> > >> > > > > > > to
> > >> > > > >> > >> > > > > > > > > the
> > >> > > > >> > >> > > > > > > > > >>>>> retention time elapsing. In this
> case,
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           // we expect to lose the
> > >> > producer
> > >> > > > >> state.
> > >> > > > >> > >> For
> > >> > > > >> > >> > > the
> > >> > > > >> > >> > > > > > > > > transactional
> > >> > > > >> > >> > > > > > > > > >>>>> producer, reset the sequences of
> all
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           // inflight batches to
> be
> > >> from
> > >> > > the
> > >> > > > >> > >> beginning
> > >> > > > >> > >> > > and
> > >> > > > >> > >> > > > > > retry
> > >> > > > >> > >> > > > > > > > > them,
> > >> > > > >> > >> > > > > > > > > >>> so
> > >> > > > >> > >> > > > > > > > > >>>>> that the transaction does not
> need to
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           // be aborted. For the
> > >> > idempotent
> > >> > > > >> > >> producer,
> > >> > > > >> > >> > > bump
> > >> > > > >> > >> > > > > the
> > >> > > > >> > >> > > > > > > > epoch
> > >> > > > >> > >> > > > > > > > > to
> > >> > > > >> > >> > > > > > > > > >>> avoid
> > >> > > > >> > >> > > > > > > > > >>>>> reusing (sequence, epoch) pairs
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           if (isTransactional()) {
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > >
> > >> > > > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > >> > > > >> > >> > > > > > > > > >>>>> this.producerIdAndEpoch);
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           } else {
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > >
> > >> requestEpochBumpForPartition(batch.topicPartition);
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           }
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>       }
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>       if (!isTransactional()) {
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           // For the idempotent
> > >> producer,
> > >> > > > >> always
> > >> > > > >> > >> retry
> > >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > >> > > > >> > >> > > > > > > > > >>>>> errors. If the batch has the
> current
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           // producer ID and
> epoch,
> > >> > > request a
> > >> > > > >> bump
> > >> > > > >> > >> of
> > >> > > > >> > >> > the
> > >> > > > >> > >> > > > > > epoch.
> > >> > > > >> > >> > > > > > > > > >>> Otherwise
> > >> > > > >> > >> > > > > > > > > >>>>> just retry the produce.
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > >
> requestEpochBumpForPartition(batch.topicPartition);
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>           return true;
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>       }
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>> }
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>> I was considering keeping this
> > >> behavior —
> > >> > > but
> > >> > > > >> am
> > >> > > > >> > >> open
> > >> > > > >> > >> > to
> > >> > > > >> > >> > > > > > > > simplifying
> > >> > > > >> > >> > > > > > > > > >>> it.
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>> We are leaving changes to older
> > >> clients
> > >> > off
> > >> > > > the
> > >> > > > >> > >> table
> > >> > > > >> > >> > > here
> > >> > > > >> > >> > > > > > since
> > >> > > > >> > >> > > > > > > it
> > >> > > > >> > >> > > > > > > > > >>> caused
> > >> > > > >> > >> > > > > > > > > >>>>> many issues for clients in the
> past.
> > >> > > > Previously
> > >> > > > >> > this
> > >> > > > >> > >> > was
> > >> > > > >> > >> > > a
> > >> > > > >> > >> > > > > > fatal
> > >> > > > >> > >> > > > > > > > > error
> > >> > > > >> > >> > > > > > > > > >>> and
> > >> > > > >> > >> > > > > > > > > >>>>> we didn't have the mechanisms in
> > >> place to
> > >> > > > >> detect
> > >> > > > >> > >> when
> > >> > > > >> > >> > > this
> > >> > > > >> > >> > > > > was
> > >> > > > >> > >> > > > > > a
> > >> > > > >> > >> > > > > > > > > >>> legitimate
> > >> > > > >> > >> > > > > > > > > >>>>> case vs some bug or gap in the
> > >> protocol.
> > >> > > > >> Ensuring
> > >> > > > >> > >> each
> > >> > > > >> > >> > > > > > > transaction
> > >> > > > >> > >> > > > > > > > > has
> > >> > > > >> > >> > > > > > > > > >>> its
> > >> > > > >> > >> > > > > > > > > >>>>> own epoch should close this gap.
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>> And to address Jeff's second
> point:
> > >> > > > >> > >> > > > > > > > > >>>>> *does the typical produce request
> path
> > >> > > append
> > >> > > > >> > >> records
> > >> > > > >> > >> > to
> > >> > > > >> > >> > > > > local
> > >> > > > >> > >> > > > > > > log
> > >> > > > >> > >> > > > > > > > > >>> along*
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>> *with the currentTxnFirstOffset
> > >> > > information?
> > >> > > > I
> > >> > > > >> > would
> > >> > > > >> > >> > like
> > >> > > > >> > >> > > > to
> > >> > > > >> > >> > > > > > > > > >>> understand*
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>> *when the field is written to
> disk.*
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>> Yes, the first produce request
> > >> populates
> > >> > > this
> > >> > > > >> > field
> > >> > > > >> > >> and
> > >> > > > >> > >> > > > > writes
> > >> > > > >> > >> > > > > > > the
> > >> > > > >> > >> > > > > > > > > >>> offset
> > >> > > > >> > >> > > > > > > > > >>>>> as part of the record batch and
> also
> > >> to
> > >> > the
> > >> > > > >> > producer
> > >> > > > >> > >> > > state
> > >> > > > >> > >> > > > > > > > snapshot.
> > >> > > > >> > >> > > > > > > > > >>> When
> > >> > > > >> > >> > > > > > > > > >>>>> we reload the records on restart
> > >> and/or
> > >> > > > >> > >> reassignment,
> > >> > > > >> > >> > we
> > >> > > > >> > >> > > > > > > repopulate
> > >> > > > >> > >> > > > > > > > > >>> this
> > >> > > > >> > >> > > > > > > > > >>>>> field with the snapshot from disk
> > >> along
> > >> > > with
> > >> > > > >> the
> > >> > > > >> > >> rest
> > >> > > > >> > >> > of
> > >> > > > >> > >> > > > the
> > >> > > > >> > >> > > > > > > > producer
> > >> > > > >> > >> > > > > > > > > >>>>> state.
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>> Let me know if there are further
> > >> comments
> > >> > > > >> and/or
> > >> > > > >> > >> > > questions.
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>> Thanks,
> > >> > > > >> > >> > > > > > > > > >>>>> Justine
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>> On Tue, Nov 22, 2022 at 9:00 PM
> Jeff
> > >> Kim
> > >> > > > >> > >> > > > > > > > > <jeff.kim@confluent.io.invalid
> > >> > > > >> > >> > > > > > > > > >>>>
> > >> > > > >> > >> > > > > > > > > >>>>> wrote:
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>> Hi Justine,
> > >> > > > >> > >> > > > > > > > > >>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>> Thanks for the KIP! I have two
> > >> > questions:
> > >> > > > >> > >> > > > > > > > > >>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>> 1) For new clients, we can once
> again
> > >> > > return
> > >> > > > >> an
> > >> > > > >> > >> error
> > >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > >> > > > >> > >> > > > > > > > > >>>>>> for sequences
> > >> > > > >> > >> > > > > > > > > >>>>>> that are non-zero when there is
> no
> > >> > > producer
> > >> > > > >> state
> > >> > > > >> > >> > > present
> > >> > > > >> > >> > > > on
> > >> > > > >> > >> > > > > > the
> > >> > > > >> > >> > > > > > > > > >>> server.
> > >> > > > >> > >> > > > > > > > > >>>>>> This will indicate we missed the
> 0
> > >> > > sequence
> > >> > > > >> and
> > >> > > > >> > we
> > >> > > > >> > >> > don't
> > >> > > > >> > >> > > > yet
> > >> > > > >> > >> > > > > > > want
> > >> > > > >> > >> > > > > > > > to
> > >> > > > >> > >> > > > > > > > > >>>>> write
> > >> > > > >> > >> > > > > > > > > >>>>>> to the log.
> > >> > > > >> > >> > > > > > > > > >>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>> I would like to understand the
> > >> current
> > >> > > > >> behavior
> > >> > > > >> > to
> > >> > > > >> > >> > > handle
> > >> > > > >> > >> > > > > > older
> > >> > > > >> > >> > > > > > > > > >>> clients,
> > >> > > > >> > >> > > > > > > > > >>>>>> and if there are any changes we
> are
> > >> > > making.
> > >> > > > >> Maybe
> > >> > > > >> > >> I'm
> > >> > > > >> > >> > > > > missing
> > >> > > > >> > >> > > > > > > > > >>> something,
> > >> > > > >> > >> > > > > > > > > >>>>>> but we would want to identify
> > >> whether we
> > >> > > > >> missed
> > >> > > > >> > >> the 0
> > >> > > > >> > >> > > > > sequence
> > >> > > > >> > >> > > > > > > for
> > >> > > > >> > >> > > > > > > > > >>> older
> > >> > > > >> > >> > > > > > > > > >>>>>> clients, no?
> > >> > > > >> > >> > > > > > > > > >>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>> 2) Upon returning from the
> > >> transaction
> > >> > > > >> > >> coordinator, we
> > >> > > > >> > >> > > can
> > >> > > > >> > >> > > > > set
> > >> > > > >> > >> > > > > > > the
> > >> > > > >> > >> > > > > > > > > >>>>>> transaction
> > >> > > > >> > >> > > > > > > > > >>>>>> as ongoing on the leader by
> > >> populating
> > >> > > > >> > >> > > > currentTxnFirstOffset
> > >> > > > >> > >> > > > > > > > > >>>>>> through the typical produce
> request
> > >> > > > handling.
> > >> > > > >> > >> > > > > > > > > >>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>> does the typical produce request
> path
> > >> > > append
> > >> > > > >> > >> records
> > >> > > > >> > >> > to
> > >> > > > >> > >> > > > > local
> > >> > > > >> > >> > > > > > > log
> > >> > > > >> > >> > > > > > > > > >>> along
> > >> > > > >> > >> > > > > > > > > >>>>>> with the currentTxnFirstOffset
> > >> > > information?
> > >> > > > I
> > >> > > > >> > would
> > >> > > > >> > >> > like
> > >> > > > >> > >> > > > to
> > >> > > > >> > >> > > > > > > > > understand
> > >> > > > >> > >> > > > > > > > > >>>>>> when the field is written to
> disk.
> > >> > > > >> > >> > > > > > > > > >>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>> Thanks,
> > >> > > > >> > >> > > > > > > > > >>>>>> Jeff
> > >> > > > >> > >> > > > > > > > > >>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>> On Tue, Nov 22, 2022 at 4:44 PM
> Artem
> > >> > > > Livshits
> > >> > > > >> > >> > > > > > > > > >>>>>> <al...@confluent.io.invalid>
> > >> wrote:
> > >> > > > >> > >> > > > > > > > > >>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>> Hi Justine,
> > >> > > > >> > >> > > > > > > > > >>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>> Thank you for the KIP.  I have
> one
> > >> > > > question.
> > >> > > > >> > >> > > > > > > > > >>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>> 5) For new clients, we can once
> > >> again
> > >> > > > return
> > >> > > > >> an
> > >> > > > >> > >> error
> > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > >> > > > >> > >> > > > > > > > > >>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>> I believe we had problems in the
> > >> past
> > >> > > with
> > >> > > > >> > >> returning
> > >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > >> > > > >> > >> > > > > > > > > >>>>>>> because it was considered fatal
> and
> > >> > > > required
> > >> > > > >> > >> client
> > >> > > > >> > >> > > > > restart.
> > >> > > > >> > >> > > > > > > It
> > >> > > > >> > >> > > > > > > > > >>> would
> > >> > > > >> > >> > > > > > > > > >>>>> be
> > >> > > > >> > >> > > > > > > > > >>>>>>> good to spell out the new client
> > >> > behavior
> > >> > > > >> when
> > >> > > > >> > it
> > >> > > > >> > >> > > > receives
> > >> > > > >> > >> > > > > > the
> > >> > > > >> > >> > > > > > > > > error.
> > >> > > > >> > >> > > > > > > > > >>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>> -Artem
> > >> > > > >> > >> > > > > > > > > >>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>> On Tue, Nov 22, 2022 at 10:00 AM
> > >> > Justine
> > >> > > > >> Olshan
> > >> > > > >> > >> > > > > > > > > >>>>>>> <jo...@confluent.io.invalid>
> > >> wrote:
> > >> > > > >> > >> > > > > > > > > >>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks for taking a look
> Matthias.
> > >> > I've
> > >> > > > >> tried
> > >> > > > >> > to
> > >> > > > >> > >> > > answer
> > >> > > > >> > >> > > > > your
> > >> > > > >> > >> > > > > > > > > >>>>> questions
> > >> > > > >> > >> > > > > > > > > >>>>>>>> below:
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>> 10)
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>> Right — so the hanging
> transaction
> > >> > only
> > >> > > > >> occurs
> > >> > > > >> > >> when
> > >> > > > >> > >> > we
> > >> > > > >> > >> > > > > have
> > >> > > > >> > >> > > > > > a
> > >> > > > >> > >> > > > > > > > late
> > >> > > > >> > >> > > > > > > > > >>>>>>> message
> > >> > > > >> > >> > > > > > > > > >>>>>>>> come in and the partition is
> never
> > >> > added
> > >> > > > to
> > >> > > > >> a
> > >> > > > >> > >> > > > transaction
> > >> > > > >> > >> > > > > > > again.
> > >> > > > >> > >> > > > > > > > > If
> > >> > > > >> > >> > > > > > > > > >>>>> we
> > >> > > > >> > >> > > > > > > > > >>>>>>>> never add the partition to a
> > >> > > transaction,
> > >> > > > we
> > >> > > > >> > will
> > >> > > > >> > >> > > never
> > >> > > > >> > >> > > > > > write
> > >> > > > >> > >> > > > > > > a
> > >> > > > >> > >> > > > > > > > > >>>>> marker
> > >> > > > >> > >> > > > > > > > > >>>>>>> and
> > >> > > > >> > >> > > > > > > > > >>>>>>>> never advance the LSO.
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>> If we do end up adding the
> > >> partition
> > >> > to
> > >> > > > the
> > >> > > > >> > >> > > transaction
> > >> > > > >> > >> > > > (I
> > >> > > > >> > >> > > > > > > > suppose
> > >> > > > >> > >> > > > > > > > > >>>>> this
> > >> > > > >> > >> > > > > > > > > >>>>>>> can
> > >> > > > >> > >> > > > > > > > > >>>>>>>> happen before or after the late
> > >> > message
> > >> > > > >> comes
> > >> > > > >> > in)
> > >> > > > >> > >> > then
> > >> > > > >> > >> > > > we
> > >> > > > >> > >> > > > > > will
> > >> > > > >> > >> > > > > > > > > >>>>> include
> > >> > > > >> > >> > > > > > > > > >>>>>>> the
> > >> > > > >> > >> > > > > > > > > >>>>>>>> late message in the next
> > >> (incorrect)
> > >> > > > >> > transaction.
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>> So perhaps it is clearer to
> make
> > >> the
> > >> > > > >> > distinction
> > >> > > > >> > >> > > between
> > >> > > > >> > >> > > > > > > > messages
> > >> > > > >> > >> > > > > > > > > >>>>> that
> > >> > > > >> > >> > > > > > > > > >>>>>>>> eventually get added to the
> > >> > transaction
> > >> > > > (but
> > >> > > > >> > the
> > >> > > > >> > >> > wrong
> > >> > > > >> > >> > > > > one)
> > >> > > > >> > >> > > > > > or
> > >> > > > >> > >> > > > > > > > > >>>>> messages
> > >> > > > >> > >> > > > > > > > > >>>>>>>> that never get added and become
> > >> > hanging.
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>> 20)
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>> The client side change for 2 is
> > >> > removing
> > >> > > > the
> > >> > > > >> > >> > > > addPartitions
> > >> > > > >> > >> > > > > > to
> > >> > > > >> > >> > > > > > > > > >>>>>> transaction
> > >> > > > >> > >> > > > > > > > > >>>>>>>> call. We don't need to make
> this
> > >> from
> > >> > > the
> > >> > > > >> > >> producer
> > >> > > > >> > >> > to
> > >> > > > >> > >> > > > the
> > >> > > > >> > >> > > > > > txn
> > >> > > > >> > >> > > > > > > > > >>>>>>> coordinator,
> > >> > > > >> > >> > > > > > > > > >>>>>>>> only server side.
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>> In my opinion, the issue with
> the
> > >> > > > >> > >> addPartitionsToTxn
> > >> > > > >> > >> > > > call
> > >> > > > >> > >> > > > > > for
> > >> > > > >> > >> > > > > > > > > older
> > >> > > > >> > >> > > > > > > > > >>>>>>> clients
> > >> > > > >> > >> > > > > > > > > >>>>>>>> is that we don't have the epoch
> > >> bump,
> > >> > so
> > >> > > > we
> > >> > > > >> > don't
> > >> > > > >> > >> > know
> > >> > > > >> > >> > > > if
> > >> > > > >> > >> > > > > > the
> > >> > > > >> > >> > > > > > > > > >>> message
> > >> > > > >> > >> > > > > > > > > >>>>>>>> belongs to the previous
> > >> transaction or
> > >> > > > this
> > >> > > > >> > one.
> > >> > > > >> > >> We
> > >> > > > >> > >> > > need
> > >> > > > >> > >> > > > > to
> > >> > > > >> > >> > > > > > > > check
> > >> > > > >> > >> > > > > > > > > if
> > >> > > > >> > >> > > > > > > > > >>>>>> the
> > >> > > > >> > >> > > > > > > > > >>>>>>>> partition has been added to
> this
> > >> > > > >> transaction.
> > >> > > > >> > Of
> > >> > > > >> > >> > > course,
> > >> > > > >> > >> > > > > > this
> > >> > > > >> > >> > > > > > > > > means
> > >> > > > >> > >> > > > > > > > > >>>>> we
> > >> > > > >> > >> > > > > > > > > >>>>>>>> won't completely cover the case
> > >> where
> > >> > we
> > >> > > > >> have a
> > >> > > > >> > >> > really
> > >> > > > >> > >> > > > > late
> > >> > > > >> > >> > > > > > > > > message
> > >> > > > >> > >> > > > > > > > > >>>>> and
> > >> > > > >> > >> > > > > > > > > >>>>>>> we
> > >> > > > >> > >> > > > > > > > > >>>>>>>> have added the partition to
> the new
> > >> > > > >> > transaction,
> > >> > > > >> > >> but
> > >> > > > >> > >> > > > > that's
> > >> > > > >> > >> > > > > > > > > >>>>>> unfortunately
> > >> > > > >> > >> > > > > > > > > >>>>>>>> something we will need the new
> > >> clients
> > >> > > to
> > >> > > > >> > cover.
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>> 30)
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>> Transaction is ongoing =
> partition
> > >> was
> > >> > > > >> added to
> > >> > > > >> > >> > > > > transaction
> > >> > > > >> > >> > > > > > > via
> > >> > > > >> > >> > > > > > > > > >>>>>>>> addPartitionsToTxn. We check
> this
> > >> with
> > >> > > the
> > >> > > > >> > >> > > > > > > DescribeTransactions
> > >> > > > >> > >> > > > > > > > > >>> call.
> > >> > > > >> > >> > > > > > > > > >>>>>> Let
> > >> > > > >> > >> > > > > > > > > >>>>>>>> me know if this wasn't
> sufficiently
> > >> > > > >> explained
> > >> > > > >> > >> here:
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > >
> > >> > > > >> > >> > > > > > > >
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > >
> > >> > > > >> > >> > > > >
> > >> > > > >> > >> > > >
> > >> > > > >> > >> > >
> > >> > > > >> > >> >
> > >> > > > >> > >>
> > >> > > > >> >
> > >> > > > >>
> > >> > > >
> > >> > >
> > >> >
> > >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>> 40)
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>> The idea here is that if any
> > >> messages
> > >> > > > >> somehow
> > >> > > > >> > >> come
> > >> > > > >> > >> > in
> > >> > > > >> > >> > > > > before
> > >> > > > >> > >> > > > > > > we
> > >> > > > >> > >> > > > > > > > > get
> > >> > > > >> > >> > > > > > > > > >>>>> the
> > >> > > > >> > >> > > > > > > > > >>>>>>> new
> > >> > > > >> > >> > > > > > > > > >>>>>>>> epoch to the producer, they
> will be
> > >> > > > fenced.
> > >> > > > >> > >> However,
> > >> > > > >> > >> > > if
> > >> > > > >> > >> > > > we
> > >> > > > >> > >> > > > > > > don't
> > >> > > > >> > >> > > > > > > > > >>>>> think
> > >> > > > >> > >> > > > > > > > > >>>>>>> this
> > >> > > > >> > >> > > > > > > > > >>>>>>>> is necessary, it can be
> discussed
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>> 50)
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>> It should be synchronous
> because
> > >> if we
> > >> > > > have
> > >> > > > >> an
> > >> > > > >> > >> event
> > >> > > > >> > >> > > > (ie,
> > >> > > > >> > >> > > > > an
> > >> > > > >> > >> > > > > > > > > error)
> > >> > > > >> > >> > > > > > > > > >>>>>> that
> > >> > > > >> > >> > > > > > > > > >>>>>>>> causes us to need to abort the
> > >> > > > transaction,
> > >> > > > >> we
> > >> > > > >> > >> need
> > >> > > > >> > >> > to
> > >> > > > >> > >> > > > > know
> > >> > > > >> > >> > > > > > > > which
> > >> > > > >> > >> > > > > > > > > >>>>>>>> partitions to send transaction
> > >> markers
> > >> > > to.
> > >> > > > >> We
> > >> > > > >> > >> know
> > >> > > > >> > >> > the
> > >> > > > >> > >> > > > > > > > partitions
> > >> > > > >> > >> > > > > > > > > >>>>>> because
> > >> > > > >> > >> > > > > > > > > >>>>>>>> we added them to the
> coordinator
> > >> via
> > >> > the
> > >> > > > >> > >> > > > > addPartitionsToTxn
> > >> > > > >> > >> > > > > > > > call.
> > >> > > > >> > >> > > > > > > > > >>>>>>>> Previously we have had
> asynchronous
> > >> > > calls
> > >> > > > in
> > >> > > > >> > the
> > >> > > > >> > >> > past
> > >> > > > >> > >> > > > (ie,
> > >> > > > >> > >> > > > > > > > writing
> > >> > > > >> > >> > > > > > > > > >>>>> the
> > >> > > > >> > >> > > > > > > > > >>>>>>>> commit markers when the
> > >> transaction is
> > >> > > > >> > completed)
> > >> > > > >> > >> > but
> > >> > > > >> > >> > > > > often
> > >> > > > >> > >> > > > > > > this
> > >> > > > >> > >> > > > > > > > > >>> just
> > >> > > > >> > >> > > > > > > > > >>>>>>>> causes confusion as we need to
> wait
> > >> > for
> > >> > > > some
> > >> > > > >> > >> > > operations
> > >> > > > >> > >> > > > to
> > >> > > > >> > >> > > > > > > > > complete.
> > >> > > > >> > >> > > > > > > > > >>>>> In
> > >> > > > >> > >> > > > > > > > > >>>>>>> the
> > >> > > > >> > >> > > > > > > > > >>>>>>>> writing commit markers case,
> > >> clients
> > >> > > often
> > >> > > > >> see
> > >> > > > >> > >> > > > > > > > > >>>>> CONCURRENT_TRANSACTIONs
> > >> > > > >> > >> > > > > > > > > >>>>>>>> error messages and that can be
> > >> > > confusing.
> > >> > > > >> For
> > >> > > > >> > >> that
> > >> > > > >> > >> > > > reason,
> > >> > > > >> > >> > > > > > it
> > >> > > > >> > >> > > > > > > > may
> > >> > > > >> > >> > > > > > > > > be
> > >> > > > >> > >> > > > > > > > > >>>>>>>> simpler to just have
> synchronous
> > >> > calls —
> > >> > > > >> > >> especially
> > >> > > > >> > >> > if
> > >> > > > >> > >> > > > we
> > >> > > > >> > >> > > > > > need
> > >> > > > >> > >> > > > > > > > to
> > >> > > > >> > >> > > > > > > > > >>>>> block
> > >> > > > >> > >> > > > > > > > > >>>>>>> on
> > >> > > > >> > >> > > > > > > > > >>>>>>>> some operation's completion
> anyway
> > >> > > before
> > >> > > > we
> > >> > > > >> > can
> > >> > > > >> > >> > start
> > >> > > > >> > >> > > > the
> > >> > > > >> > >> > > > > > > next
> > >> > > > >> > >> > > > > > > > > >>>>>>>> transaction. And yes, I meant
> > >> > > > coordinator. I
> > >> > > > >> > will
> > >> > > > >> > >> > fix
> > >> > > > >> > >> > > > > that.
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>> 60)
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>> When we are checking if the
> > >> > transaction
> > >> > > is
> > >> > > > >> > >> ongoing,
> > >> > > > >> > >> > we
> > >> > > > >> > >> > > > > need
> > >> > > > >> > >> > > > > > to
> > >> > > > >> > >> > > > > > > > > make
> > >> > > > >> > >> > > > > > > > > >>> a
> > >> > > > >> > >> > > > > > > > > >>>>>>> round
> > >> > > > >> > >> > > > > > > > > >>>>>>>> trip from the leader partition
> to
> > >> the
> > >> > > > >> > transaction
> > >> > > > >> > >> > > > > > coordinator.
> > >> > > > >> > >> > > > > > > > In
> > >> > > > >> > >> > > > > > > > > >>> the
> > >> > > > >> > >> > > > > > > > > >>>>>>> time
> > >> > > > >> > >> > > > > > > > > >>>>>>>> we are waiting for this
> message to
> > >> > come
> > >> > > > >> back,
> > >> > > > >> > in
> > >> > > > >> > >> > > theory
> > >> > > > >> > >> > > > we
> > >> > > > >> > >> > > > > > > could
> > >> > > > >> > >> > > > > > > > > >>> have
> > >> > > > >> > >> > > > > > > > > >>>>>>> sent
> > >> > > > >> > >> > > > > > > > > >>>>>>>> a commit/abort call that would
> make
> > >> > the
> > >> > > > >> > original
> > >> > > > >> > >> > > result
> > >> > > > >> > >> > > > of
> > >> > > > >> > >> > > > > > the
> > >> > > > >> > >> > > > > > > > > check
> > >> > > > >> > >> > > > > > > > > >>>>>> out
> > >> > > > >> > >> > > > > > > > > >>>>>>> of
> > >> > > > >> > >> > > > > > > > > >>>>>>>> date. That is why we can check
> the
> > >> > > leader
> > >> > > > >> state
> > >> > > > >> > >> > before
> > >> > > > >> > >> > > > we
> > >> > > > >> > >> > > > > > > write
> > >> > > > >> > >> > > > > > > > to
> > >> > > > >> > >> > > > > > > > > >>>>> the
> > >> > > > >> > >> > > > > > > > > >>>>>>> log.
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>> I'm happy to update the KIP if
> > >> some of
> > >> > > > these
> > >> > > > >> > >> things
> > >> > > > >> > >> > > were
> > >> > > > >> > >> > > > > not
> > >> > > > >> > >> > > > > > > > > clear.
> > >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks,
> > >> > > > >> > >> > > > > > > > > >>>>>>>> Justine
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>> On Mon, Nov 21, 2022 at 7:11 PM
> > >> > Matthias
> > >> > > > J.
> > >> > > > >> > Sax <
> > >> > > > >> > >> > > > > > > > mjsax@apache.org
> > >> > > > >> > >> > > > > > > > > >
> > >> > > > >> > >> > > > > > > > > >>>>>>> wrote:
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> Thanks for the KIP.
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> Couple of clarification
> questions
> > >> (I
> > >> > am
> > >> > > > >> not a
> > >> > > > >> > >> > broker
> > >> > > > >> > >> > > > > expert
> > >> > > > >> > >> > > > > > > do
> > >> > > > >> > >> > > > > > > > > >>>>> maybe
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> some question are obvious for
> > >> others,
> > >> > > but
> > >> > > > >> not
> > >> > > > >> > >> for
> > >> > > > >> > >> > me
> > >> > > > >> > >> > > > with
> > >> > > > >> > >> > > > > > my
> > >> > > > >> > >> > > > > > > > lack
> > >> > > > >> > >> > > > > > > > > >>>>> of
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> broker knowledge).
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> (10)
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>> 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.
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> What happens if the message
> come
> > >> in
> > >> > > > before
> > >> > > > >> the
> > >> > > > >> > >> next
> > >> > > > >> > >> > > > > > > > > >>>>>> addPartitionsToTxn
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> request? It seems the broker
> > >> hosting
> > >> > > the
> > >> > > > >> data
> > >> > > > >> > >> > > > partitions
> > >> > > > >> > >> > > > > > > won't
> > >> > > > >> > >> > > > > > > > > know
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> anything about it and append
> it to
> > >> > the
> > >> > > > >> > >> partition,
> > >> > > > >> > >> > > too?
> > >> > > > >> > >> > > > > What
> > >> > > > >> > >> > > > > > > is
> > >> > > > >> > >> > > > > > > > > the
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> difference between both cases?
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> Also, it seems a TX would only
> > >> hang,
> > >> > if
> > >> > > > >> there
> > >> > > > >> > >> is no
> > >> > > > >> > >> > > > > > following
> > >> > > > >> > >> > > > > > > > TX
> > >> > > > >> > >> > > > > > > > > >>>>> that
> > >> > > > >> > >> > > > > > > > > >>>>>>> is
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> either committer or aborted?
> Thus,
> > >> > for
> > >> > > > the
> > >> > > > >> > case
> > >> > > > >> > >> > > above,
> > >> > > > >> > >> > > > > the
> > >> > > > >> > >> > > > > > TX
> > >> > > > >> > >> > > > > > > > > might
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> actually not hang (of course,
> we
> > >> > might
> > >> > > > get
> > >> > > > >> an
> > >> > > > >> > >> EOS
> > >> > > > >> > >> > > > > violation
> > >> > > > >> > >> > > > > > > if
> > >> > > > >> > >> > > > > > > > > the
> > >> > > > >> > >> > > > > > > > > >>>>>>> first
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> TX was aborted and the second
> > >> > > committed,
> > >> > > > or
> > >> > > > >> > the
> > >> > > > >> > >> > other
> > >> > > > >> > >> > > > way
> > >> > > > >> > >> > > > > > > > > around).
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> (20)
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Of course, 1 and 2 require
> > >> > client-side
> > >> > > > >> > >> changes, so
> > >> > > > >> > >> > > for
> > >> > > > >> > >> > > > > > older
> > >> > > > >> > >> > > > > > > > > >>>>>> clients,
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> those approaches won’t apply.
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> For (1) I understand why a
> client
> > >> > > change
> > >> > > > is
> > >> > > > >> > >> > > necessary,
> > >> > > > >> > >> > > > > but
> > >> > > > >> > >> > > > > > > not
> > >> > > > >> > >> > > > > > > > > sure
> > >> > > > >> > >> > > > > > > > > >>>>>> why
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> we need a client change for
> (2).
> > >> Can
> > >> > > you
> > >> > > > >> > >> elaborate?
> > >> > > > >> > >> > > --
> > >> > > > >> > >> > > > > > Later
> > >> > > > >> > >> > > > > > > > you
> > >> > > > >> > >> > > > > > > > > >>>>>>> explain
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> that we should send a
> > >> > > > >> > >> DescribeTransactionRequest,
> > >> > > > >> > >> > > but I
> > >> > > > >> > >> > > > > am
> > >> > > > >> > >> > > > > > > not
> > >> > > > >> > >> > > > > > > > > sure
> > >> > > > >> > >> > > > > > > > > >>>>>>> why?
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> Can't we not just do an
> implicit
> > >> > > > >> > >> AddPartiitonToTx,
> > >> > > > >> > >> > > too?
> > >> > > > >> > >> > > > > If
> > >> > > > >> > >> > > > > > > the
> > >> > > > >> > >> > > > > > > > > old
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> producer correctly registered
> the
> > >> > > > partition
> > >> > > > >> > >> > already,
> > >> > > > >> > >> > > > the
> > >> > > > >> > >> > > > > > > > > >>>>>> TX-coordinator
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> can just ignore it as it's an
> > >> > > idempotent
> > >> > > > >> > >> operation?
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> (30)
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>> To cover older clients, we
> will
> > >> > > ensure a
> > >> > > > >> > >> > transaction
> > >> > > > >> > >> > > > is
> > >> > > > >> > >> > > > > > > > ongoing
> > >> > > > >> > >> > > > > > > > > >>>>>>> before
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> we write to a transaction
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> Not sure what you mean by
> this?
> > >> Can
> > >> > you
> > >> > > > >> > >> elaborate?
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> (40)
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>> [the TX-coordinator] will
> write
> > >> the
> > >> > > > >> prepare
> > >> > > > >> > >> commit
> > >> > > > >> > >> > > > > message
> > >> > > > >> > >> > > > > > > > with
> > >> > > > >> > >> > > > > > > > > a
> > >> > > > >> > >> > > > > > > > > >>>>>>>> bumped
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> epoch and send
> > >> WriteTxnMarkerRequests
> > >> > > > with
> > >> > > > >> the
> > >> > > > >> > >> > bumped
> > >> > > > >> > >> > > > > > epoch.
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> Why do we use the bumped
> epoch for
> > >> > > both?
> > >> > > > It
> > >> > > > >> > >> seems
> > >> > > > >> > >> > > more
> > >> > > > >> > >> > > > > > > > intuitive
> > >> > > > >> > >> > > > > > > > > to
> > >> > > > >> > >> > > > > > > > > >>>>>> use
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> the current epoch, and only
> return
> > >> > the
> > >> > > > >> bumped
> > >> > > > >> > >> epoch
> > >> > > > >> > >> > > to
> > >> > > > >> > >> > > > > the
> > >> > > > >> > >> > > > > > > > > >>>>> producer?
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> (50) "Implicit
> > >> > > AddPartitionToTransaction"
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> Why does the implicitly sent
> > >> request
> > >> > > need
> > >> > > > >> to
> > >> > > > >> > be
> > >> > > > >> > >> > > > > > synchronous?
> > >> > > > >> > >> > > > > > > > The
> > >> > > > >> > >> > > > > > > > > >>>>> KIP
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> also says
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>> in case we need to abort and
> > >> need to
> > >> > > > know
> > >> > > > >> > which
> > >> > > > >> > >> > > > > partitions
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> What do you mean by this?
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>> we don’t want to write to it
> > >> before
> > >> > we
> > >> > > > >> store
> > >> > > > >> > in
> > >> > > > >> > >> > the
> > >> > > > >> > >> > > > > > > > transaction
> > >> > > > >> > >> > > > > > > > > >>>>>>> manager
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> Do you mean TX-coordinator
> > >> instead of
> > >> > > > >> > "manager"?
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> (60)
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> For older clients and ensuring
> > >> that
> > >> > the
> > >> > > > TX
> > >> > > > >> is
> > >> > > > >> > >> > > ongoing,
> > >> > > > >> > >> > > > > you
> > >> > > > >> > >> > > > > > > > > >>>>> describe a
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> race condition. I am not sure
> if I
> > >> > can
> > >> > > > >> follow
> > >> > > > >> > >> here.
> > >> > > > >> > >> > > Can
> > >> > > > >> > >> > > > > you
> > >> > > > >> > >> > > > > > > > > >>>>>> elaborate?
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> -Matthias
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> On 11/18/22 1:21 PM, Justine
> > >> Olshan
> > >> > > > wrote:
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Hey all!
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>> I'd like to start a
> discussion
> > >> on my
> > >> > > > >> proposal
> > >> > > > >> > >> to
> > >> > > > >> > >> > add
> > >> > > > >> > >> > > > > some
> > >> > > > >> > >> > > > > > > > > >>>>>> server-side
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>> checks on transactions to
> avoid
> > >> > > hanging
> > >> > > > >> > >> > > transactions.
> > >> > > > >> > >> > > > I
> > >> > > > >> > >> > > > > > know
> > >> > > > >> > >> > > > > > > > > this
> > >> > > > >> > >> > > > > > > > > >>>>>> has
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> been
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>> an issue for some time, so I
> > >> really
> > >> > > hope
> > >> > > > >> this
> > >> > > > >> > >> KIP
> > >> > > > >> > >> > > will
> > >> > > > >> > >> > > > > be
> > >> > > > >> > >> > > > > > > > > helpful
> > >> > > > >> > >> > > > > > > > > >>>>>> for
> > >> > > > >> > >> > > > > > > > > >>>>>>>>> many
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>> users of EOS.
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>> The KIP includes changes that
> > >> will
> > >> > be
> > >> > > > >> > >> compatible
> > >> > > > >> > >> > > with
> > >> > > > >> > >> > > > > old
> > >> > > > >> > >> > > > > > > > > clients
> > >> > > > >> > >> > > > > > > > > >>>>>> and
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>> changes to improve
> performance
> > >> and
> > >> > > > >> > correctness
> > >> > > > >> > >> on
> > >> > > > >> > >> > > new
> > >> > > > >> > >> > > > > > > clients.
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Please take a look and leave
> any
> > >> > > > comments
> > >> > > > >> you
> > >> > > > >> > >> may
> > >> > > > >> > >> > > > have!
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>> KIP:
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > >
> > >> > > > >> > >> > > > > > > >
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > >
> > >> > > > >> > >> > > > >
> > >> > > > >> > >> > > >
> > >> > > > >> > >> > >
> > >> > > > >> > >> >
> > >> > > > >> > >>
> > >> > > > >> >
> > >> > > > >>
> > >> > > >
> > >> > >
> > >> >
> > >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>> JIRA:
> > >> > > > >> > >> > > > https://issues.apache.org/jira/browse/KAFKA-14402
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Thanks!
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>> Justine
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>>
> > >> > > > >> > >> > > > > > > > > >>>>>
> > >> > > > >> > >> > > > > > > > > >>>>
> > >> > > > >> > >> > > > > > > > > >>>
> > >> > > > >> > >> > > > > > > > > >>
> > >> > > > >> > >> > > > > > > > > >
> > >> > > > >> > >> > > > > > > > >
> > >> > > > >> > >> > > > > > > >
> > >> > > > >> > >> > > > > > >
> > >> > > > >> > >> > > > > >
> > >> > > > >> > >> > > > >
> > >> > > > >> > >> > > >
> > >> > > > >> > >> > >
> > >> > > > >> > >> >
> > >> > > > >> > >>
> > >> > > > >> > >
> > >> > > > >> >
> > >> > > > >>
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Guozhang Wang <gu...@gmail.com>.
Hello Justine,

Thanks for the great write-up! I made a quick pass through it and here
are some thoughts (I have not been able to read through this thread so
pardon me if they have overlapped or subsumed by previous comments):

First are some meta ones:

1. I think we need to also improve the client's experience once we
have this defence in place. More concretely, say a user's producer
code is like following:

future = producer.send();
// producer.flush();
producer.commitTransaction();
future.get();

Which resulted in the order of a) produce-request sent by producer, b)
end-txn-request sent by producer, c) end-txn-response sent back, d)
txn-marker-request sent from coordinator to partition leader, e)
produce-request finally received by the partition leader, before this
KIP e) step would be accepted causing a dangling txn; now it would be
rejected in step e) which is good. But from the client's point of view
now it becomes confusing since the `commitTransaction()` returns
successfully, but the "future" throws an invalid-epoch error, and they
are not sure if the transaction did succeed or not. In fact, it
"partially succeeded" with some msgs being rejected but others
committed successfully.

Of course the easy way to avoid this is, always call
"producer.flush()" before commitTxn and that's what we do ourselves,
and what we recommend users do. But I suspect not everyone does it. In
fact I just checked the javadoc in KafkaProducer and our code snippet
does not include a `flush()` call. So I'm thinking maybe we can in
side the `commitTxn` code to enforce flushing before sending the
end-txn request.

2. I'd like to clarify a bit details on "just add partitions to the
transaction on the first produce request during a transaction". My
understanding is that the partition leader's cache has the producer id
/ sequence / epoch for the latest txn, either on-going or is completed
(upon receiving the marker request from coordinator). When a produce
request is received, if

* producer's epoch < cached epoch, or producer's epoch == cached epoch
but the latest txn is completed, leader directly reject with
invalid-epoch.
* producer's epoch > cached epoch, park the the request and send
add-partitions request to coordinator.

In order to do it, does the coordinator need to bump the sequence and
reset epoch to 0 when the next epoch is going to overflow? If no need
to do so, then how we handle the (admittedly rare, but still may
happen) epoch overflow situation?

3. I'm a bit concerned about adding a generic "ABORTABLE_ERROR" given
we already have a pretty messy error classification and error handling
on the producer clients side --- I have a summary about the issues and
a proposal to address this in
https://cwiki.apache.org/confluence/display/KAFKA/KIP-691%3A+Enhance+Transactional+Producer+Exception+Handling
-- I understand we do not want to use "UNKNOWN_PRODUCER_ID" anymore
and in fact we intend to deprecate it in KIP-360 and eventually remove
it; but I'm wondering can we still use specific error codes. E.g. what
about "InvalidProducerEpochException" since for new clients, the
actual reason this would actually be rejected is indeed because the
epoch on the coordinator caused the add-partitions-request from the
brokers to be rejected anyways?

4. It seems we put the producer request into purgatory before we ever
append the records, while other producer's records may still be
appended during the time; and that potentially may result in some
re-ordering compared with reception order. I'm not super concerned
about it since Kafka does not guarantee reception ordering across
producers anyways, but it may make the timestamps of records inside a
partition to be more out-of-ordered. Are we aware of any scenarios
such as future enhancements on log compactions that may be affected by
this effect?

Below are just minor comments:

5. In "AddPartitionsToTxnTransaction" field of
"AddPartitionsToTxnRequest" RPC, the versions of those inner fields
are "0-3" while I thought they should be "0+" still?

6. Regarding "we can place the request in a purgatory of sorts and
check if there is any state for the transaction on the broker": i
think at this time when we just do the checks against the cached
state, we do not need to put the request to purgatory yet?

7. This is related to 3) above. I feel using "InvalidRecordException"
for older clients may also be a bit confusing, and also it is not
fatal -- for old clients, it better to be fatal since this indicates
the clients is doing something wrong and hence it should be closed.
And in general I'd prefer to use slightly more specific meaning error
codes for clients. That being said, I also feel
"InvalidProducerEpochException" is not suitable for old versioned
clients, and we'd have to pick one that old clients recognize. I'd
prefer "InvalidTxnStateException" but that one is supposed to be
returned from txn coordinators only today. I'd suggest we do a quick
check in the current client's code path and see if that one would be
handled if it's from a produce-response, and if yes, use this one;
otherwise, use "ProducerFencedException" which is much less meaningful
but it's still a fatal error.


Thanks,
Guozhang



On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan
<jo...@confluent.io.invalid> wrote:
>
> Yeah -- looks like we already have code to handle bumping the epoch and
> when the epoch is Short.MAX_VALUE, we get a new producer ID. Since this is
> already the behavior, do we want to change it further?
>
> Justine
>
> On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <jo...@confluent.io> wrote:
>
> > Hey all, just wanted to quickly update and say I've modified the KIP to
> > explicitly mention that AddOffsetCommitsToTxnRequest will be replaced by
> > a coordinator-side (inter-broker) AddPartitionsToTxn implicit request. This
> > mirrors the user partitions and will implicitly add offset partitions to
> > transactions when we commit offsets on them. We will deprecate AddOffsetCommitsToTxnRequest
> > for new clients.
> >
> > Also to address Artem's comments --
> > I'm a bit unsure if the changes here will change the previous behavior for
> > fencing producers. In the case you mention in the first paragraph, are you
> > saying we bump the epoch before we try to abort the transaction? I think I
> > need to understand the scenarios you mention a bit better.
> >
> > As for the second part -- I think it makes sense to have some sort of
> > "sentinel" epoch to signal epoch is about to overflow (I think we sort of
> > have this value in place in some ways) so we can codify it in the KIP. I'll
> > look into that and try to update soon.
> >
> > Thanks,
> > Justine.
> >
> > On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
> > <al...@confluent.io.invalid> wrote:
> >
> >> It's good to know that KIP-588 addressed some of the issues.  Looking at
> >> the code, it still looks like there are some cases that would result in
> >> fatal error, e.g. PRODUCER_FENCED is issued by the transaction coordinator
> >> if epoch doesn't match, and the client treats it as a fatal error (code in
> >> TransactionManager request handling).  If we consider, for example,
> >> committing a transaction that returns a timeout, but actually succeeds,
> >> trying to abort it or re-commit may result in PRODUCER_FENCED error
> >> (because of epoch bump).
> >>
> >> For failed commits, specifically, we need to know the actual outcome,
> >> because if we return an error the application may think that the
> >> transaction is aborted and redo the work, leading to duplicates.
> >>
> >> Re: overflowing epoch.  We could either do it on the TC and return both
> >> producer id and epoch (e.g. change the protocol), or signal the client
> >> that
> >> it needs to get a new producer id.  Checking for max epoch could be a
> >> reasonable signal, the value to check should probably be present in the
> >> KIP
> >> as this is effectively a part of the contract.  Also, the TC should
> >> probably return an error if the client didn't change producer id after
> >> hitting max epoch.
> >>
> >> -Artem
> >>
> >>
> >> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
> >> <jo...@confluent.io.invalid> wrote:
> >>
> >> > Thanks for the discussion Artem.
> >> >
> >> > With respect to the handling of fenced producers, we have some behavior
> >> > already in place. As of KIP-588:
> >> >
> >> >
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> >> > ,
> >> > we handle timeouts more gracefully. The producer can recover.
> >> >
> >> > Produce requests can also recover from epoch fencing by aborting the
> >> > transaction and starting over.
> >> >
> >> > What other cases were you considering that would cause us to have a
> >> fenced
> >> > epoch but we'd want to recover?
> >> >
> >> > The first point about handling epoch overflows is fair. I think there is
> >> > some logic we'd need to consider. (ie, if we are one away from the max
> >> > epoch, we need to reset the producer ID.) I'm still wondering if there
> >> is a
> >> > way to direct this from the response, or if everything should be done on
> >> > the client side. Let me know if you have any thoughts here.
> >> >
> >> > Thanks,
> >> > Justine
> >> >
> >> > On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
> >> > <al...@confluent.io.invalid> wrote:
> >> >
> >> > > There are some workflows in the client that are implied by protocol
> >> > > changes, e.g.:
> >> > >
> >> > > - for new clients, epoch changes with every transaction and can
> >> overflow,
> >> > > in old clients this condition was handled transparently, because epoch
> >> > was
> >> > > bumped in InitProducerId and it would return a new producer id if
> >> epoch
> >> > > overflows, the new clients would need to implement some workflow to
> >> > refresh
> >> > > producer id
> >> > > - how to handle fenced producers, for new clients epoch changes with
> >> > every
> >> > > transaction, so in presence of failures during commits / aborts, the
> >> > > producer could get easily fenced, old clients would pretty much would
> >> get
> >> > > fenced when a new incarnation of the producer was initialized with
> >> > > InitProducerId so it's ok to treat as a fatal error, the new clients
> >> > would
> >> > > need to implement some workflow to handle that error, otherwise they
> >> > could
> >> > > get fenced by themselves
> >> > > - in particular (as a subset of the previous issue), what would the
> >> > client
> >> > > do if it got a timeout during commit?  commit could've succeeded or
> >> > failed
> >> > >
> >> > > Not sure if this has to be defined in the KIP as implementing those
> >> > > probably wouldn't require protocol changes, but we have multiple
> >> > > implementations of Kafka clients, so probably would be good to have
> >> some
> >> > > client implementation guidance.  Could also be done as a separate doc.
> >> > >
> >> > > -Artem
> >> > >
> >> > > On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
> >> > <jolshan@confluent.io.invalid
> >> > > >
> >> > > wrote:
> >> > >
> >> > > > Hey all, I've updated the KIP to incorporate Jason's suggestions.
> >> > > >
> >> > > >
> >> > > >
> >> > >
> >> >
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> >> > > >
> >> > > >
> >> > > > 1. Use AddPartitionsToTxn + verify flag to check on old clients
> >> > > > 2. Updated AddPartitionsToTxn API to support transaction batching
> >> > > > 3. Mention IBP bump
> >> > > > 4. Mention auth change on new AddPartitionsToTxn version.
> >> > > >
> >> > > > I'm planning on opening a vote soon.
> >> > > > Thanks,
> >> > > > Justine
> >> > > >
> >> > > > On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <jolshan@confluent.io
> >> >
> >> > > > wrote:
> >> > > >
> >> > > > > Thanks Jason. Those changes make sense to me. I will update the
> >> KIP.
> >> > > > >
> >> > > > >
> >> > > > >
> >> > > > > On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
> >> > > > <ja...@confluent.io.invalid>
> >> > > > > wrote:
> >> > > > >
> >> > > > >> Hey Justine,
> >> > > > >>
> >> > > > >> > I was wondering about compatibility here. When we send requests
> >> > > > >> between brokers, we want to ensure that the receiving broker
> >> > > understands
> >> > > > >> the request (specifically the new fields). Typically this is done
> >> > via
> >> > > > >> IBP/metadata version.
> >> > > > >> I'm trying to think if there is a way around it but I'm not sure
> >> > there
> >> > > > is.
> >> > > > >>
> >> > > > >> Yes. I think we would gate usage of this behind an IBP bump. Does
> >> > that
> >> > > > >> seem
> >> > > > >> reasonable?
> >> > > > >>
> >> > > > >> > As for the improvements -- can you clarify how the multiple
> >> > > > >> transactional
> >> > > > >> IDs would help here? Were you thinking of a case where we
> >> wait/batch
> >> > > > >> multiple produce requests together? My understanding for now was
> >> 1
> >> > > > >> transactional ID and one validation per 1 produce request.
> >> > > > >>
> >> > > > >> Each call to `AddPartitionsToTxn` is essentially a write to the
> >> > > > >> transaction
> >> > > > >> log and must block on replication. The more we can fit into a
> >> single
> >> > > > >> request, the more writes we can do in parallel. The alternative
> >> is
> >> > to
> >> > > > make
> >> > > > >> use of more connections, but usually we prefer batching since the
> >> > > > network
> >> > > > >> stack is not really optimized for high connection/request loads.
> >> > > > >>
> >> > > > >> > Finally with respect to the authorizations, I think it makes
> >> sense
> >> > > to
> >> > > > >> skip
> >> > > > >> topic authorizations, but I'm a bit confused by the "leader ID"
> >> > field.
> >> > > > >> Wouldn't we just want to flag the request as from a broker (does
> >> it
> >> > > > matter
> >> > > > >> which one?).
> >> > > > >>
> >> > > > >> We could also make it version-based. For the next version, we
> >> could
> >> > > > >> require
> >> > > > >> CLUSTER auth. So clients would not be able to use the API
> >> anymore,
> >> > > which
> >> > > > >> is
> >> > > > >> probably what we want.
> >> > > > >>
> >> > > > >> -Jason
> >> > > > >>
> >> > > > >> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> >> > > > >> <jo...@confluent.io.invalid>
> >> > > > >> wrote:
> >> > > > >>
> >> > > > >> > As a follow up, I was just thinking about the batching a bit
> >> more.
> >> > > > >> > I suppose if we have one request in flight and we queue up the
> >> > other
> >> > > > >> > produce requests in some sort of purgatory, we could send
> >> > > information
> >> > > > >> out
> >> > > > >> > for all of them rather than one by one. So that would be a
> >> benefit
> >> > > of
> >> > > > >> > batching partitions to add per transaction.
> >> > > > >> >
> >> > > > >> > I'll need to think a bit more on the design of this part of the
> >> > KIP,
> >> > > > and
> >> > > > >> > will update the KIP in the next few days.
> >> > > > >> >
> >> > > > >> > Thanks,
> >> > > > >> > Justine
> >> > > > >> >
> >> > > > >> > On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan <
> >> > > jolshan@confluent.io>
> >> > > > >> > wrote:
> >> > > > >> >
> >> > > > >> > > Hey Jason -- thanks for the input -- I was just digging a bit
> >> > > deeper
> >> > > > >> into
> >> > > > >> > > the design + implementation of the validation calls here and
> >> > what
> >> > > > you
> >> > > > >> say
> >> > > > >> > > makes sense.
> >> > > > >> > >
> >> > > > >> > > I was wondering about compatibility here. When we send
> >> requests
> >> > > > >> > > between brokers, we want to ensure that the receiving broker
> >> > > > >> understands
> >> > > > >> > > the request (specifically the new fields). Typically this is
> >> > done
> >> > > > via
> >> > > > >> > > IBP/metadata version.
> >> > > > >> > > I'm trying to think if there is a way around it but I'm not
> >> sure
> >> > > > there
> >> > > > >> > is.
> >> > > > >> > >
> >> > > > >> > > As for the improvements -- can you clarify how the multiple
> >> > > > >> transactional
> >> > > > >> > > IDs would help here? Were you thinking of a case where we
> >> > > wait/batch
> >> > > > >> > > multiple produce requests together? My understanding for now
> >> > was 1
> >> > > > >> > > transactional ID and one validation per 1 produce request.
> >> > > > >> > >
> >> > > > >> > > Finally with respect to the authorizations, I think it makes
> >> > sense
> >> > > > to
> >> > > > >> > skip
> >> > > > >> > > topic authorizations, but I'm a bit confused by the "leader
> >> ID"
> >> > > > field.
> >> > > > >> > > Wouldn't we just want to flag the request as from a broker
> >> (does
> >> > > it
> >> > > > >> > matter
> >> > > > >> > > which one?).
> >> > > > >> > >
> >> > > > >> > > I think I want to adopt these suggestions, just had a few
> >> > > questions
> >> > > > on
> >> > > > >> > the
> >> > > > >> > > details.
> >> > > > >> > >
> >> > > > >> > > Thanks,
> >> > > > >> > > Justine
> >> > > > >> > >
> >> > > > >> > > On Thu, Jan 5, 2023 at 5:05 PM Jason Gustafson
> >> > > > >> > <ja...@confluent.io.invalid>
> >> > > > >> > > wrote:
> >> > > > >> > >
> >> > > > >> > >> Hi Justine,
> >> > > > >> > >>
> >> > > > >> > >> Thanks for the proposal.
> >> > > > >> > >>
> >> > > > >> > >> I was thinking about the implementation a little bit. In the
> >> > > > current
> >> > > > >> > >> proposal, the behavior depends on whether we have an old or
> >> new
> >> > > > >> client.
> >> > > > >> > >> For
> >> > > > >> > >> old clients, we send `DescribeTransactions` and verify the
> >> > result
> >> > > > and
> >> > > > >> > for
> >> > > > >> > >> new clients, we send `AddPartitionsToTxn`. We might be able
> >> to
> >> > > > >> simplify
> >> > > > >> > >> the
> >> > > > >> > >> implementation if we can use the same request type. For
> >> > example,
> >> > > > >> what if
> >> > > > >> > >> we
> >> > > > >> > >> bump the protocol version for `AddPartitionsToTxn` and add a
> >> > > > >> > >> `validateOnly`
> >> > > > >> > >> flag? For older versions, we can set `validateOnly=true` so
> >> > that
> >> > > > the
> >> > > > >> > >> request only returns successfully if the partition had
> >> already
> >> > > been
> >> > > > >> > added.
> >> > > > >> > >> For new versions, we can set `validateOnly=false` and the
> >> > > partition
> >> > > > >> will
> >> > > > >> > >> be
> >> > > > >> > >> added to the transaction. The other slightly annoying thing
> >> > that
> >> > > > this
> >> > > > >> > >> would
> >> > > > >> > >> get around is the need to collect the transaction state for
> >> all
> >> > > > >> > partitions
> >> > > > >> > >> even when we only care about a subset.
> >> > > > >> > >>
> >> > > > >> > >> Some additional improvements to consider:
> >> > > > >> > >>
> >> > > > >> > >> - We can give `AddPartitionsToTxn` better batch support for
> >> > > > >> inter-broker
> >> > > > >> > >> usage. Currently we only allow one `TransactionalId` to be
> >> > > > specified,
> >> > > > >> > but
> >> > > > >> > >> the broker may get some benefit being able to batch across
> >> > > multiple
> >> > > > >> > >> transactions.
> >> > > > >> > >> - Another small improvement is skipping topic authorization
> >> > > checks
> >> > > > >> for
> >> > > > >> > >> `AddPartitionsToTxn` when the request is from a broker.
> >> Perhaps
> >> > > we
> >> > > > >> can
> >> > > > >> > add
> >> > > > >> > >> a field for the `LeaderId` or something like that and
> >> require
> >> > > > CLUSTER
> >> > > > >> > >> permission when set.
> >> > > > >> > >>
> >> > > > >> > >> Best,
> >> > > > >> > >> Jason
> >> > > > >> > >>
> >> > > > >> > >>
> >> > > > >> > >>
> >> > > > >> > >> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
> >> > <jun@confluent.io.invalid
> >> > > >
> >> > > > >> > wrote:
> >> > > > >> > >>
> >> > > > >> > >> > Hi, Justine,
> >> > > > >> > >> >
> >> > > > >> > >> > Thanks for the explanation. It makes sense to me now.
> >> > > > >> > >> >
> >> > > > >> > >> > Jun
> >> > > > >> > >> >
> >> > > > >> > >> > On Mon, Dec 19, 2022 at 1:42 PM Justine Olshan
> >> > > > >> > >> > <jo...@confluent.io.invalid>
> >> > > > >> > >> > wrote:
> >> > > > >> > >> >
> >> > > > >> > >> > > Hi Jun,
> >> > > > >> > >> > >
> >> > > > >> > >> > > My understanding of the mechanism is that when we get to
> >> > the
> >> > > > last
> >> > > > >> > >> epoch,
> >> > > > >> > >> > we
> >> > > > >> > >> > > increment to the fencing/last epoch and if any further
> >> > > requests
> >> > > > >> come
> >> > > > >> > >> in
> >> > > > >> > >> > for
> >> > > > >> > >> > > this producer ID they are fenced. Then the producer
> >> gets a
> >> > > new
> >> > > > ID
> >> > > > >> > and
> >> > > > >> > >> > > restarts with epoch/sequence 0. The fenced epoch sticks
> >> > > around
> >> > > > >> for
> >> > > > >> > the
> >> > > > >> > >> > > duration of producer.id.expiration.ms and blocks any
> >> late
> >> > > > >> messages
> >> > > > >> > >> > there.
> >> > > > >> > >> > > The new ID will get to take advantage of the improved
> >> > > semantics
> >> > > > >> > around
> >> > > > >> > >> > > non-zero start sequences. So I think we are covered.
> >> > > > >> > >> > >
> >> > > > >> > >> > > The only potential issue is overloading the cache, but
> >> > > > hopefully
> >> > > > >> the
> >> > > > >> > >> > > improvements (lowered producer.id.expiration.ms) will
> >> help
> >> > > > with
> >> > > > >> > that.
> >> > > > >> > >> > Let
> >> > > > >> > >> > > me know if you still have concerns.
> >> > > > >> > >> > >
> >> > > > >> > >> > > Thanks,
> >> > > > >> > >> > > Justine
> >> > > > >> > >> > >
> >> > > > >> > >> > > On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
> >> > > > >> <ju...@confluent.io.invalid>
> >> > > > >> > >> > wrote:
> >> > > > >> > >> > >
> >> > > > >> > >> > > > Hi, Justine,
> >> > > > >> > >> > > >
> >> > > > >> > >> > > > Thanks for the explanation.
> >> > > > >> > >> > > >
> >> > > > >> > >> > > > 70. The proposed fencing logic doesn't apply when pid
> >> > > > changes,
> >> > > > >> is
> >> > > > >> > >> that
> >> > > > >> > >> > > > right? If so, I am not sure how complete we are
> >> > addressing
> >> > > > this
> >> > > > >> > >> issue
> >> > > > >> > >> > if
> >> > > > >> > >> > > > the pid changes more frequently.
> >> > > > >> > >> > > >
> >> > > > >> > >> > > > Thanks,
> >> > > > >> > >> > > >
> >> > > > >> > >> > > > Jun
> >> > > > >> > >> > > >
> >> > > > >> > >> > > >
> >> > > > >> > >> > > >
> >> > > > >> > >> > > > On Fri, Dec 16, 2022 at 9:16 AM Justine Olshan
> >> > > > >> > >> > > > <jo...@confluent.io.invalid>
> >> > > > >> > >> > > > wrote:
> >> > > > >> > >> > > >
> >> > > > >> > >> > > > > Hi Jun,
> >> > > > >> > >> > > > >
> >> > > > >> > >> > > > > Thanks for replying!
> >> > > > >> > >> > > > >
> >> > > > >> > >> > > > > 70.We already do the overflow mechanism, so my
> >> change
> >> > > would
> >> > > > >> just
> >> > > > >> > >> make
> >> > > > >> > >> > > it
> >> > > > >> > >> > > > > happen more often.
> >> > > > >> > >> > > > > I was also not suggesting a new field in the log,
> >> but
> >> > in
> >> > > > the
> >> > > > >> > >> > response,
> >> > > > >> > >> > > > > which would be gated by the client version. Sorry if
> >> > > > >> something
> >> > > > >> > >> there
> >> > > > >> > >> > is
> >> > > > >> > >> > > > > unclear. I think we are starting to diverge.
> >> > > > >> > >> > > > > The goal of this KIP is to not change to the marker
> >> > > format
> >> > > > at
> >> > > > >> > all.
> >> > > > >> > >> > > > >
> >> > > > >> > >> > > > > 71. Yes, I guess I was going under the assumption
> >> that
> >> > > the
> >> > > > >> log
> >> > > > >> > >> would
> >> > > > >> > >> > > just
> >> > > > >> > >> > > > > look at its last epoch and treat it as the current
> >> > > epoch. I
> >> > > > >> > >> suppose
> >> > > > >> > >> > we
> >> > > > >> > >> > > > can
> >> > > > >> > >> > > > > have some special logic that if the last epoch was
> >> on a
> >> > > > >> marker
> >> > > > >> > we
> >> > > > >> > >> > > > actually
> >> > > > >> > >> > > > > expect the next epoch or something like that. We
> >> just
> >> > > need
> >> > > > to
> >> > > > >> > >> > > distinguish
> >> > > > >> > >> > > > > based on whether we had a commit/abort marker.
> >> > > > >> > >> > > > >
> >> > > > >> > >> > > > > 72.
> >> > > > >> > >> > > > > > if the producer epoch hasn't been bumped on the
> >> > > > >> > >> > > > > broker, it seems that the stucked message will fail
> >> the
> >> > > > >> sequence
> >> > > > >> > >> > > > validation
> >> > > > >> > >> > > > > and will be ignored. If the producer epoch has been
> >> > > bumped,
> >> > > > >> we
> >> > > > >> > >> ignore
> >> > > > >> > >> > > the
> >> > > > >> > >> > > > > sequence check and the stuck message could be
> >> appended
> >> > to
> >> > > > the
> >> > > > >> > log.
> >> > > > >> > >> > So,
> >> > > > >> > >> > > is
> >> > > > >> > >> > > > > the latter case that we want to guard?
> >> > > > >> > >> > > > >
> >> > > > >> > >> > > > > I'm not sure I follow that "the message will fail
> >> the
> >> > > > >> sequence
> >> > > > >> > >> > > > validation".
> >> > > > >> > >> > > > > In some of these cases, we had an abort marker (due
> >> to
> >> > an
> >> > > > >> error)
> >> > > > >> > >> and
> >> > > > >> > >> > > then
> >> > > > >> > >> > > > > the late message comes in with the correct sequence
> >> > > number.
> >> > > > >> This
> >> > > > >> > >> is a
> >> > > > >> > >> > > > case
> >> > > > >> > >> > > > > covered by the KIP.
> >> > > > >> > >> > > > > The latter case is actually not something we've
> >> > > considered
> >> > > > >> > here. I
> >> > > > >> > >> > > think
> >> > > > >> > >> > > > > generally when we bump the epoch, we are accepting
> >> that
> >> > > the
> >> > > > >> > >> sequence
> >> > > > >> > >> > > does
> >> > > > >> > >> > > > > not need to be checked anymore. My understanding is
> >> > also
> >> > > > >> that we
> >> > > > >> > >> > don't
> >> > > > >> > >> > > > > typically bump epoch mid transaction (based on a
> >> quick
> >> > > look
> >> > > > >> at
> >> > > > >> > the
> >> > > > >> > >> > > code)
> >> > > > >> > >> > > > > but let me know if that is the case.
> >> > > > >> > >> > > > >
> >> > > > >> > >> > > > > Thanks,
> >> > > > >> > >> > > > > Justine
> >> > > > >> > >> > > > >
> >> > > > >> > >> > > > > On Thu, Dec 15, 2022 at 12:23 PM Jun Rao
> >> > > > >> > <jun@confluent.io.invalid
> >> > > > >> > >> >
> >> > > > >> > >> > > > wrote:
> >> > > > >> > >> > > > >
> >> > > > >> > >> > > > > > Hi, Justine,
> >> > > > >> > >> > > > > >
> >> > > > >> > >> > > > > > Thanks for the reply.
> >> > > > >> > >> > > > > >
> >> > > > >> > >> > > > > > 70. Assigning a new pid on int overflow seems a
> >> bit
> >> > > > hacky.
> >> > > > >> If
> >> > > > >> > we
> >> > > > >> > >> > > need a
> >> > > > >> > >> > > > > txn
> >> > > > >> > >> > > > > > level id, it will be better to model this
> >> explicitly.
> >> > > > >> Adding a
> >> > > > >> > >> new
> >> > > > >> > >> > > > field
> >> > > > >> > >> > > > > > would require a bit more work since it requires a
> >> new
> >> > > txn
> >> > > > >> > marker
> >> > > > >> > >> > > format
> >> > > > >> > >> > > > > in
> >> > > > >> > >> > > > > > the log. So, we probably need to guard it with an
> >> IBP
> >> > > or
> >> > > > >> > >> metadata
> >> > > > >> > >> > > > version
> >> > > > >> > >> > > > > > and document the impact on downgrade once the new
> >> > > format
> >> > > > is
> >> > > > >> > >> written
> >> > > > >> > >> > > to
> >> > > > >> > >> > > > > the
> >> > > > >> > >> > > > > > log.
> >> > > > >> > >> > > > > >
> >> > > > >> > >> > > > > > 71. Hmm, once the marker is written, the partition
> >> > will
> >> > > > >> expect
> >> > > > >> > >> the
> >> > > > >> > >> > > next
> >> > > > >> > >> > > > > > append to be on the next epoch. Does that cover
> >> the
> >> > > case
> >> > > > >> you
> >> > > > >> > >> > > mentioned?
> >> > > > >> > >> > > > > >
> >> > > > >> > >> > > > > > 72. Also, just to be clear on the stucked message
> >> > issue
> >> > > > >> > >> described
> >> > > > >> > >> > in
> >> > > > >> > >> > > > the
> >> > > > >> > >> > > > > > motivation. With EoS, we also validate the
> >> sequence
> >> > id
> >> > > > for
> >> > > > >> > >> > > idempotency.
> >> > > > >> > >> > > > > So,
> >> > > > >> > >> > > > > > with the current logic, if the producer epoch
> >> hasn't
> >> > > been
> >> > > > >> > >> bumped on
> >> > > > >> > >> > > the
> >> > > > >> > >> > > > > > broker, it seems that the stucked message will
> >> fail
> >> > the
> >> > > > >> > sequence
> >> > > > >> > >> > > > > validation
> >> > > > >> > >> > > > > > and will be ignored. If the producer epoch has
> >> been
> >> > > > >> bumped, we
> >> > > > >> > >> > ignore
> >> > > > >> > >> > > > the
> >> > > > >> > >> > > > > > sequence check and the stuck message could be
> >> > appended
> >> > > to
> >> > > > >> the
> >> > > > >> > >> log.
> >> > > > >> > >> > > So,
> >> > > > >> > >> > > > is
> >> > > > >> > >> > > > > > the latter case that we want to guard?
> >> > > > >> > >> > > > > >
> >> > > > >> > >> > > > > > Thanks,
> >> > > > >> > >> > > > > >
> >> > > > >> > >> > > > > > Jun
> >> > > > >> > >> > > > > >
> >> > > > >> > >> > > > > > On Wed, Dec 14, 2022 at 10:44 AM Justine Olshan
> >> > > > >> > >> > > > > > <jo...@confluent.io.invalid> wrote:
> >> > > > >> > >> > > > > >
> >> > > > >> > >> > > > > > > Matthias — thanks again for taking time to look
> >> a
> >> > > this.
> >> > > > >> You
> >> > > > >> > >> said:
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > > My proposal was only focusing to avoid
> >> dangling
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > transactions if records are added without
> >> > registered
> >> > > > >> > >> partition.
> >> > > > >> > >> > --
> >> > > > >> > >> > > > > Maybe
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > you can add a few more details to the KIP about
> >> > this
> >> > > > >> > scenario
> >> > > > >> > >> for
> >> > > > >> > >> > > > > better
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > documentation purpose?
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > I'm not sure I understand what you mean here.
> >> The
> >> > > > >> motivation
> >> > > > >> > >> > > section
> >> > > > >> > >> > > > > > > describes two scenarios about how the record
> >> can be
> >> > > > added
> >> > > > >> > >> > without a
> >> > > > >> > >> > > > > > > registered partition:
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > > 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.
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > > Another way hanging transactions can occur is
> >> > that
> >> > > a
> >> > > > >> > client
> >> > > > >> > >> is
> >> > > > >> > >> > > > buggy
> >> > > > >> > >> > > > > > and
> >> > > > >> > >> > > > > > > may somehow try to write to a partition before
> >> it
> >> > > adds
> >> > > > >> the
> >> > > > >> > >> > > partition
> >> > > > >> > >> > > > to
> >> > > > >> > >> > > > > > the
> >> > > > >> > >> > > > > > > transaction.
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > For the first example of this would it be
> >> helpful
> >> > to
> >> > > > say
> >> > > > >> > that
> >> > > > >> > >> > this
> >> > > > >> > >> > > > > > message
> >> > > > >> > >> > > > > > > comes in after the abort, but before the
> >> partition
> >> > is
> >> > > > >> added
> >> > > > >> > to
> >> > > > >> > >> > the
> >> > > > >> > >> > > > next
> >> > > > >> > >> > > > > > > transaction so it becomes "hanging." Perhaps the
> >> > next
> >> > > > >> > sentence
> >> > > > >> > >> > > > > describing
> >> > > > >> > >> > > > > > > the message becoming part of the next
> >> transaction
> >> > (a
> >> > > > >> > different
> >> > > > >> > >> > > case)
> >> > > > >> > >> > > > > was
> >> > > > >> > >> > > > > > > not properly differentiated.
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > Jun — thanks for reading the KIP.
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > 70. The int typing was a concern. Currently we
> >> > have a
> >> > > > >> > >> mechanism
> >> > > > >> > >> > in
> >> > > > >> > >> > > > > place
> >> > > > >> > >> > > > > > to
> >> > > > >> > >> > > > > > > fence the final epoch when the epoch is about to
> >> > > > overflow
> >> > > > >> > and
> >> > > > >> > >> > > assign
> >> > > > >> > >> > > > a
> >> > > > >> > >> > > > > > new
> >> > > > >> > >> > > > > > > producer ID with epoch 0. Of course, this is a
> >> bit
> >> > > > tricky
> >> > > > >> > >> when it
> >> > > > >> > >> > > > comes
> >> > > > >> > >> > > > > > to
> >> > > > >> > >> > > > > > > the response back to the client.
> >> > > > >> > >> > > > > > > Making this a long could be another option, but
> >> I
> >> > > > wonder
> >> > > > >> are
> >> > > > >> > >> > there
> >> > > > >> > >> > > > any
> >> > > > >> > >> > > > > > > implications on changing this field if the
> >> epoch is
> >> > > > >> > persisted
> >> > > > >> > >> to
> >> > > > >> > >> > > > disk?
> >> > > > >> > >> > > > > > I'd
> >> > > > >> > >> > > > > > > need to check the usages.
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > 71.This was something Matthias asked about as
> >> > well. I
> >> > > > was
> >> > > > >> > >> > > > considering a
> >> > > > >> > >> > > > > > > possible edge case where a produce request from
> >> a
> >> > new
> >> > > > >> > >> transaction
> >> > > > >> > >> > > > > somehow
> >> > > > >> > >> > > > > > > gets sent right after the marker is written, but
> >> > > before
> >> > > > >> the
> >> > > > >> > >> > > producer
> >> > > > >> > >> > > > is
> >> > > > >> > >> > > > > > > alerted of the newly bumped epoch. In this
> >> case, we
> >> > > may
> >> > > > >> > >> include
> >> > > > >> > >> > > this
> >> > > > >> > >> > > > > > record
> >> > > > >> > >> > > > > > > when we don't want to. I suppose we could try
> >> to do
> >> > > > >> > something
> >> > > > >> > >> > > client
> >> > > > >> > >> > > > > side
> >> > > > >> > >> > > > > > > to bump the epoch after sending an endTxn as
> >> well
> >> > in
> >> > > > this
> >> > > > >> > >> > scenario
> >> > > > >> > >> > > —
> >> > > > >> > >> > > > > but
> >> > > > >> > >> > > > > > I
> >> > > > >> > >> > > > > > > wonder how it would work when the server is
> >> > aborting
> >> > > > >> based
> >> > > > >> > on
> >> > > > >> > >> a
> >> > > > >> > >> > > > > > server-side
> >> > > > >> > >> > > > > > > error. I could also be missing something and
> >> this
> >> > > > >> scenario
> >> > > > >> > is
> >> > > > >> > >> > > > actually
> >> > > > >> > >> > > > > > not
> >> > > > >> > >> > > > > > > possible.
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > Thanks again to everyone reading and commenting.
> >> > Let
> >> > > me
> >> > > > >> know
> >> > > > >> > >> > about
> >> > > > >> > >> > > > any
> >> > > > >> > >> > > > > > > further questions or comments.
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > Justine
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > On Wed, Dec 14, 2022 at 9:41 AM Jun Rao
> >> > > > >> > >> <jun@confluent.io.invalid
> >> > > > >> > >> > >
> >> > > > >> > >> > > > > > wrote:
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > > Hi, Justine,
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > > > Thanks for the KIP. A couple of comments.
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > > > 70. Currently, the producer epoch is an int.
> >> I am
> >> > > not
> >> > > > >> sure
> >> > > > >> > >> if
> >> > > > >> > >> > > it's
> >> > > > >> > >> > > > > > enough
> >> > > > >> > >> > > > > > > > to accommodate all transactions in the
> >> lifetime
> >> > of
> >> > > a
> >> > > > >> > >> producer.
> >> > > > >> > >> > > > Should
> >> > > > >> > >> > > > > > we
> >> > > > >> > >> > > > > > > > change that to a long or add a new long field
> >> > like
> >> > > > >> txnId?
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > > > 71. "it will write the prepare commit message
> >> > with
> >> > > a
> >> > > > >> > bumped
> >> > > > >> > >> > epoch
> >> > > > >> > >> > > > and
> >> > > > >> > >> > > > > > > send
> >> > > > >> > >> > > > > > > > WriteTxnMarkerRequests with the bumped epoch."
> >> > Hmm,
> >> > > > the
> >> > > > >> > >> epoch
> >> > > > >> > >> > is
> >> > > > >> > >> > > > > > > associated
> >> > > > >> > >> > > > > > > > with the current txn right? So, it seems
> >> weird to
> >> > > > >> write a
> >> > > > >> > >> > commit
> >> > > > >> > >> > > > > > message
> >> > > > >> > >> > > > > > > > with a bumped epoch. Should we only bump up
> >> the
> >> > > epoch
> >> > > > >> in
> >> > > > >> > >> > > > > EndTxnResponse
> >> > > > >> > >> > > > > > > and
> >> > > > >> > >> > > > > > > > rename the field to sth like
> >> nextProducerEpoch?
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > > > Thanks,
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > > > Jun
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > > > On Mon, Dec 12, 2022 at 8:54 PM Matthias J.
> >> Sax <
> >> > > > >> > >> > > mjsax@apache.org>
> >> > > > >> > >> > > > > > > wrote:
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > > > > Thanks for the background.
> >> > > > >> > >> > > > > > > > >
> >> > > > >> > >> > > > > > > > > 20/30: SGTM. My proposal was only focusing
> >> to
> >> > > avoid
> >> > > > >> > >> dangling
> >> > > > >> > >> > > > > > > > > transactions if records are added without
> >> > > > registered
> >> > > > >> > >> > partition.
> >> > > > >> > >> > > > --
> >> > > > >> > >> > > > > > > Maybe
> >> > > > >> > >> > > > > > > > > you can add a few more details to the KIP
> >> about
> >> > > > this
> >> > > > >> > >> scenario
> >> > > > >> > >> > > for
> >> > > > >> > >> > > > > > > better
> >> > > > >> > >> > > > > > > > > documentation purpose?
> >> > > > >> > >> > > > > > > > >
> >> > > > >> > >> > > > > > > > > 40: I think you hit a fair point about race
> >> > > > >> conditions
> >> > > > >> > or
> >> > > > >> > >> > > client
> >> > > > >> > >> > > > > bugs
> >> > > > >> > >> > > > > > > > > (incorrectly not bumping the epoch). The
> >> > > > >> > >> complexity/confusion
> >> > > > >> > >> > > for
> >> > > > >> > >> > > > > > using
> >> > > > >> > >> > > > > > > > > the bumped epoch I see, is mainly for
> >> internal
> >> > > > >> > debugging,
> >> > > > >> > >> ie,
> >> > > > >> > >> > > > > > > inspecting
> >> > > > >> > >> > > > > > > > > log segment dumps -- it seems harder to
> >> reason
> >> > > > about
> >> > > > >> the
> >> > > > >> > >> > system
> >> > > > >> > >> > > > for
> >> > > > >> > >> > > > > > us
> >> > > > >> > >> > > > > > > > > humans. But if we get better guarantees, it
> >> > would
> >> > > > be
> >> > > > >> > >> worth to
> >> > > > >> > >> > > use
> >> > > > >> > >> > > > > the
> >> > > > >> > >> > > > > > > > > bumped epoch.
> >> > > > >> > >> > > > > > > > >
> >> > > > >> > >> > > > > > > > > 60: as I mentioned already, I don't know the
> >> > > broker
> >> > > > >> > >> internals
> >> > > > >> > >> > > to
> >> > > > >> > >> > > > > > > provide
> >> > > > >> > >> > > > > > > > > more input. So if nobody else chimes in, we
> >> > > should
> >> > > > >> just
> >> > > > >> > >> move
> >> > > > >> > >> > > > > forward
> >> > > > >> > >> > > > > > > > > with your proposal.
> >> > > > >> > >> > > > > > > > >
> >> > > > >> > >> > > > > > > > >
> >> > > > >> > >> > > > > > > > > -Matthias
> >> > > > >> > >> > > > > > > > >
> >> > > > >> > >> > > > > > > > >
> >> > > > >> > >> > > > > > > > > On 12/6/22 4:22 PM, Justine Olshan wrote:
> >> > > > >> > >> > > > > > > > > > Hi all,
> >> > > > >> > >> > > > > > > > > > After Artem's questions about error
> >> behavior,
> >> > > > I've
> >> > > > >> > >> > > re-evaluated
> >> > > > >> > >> > > > > the
> >> > > > >> > >> > > > > > > > > > unknown producer ID exception and had some
> >> > > > >> discussions
> >> > > > >> > >> > > offline.
> >> > > > >> > >> > > > > > > > > >
> >> > > > >> > >> > > > > > > > > > I think generally it makes sense to
> >> simplify
> >> > > > error
> >> > > > >> > >> handling
> >> > > > >> > >> > > in
> >> > > > >> > >> > > > > > cases
> >> > > > >> > >> > > > > > > > like
> >> > > > >> > >> > > > > > > > > > this and the UNKNOWN_PRODUCER_ID error
> >> has a
> >> > > > pretty
> >> > > > >> > long
> >> > > > >> > >> > and
> >> > > > >> > >> > > > > > > > complicated
> >> > > > >> > >> > > > > > > > > > history. Because of this, I propose
> >> adding a
> >> > > new
> >> > > > >> error
> >> > > > >> > >> code
> >> > > > >> > >> > > > > > > > > ABORTABLE_ERROR
> >> > > > >> > >> > > > > > > > > > that when encountered by new clients
> >> (gated
> >> > by
> >> > > > the
> >> > > > >> > >> produce
> >> > > > >> > >> > > > > request
> >> > > > >> > >> > > > > > > > > version)
> >> > > > >> > >> > > > > > > > > > will simply abort the transaction. This
> >> > allows
> >> > > > the
> >> > > > >> > >> server
> >> > > > >> > >> > to
> >> > > > >> > >> > > > have
> >> > > > >> > >> > > > > > > some
> >> > > > >> > >> > > > > > > > > say
> >> > > > >> > >> > > > > > > > > > in whether the client aborts and makes
> >> > handling
> >> > > > >> much
> >> > > > >> > >> > simpler.
> >> > > > >> > >> > > > In
> >> > > > >> > >> > > > > > the
> >> > > > >> > >> > > > > > > > > > future, we can also use this error in
> >> other
> >> > > > >> situations
> >> > > > >> > >> > where
> >> > > > >> > >> > > we
> >> > > > >> > >> > > > > > want
> >> > > > >> > >> > > > > > > to
> >> > > > >> > >> > > > > > > > > > abort the transactions. We can even use on
> >> > > other
> >> > > > >> apis.
> >> > > > >> > >> > > > > > > > > >
> >> > > > >> > >> > > > > > > > > > I've added this to the KIP. Let me know if
> >> > > there
> >> > > > >> are
> >> > > > >> > any
> >> > > > >> > >> > > > > questions
> >> > > > >> > >> > > > > > or
> >> > > > >> > >> > > > > > > > > > issues.
> >> > > > >> > >> > > > > > > > > >
> >> > > > >> > >> > > > > > > > > > Justine
> >> > > > >> > >> > > > > > > > > >
> >> > > > >> > >> > > > > > > > > > On Fri, Dec 2, 2022 at 10:22 AM Justine
> >> > Olshan
> >> > > <
> >> > > > >> > >> > > > > > jolshan@confluent.io
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > > > > wrote:
> >> > > > >> > >> > > > > > > > > >
> >> > > > >> > >> > > > > > > > > >> Hey Matthias,
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >> 20/30 — Maybe I also didn't express
> >> myself
> >> > > > >> clearly.
> >> > > > >> > For
> >> > > > >> > >> > > older
> >> > > > >> > >> > > > > > > clients
> >> > > > >> > >> > > > > > > > we
> >> > > > >> > >> > > > > > > > > >> don't have a way to distinguish between a
> >> > > > previous
> >> > > > >> > and
> >> > > > >> > >> the
> >> > > > >> > >> > > > > current
> >> > > > >> > >> > > > > > > > > >> transaction since we don't have the epoch
> >> > > bump.
> >> > > > >> This
> >> > > > >> > >> means
> >> > > > >> > >> > > > that
> >> > > > >> > >> > > > > a
> >> > > > >> > >> > > > > > > late
> >> > > > >> > >> > > > > > > > > >> message from the previous transaction
> >> may be
> >> > > > >> added to
> >> > > > >> > >> the
> >> > > > >> > >> > > new
> >> > > > >> > >> > > > > one.
> >> > > > >> > >> > > > > > > > With
> >> > > > >> > >> > > > > > > > > >> older clients — we can't guarantee this
> >> > won't
> >> > > > >> happen
> >> > > > >> > >> if we
> >> > > > >> > >> > > > > already
> >> > > > >> > >> > > > > > > > sent
> >> > > > >> > >> > > > > > > > > the
> >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call (why we make
> >> changes
> >> > > for
> >> > > > >> the
> >> > > > >> > >> newer
> >> > > > >> > >> > > > > client)
> >> > > > >> > >> > > > > > > but
> >> > > > >> > >> > > > > > > > > we
> >> > > > >> > >> > > > > > > > > >> can at least gate some by ensuring that
> >> the
> >> > > > >> partition
> >> > > > >> > >> has
> >> > > > >> > >> > > been
> >> > > > >> > >> > > > > > added
> >> > > > >> > >> > > > > > > > to
> >> > > > >> > >> > > > > > > > > the
> >> > > > >> > >> > > > > > > > > >> transaction. The rationale here is that
> >> > there
> >> > > > are
> >> > > > >> > >> likely
> >> > > > >> > >> > > LESS
> >> > > > >> > >> > > > > late
> >> > > > >> > >> > > > > > > > > arrivals
> >> > > > >> > >> > > > > > > > > >> as time goes on, so hopefully most late
> >> > > arrivals
> >> > > > >> will
> >> > > > >> > >> come
> >> > > > >> > >> > > in
> >> > > > >> > >> > > > > > BEFORE
> >> > > > >> > >> > > > > > > > the
> >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call. Those that
> >> arrive
> >> > > > before
> >> > > > >> > will
> >> > > > >> > >> be
> >> > > > >> > >> > > > > properly
> >> > > > >> > >> > > > > > > > gated
> >> > > > >> > >> > > > > > > > > >> with the describeTransactions approach.
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >> If we take the approach you suggested,
> >> ANY
> >> > > late
> >> > > > >> > arrival
> >> > > > >> > >> > > from a
> >> > > > >> > >> > > > > > > > previous
> >> > > > >> > >> > > > > > > > > >> transaction will be added. And we don't
> >> want
> >> > > > >> that. I
> >> > > > >> > >> also
> >> > > > >> > >> > > > don't
> >> > > > >> > >> > > > > > see
> >> > > > >> > >> > > > > > > > any
> >> > > > >> > >> > > > > > > > > >> benefit in sending addPartitionsToTxn
> >> over
> >> > the
> >> > > > >> > >> > describeTxns
> >> > > > >> > >> > > > > call.
> >> > > > >> > >> > > > > > > They
> >> > > > >> > >> > > > > > > > > will
> >> > > > >> > >> > > > > > > > > >> both be one extra RPC to the Txn
> >> > coordinator.
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >> To be clear — newer clients will use
> >> > > > >> > addPartitionsToTxn
> >> > > > >> > >> > > > instead
> >> > > > >> > >> > > > > of
> >> > > > >> > >> > > > > > > the
> >> > > > >> > >> > > > > > > > > >> DescribeTxns.
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >> 40)
> >> > > > >> > >> > > > > > > > > >> My concern is that if we have some delay
> >> in
> >> > > the
> >> > > > >> > client
> >> > > > >> > >> to
> >> > > > >> > >> > > bump
> >> > > > >> > >> > > > > the
> >> > > > >> > >> > > > > > > > > epoch,
> >> > > > >> > >> > > > > > > > > >> it could continue to send epoch 73 and
> >> those
> >> > > > >> records
> >> > > > >> > >> would
> >> > > > >> > >> > > not
> >> > > > >> > >> > > > > be
> >> > > > >> > >> > > > > > > > > fenced.
> >> > > > >> > >> > > > > > > > > >> Perhaps this is not an issue if we don't
> >> > allow
> >> > > > the
> >> > > > >> > next
> >> > > > >> > >> > > > produce
> >> > > > >> > >> > > > > to
> >> > > > >> > >> > > > > > > go
> >> > > > >> > >> > > > > > > > > >> through before the EndTxn request
> >> returns.
> >> > I'm
> >> > > > >> also
> >> > > > >> > >> > thinking
> >> > > > >> > >> > > > > about
> >> > > > >> > >> > > > > > > > > cases of
> >> > > > >> > >> > > > > > > > > >> failure. I will need to think on this a
> >> bit.
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >> I wasn't sure if it was that confusing.
> >> But
> >> > if
> >> > > > we
> >> > > > >> > >> think it
> >> > > > >> > >> > > is,
> >> > > > >> > >> > > > > we
> >> > > > >> > >> > > > > > > can
> >> > > > >> > >> > > > > > > > > >> investigate other ways.
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >> 60)
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >> I'm not sure these are the same
> >> purgatories
> >> > > > since
> >> > > > >> one
> >> > > > >> > >> is a
> >> > > > >> > >> > > > > produce
> >> > > > >> > >> > > > > > > > > >> purgatory (I was planning on using a
> >> > callback
> >> > > > >> rather
> >> > > > >> > >> than
> >> > > > >> > >> > > > > > purgatory)
> >> > > > >> > >> > > > > > > > and
> >> > > > >> > >> > > > > > > > > >> the other is simply a request to append
> >> to
> >> > the
> >> > > > >> log.
> >> > > > >> > Not
> >> > > > >> > >> > sure
> >> > > > >> > >> > > > we
> >> > > > >> > >> > > > > > have
> >> > > > >> > >> > > > > > > > any
> >> > > > >> > >> > > > > > > > > >> structure here for ordering, but my
> >> > > > understanding
> >> > > > >> is
> >> > > > >> > >> that
> >> > > > >> > >> > > the
> >> > > > >> > >> > > > > > broker
> >> > > > >> > >> > > > > > > > > could
> >> > > > >> > >> > > > > > > > > >> handle the write request before it hears
> >> > back
> >> > > > from
> >> > > > >> > the
> >> > > > >> > >> Txn
> >> > > > >> > >> > > > > > > > Coordinator.
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >> Let me know if I misunderstood something
> >> or
> >> > > > >> something
> >> > > > >> > >> was
> >> > > > >> > >> > > > > unclear.
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >> Justine
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >> On Thu, Dec 1, 2022 at 12:15 PM Matthias
> >> J.
> >> > > Sax
> >> > > > <
> >> > > > >> > >> > > > > mjsax@apache.org
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > > > wrote:
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >>> Thanks for the details Justine!
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>> 20)
> >> > > > >> > >> > > > > > > > > >>>>
> >> > > > >> > >> > > > > > > > > >>>> The client side change for 2 is
> >> removing
> >> > the
> >> > > > >> > >> > addPartitions
> >> > > > >> > >> > > > to
> >> > > > >> > >> > > > > > > > > >>> transaction
> >> > > > >> > >> > > > > > > > > >>>> call. We don't need to make this from
> >> the
> >> > > > >> producer
> >> > > > >> > to
> >> > > > >> > >> > the
> >> > > > >> > >> > > > txn
> >> > > > >> > >> > > > > > > > > >>> coordinator,
> >> > > > >> > >> > > > > > > > > >>>> only server side.
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>> I think I did not express myself
> >> clearly. I
> >> > > > >> > understand
> >> > > > >> > >> > that
> >> > > > >> > >> > > > we
> >> > > > >> > >> > > > > > can
> >> > > > >> > >> > > > > > > > (and
> >> > > > >> > >> > > > > > > > > >>> should) change the producer to not send
> >> the
> >> > > > >> > >> > `addPartitions`
> >> > > > >> > >> > > > > > request
> >> > > > >> > >> > > > > > > > any
> >> > > > >> > >> > > > > > > > > >>> longer. But I don't thinks it's
> >> requirement
> >> > > to
> >> > > > >> > change
> >> > > > >> > >> the
> >> > > > >> > >> > > > > broker?
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>> What I am trying to say is: as a
> >> safe-guard
> >> > > and
> >> > > > >> > >> > improvement
> >> > > > >> > >> > > > for
> >> > > > >> > >> > > > > > > older
> >> > > > >> > >> > > > > > > > > >>> producers, the partition leader can just
> >> > send
> >> > > > the
> >> > > > >> > >> > > > > `addPartitions`
> >> > > > >> > >> > > > > > > > > >>> request to the TX-coordinator in any
> >> case
> >> > --
> >> > > if
> >> > > > >> the
> >> > > > >> > >> old
> >> > > > >> > >> > > > > producer
> >> > > > >> > >> > > > > > > > > >>> correctly did send the `addPartition`
> >> > request
> >> > > > to
> >> > > > >> the
> >> > > > >> > >> > > > > > TX-coordinator
> >> > > > >> > >> > > > > > > > > >>> already, the TX-coordinator can just
> >> > "ignore"
> >> > > > is
> >> > > > >> as
> >> > > > >> > >> > > > idempotent.
> >> > > > >> > >> > > > > > > > > However,
> >> > > > >> > >> > > > > > > > > >>> if the old producer has a bug and did
> >> > forget
> >> > > to
> >> > > > >> sent
> >> > > > >> > >> the
> >> > > > >> > >> > > > > > > > `addPartition`
> >> > > > >> > >> > > > > > > > > >>> request, we would now ensure that the
> >> > > partition
> >> > > > >> is
> >> > > > >> > >> indeed
> >> > > > >> > >> > > > added
> >> > > > >> > >> > > > > > to
> >> > > > >> > >> > > > > > > > the
> >> > > > >> > >> > > > > > > > > >>> TX and thus fix a potential producer bug
> >> > > (even
> >> > > > >> if we
> >> > > > >> > >> > don't
> >> > > > >> > >> > > > get
> >> > > > >> > >> > > > > > the
> >> > > > >> > >> > > > > > > > > >>> fencing via the bump epoch). -- It
> >> seems to
> >> > > be
> >> > > > a
> >> > > > >> > good
> >> > > > >> > >> > > > > > improvement?
> >> > > > >> > >> > > > > > > Or
> >> > > > >> > >> > > > > > > > > is
> >> > > > >> > >> > > > > > > > > >>> there a reason to not do this?
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>> 30)
> >> > > > >> > >> > > > > > > > > >>>>
> >> > > > >> > >> > > > > > > > > >>>> Transaction is ongoing = partition was
> >> > added
> >> > > > to
> >> > > > >> > >> > > transaction
> >> > > > >> > >> > > > > via
> >> > > > >> > >> > > > > > > > > >>>> addPartitionsToTxn. We check this with
> >> the
> >> > > > >> > >> > > > > DescribeTransactions
> >> > > > >> > >> > > > > > > > call.
> >> > > > >> > >> > > > > > > > > >>> Let
> >> > > > >> > >> > > > > > > > > >>>> me know if this wasn't sufficiently
> >> > > explained
> >> > > > >> here:
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>> If we do what I propose in (20), we
> >> don't
> >> > > > really
> >> > > > >> > need
> >> > > > >> > >> to
> >> > > > >> > >> > > make
> >> > > > >> > >> > > > > > this
> >> > > > >> > >> > > > > > > > > >>> `DescribeTransaction` call, as the
> >> > partition
> >> > > > >> leader
> >> > > > >> > >> adds
> >> > > > >> > >> > > the
> >> > > > >> > >> > > > > > > > partition
> >> > > > >> > >> > > > > > > > > >>> for older clients and we get this check
> >> for
> >> > > > free.
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>> 40)
> >> > > > >> > >> > > > > > > > > >>>>
> >> > > > >> > >> > > > > > > > > >>>> The idea here is that if any messages
> >> > > somehow
> >> > > > >> come
> >> > > > >> > in
> >> > > > >> > >> > > before
> >> > > > >> > >> > > > > we
> >> > > > >> > >> > > > > > > get
> >> > > > >> > >> > > > > > > > > the
> >> > > > >> > >> > > > > > > > > >>> new
> >> > > > >> > >> > > > > > > > > >>>> epoch to the producer, they will be
> >> > fenced.
> >> > > > >> > However,
> >> > > > >> > >> if
> >> > > > >> > >> > we
> >> > > > >> > >> > > > > don't
> >> > > > >> > >> > > > > > > > think
> >> > > > >> > >> > > > > > > > > >>> this
> >> > > > >> > >> > > > > > > > > >>>> is necessary, it can be discussed
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>> I agree that we should have epoch
> >> fencing.
> >> > My
> >> > > > >> > >> question is
> >> > > > >> > >> > > > > > > different:
> >> > > > >> > >> > > > > > > > > >>> Assume we are at epoch 73, and we have
> >> an
> >> > > > ongoing
> >> > > > >> > >> > > > transaction,
> >> > > > >> > >> > > > > > that
> >> > > > >> > >> > > > > > > > is
> >> > > > >> > >> > > > > > > > > >>> committed. It seems natural to write the
> >> > > > "prepare
> >> > > > >> > >> commit"
> >> > > > >> > >> > > > > marker
> >> > > > >> > >> > > > > > > and
> >> > > > >> > >> > > > > > > > > the
> >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` both with epoch
> >> 73,
> >> > > too,
> >> > > > >> as
> >> > > > >> > it
> >> > > > >> > >> > > belongs
> >> > > > >> > >> > > > > to
> >> > > > >> > >> > > > > > > the
> >> > > > >> > >> > > > > > > > > >>> current transaction. Of course, we now
> >> also
> >> > > > bump
> >> > > > >> the
> >> > > > >> > >> > epoch
> >> > > > >> > >> > > > and
> >> > > > >> > >> > > > > > > expect
> >> > > > >> > >> > > > > > > > > >>> the next requests to have epoch 74, and
> >> > would
> >> > > > >> reject
> >> > > > >> > >> an
> >> > > > >> > >> > > > request
> >> > > > >> > >> > > > > > > with
> >> > > > >> > >> > > > > > > > > >>> epoch 73, as the corresponding TX for
> >> epoch
> >> > > 73
> >> > > > >> was
> >> > > > >> > >> > already
> >> > > > >> > >> > > > > > > committed.
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>> It seems you propose to write the
> >> "prepare
> >> > > > commit
> >> > > > >> > >> marker"
> >> > > > >> > >> > > and
> >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` with epoch 74
> >> > though,
> >> > > > what
> >> > > > >> > >> would
> >> > > > >> > >> > > work,
> >> > > > >> > >> > > > > but
> >> > > > >> > >> > > > > > > it
> >> > > > >> > >> > > > > > > > > >>> seems confusing. Is there a reason why
> >> we
> >> > > would
> >> > > > >> use
> >> > > > >> > >> the
> >> > > > >> > >> > > > bumped
> >> > > > >> > >> > > > > > > epoch
> >> > > > >> > >> > > > > > > > 74
> >> > > > >> > >> > > > > > > > > >>> instead of the current epoch 73?
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>> 60)
> >> > > > >> > >> > > > > > > > > >>>>
> >> > > > >> > >> > > > > > > > > >>>> When we are checking if the
> >> transaction is
> >> > > > >> ongoing,
> >> > > > >> > >> we
> >> > > > >> > >> > > need
> >> > > > >> > >> > > > to
> >> > > > >> > >> > > > > > > make
> >> > > > >> > >> > > > > > > > a
> >> > > > >> > >> > > > > > > > > >>> round
> >> > > > >> > >> > > > > > > > > >>>> trip from the leader partition to the
> >> > > > >> transaction
> >> > > > >> > >> > > > coordinator.
> >> > > > >> > >> > > > > > In
> >> > > > >> > >> > > > > > > > the
> >> > > > >> > >> > > > > > > > > >>> time
> >> > > > >> > >> > > > > > > > > >>>> we are waiting for this message to come
> >> > > back,
> >> > > > in
> >> > > > >> > >> theory
> >> > > > >> > >> > we
> >> > > > >> > >> > > > > could
> >> > > > >> > >> > > > > > > > have
> >> > > > >> > >> > > > > > > > > >>> sent
> >> > > > >> > >> > > > > > > > > >>>> a commit/abort call that would make the
> >> > > > original
> >> > > > >> > >> result
> >> > > > >> > >> > of
> >> > > > >> > >> > > > the
> >> > > > >> > >> > > > > > > check
> >> > > > >> > >> > > > > > > > > >>> out of
> >> > > > >> > >> > > > > > > > > >>>> date. That is why we can check the
> >> leader
> >> > > > state
> >> > > > >> > >> before
> >> > > > >> > >> > we
> >> > > > >> > >> > > > > write
> >> > > > >> > >> > > > > > to
> >> > > > >> > >> > > > > > > > the
> >> > > > >> > >> > > > > > > > > >>> log.
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>> Thanks. Got it.
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>> However, is this really an issue? We put
> >> > the
> >> > > > >> produce
> >> > > > >> > >> > > request
> >> > > > >> > >> > > > in
> >> > > > >> > >> > > > > > > > > >>> purgatory, so how could we process the
> >> > > > >> > >> > > > `WriteTxnMarkerRequest`
> >> > > > >> > >> > > > > > > first?
> >> > > > >> > >> > > > > > > > > >>> Don't we need to put the
> >> > > > `WriteTxnMarkerRequest`
> >> > > > >> > into
> >> > > > >> > >> > > > > purgatory,
> >> > > > >> > >> > > > > > > too,
> >> > > > >> > >> > > > > > > > > >>> for this case, and process both request
> >> > > > in-order?
> >> > > > >> > >> (Again,
> >> > > > >> > >> > > my
> >> > > > >> > >> > > > > > broker
> >> > > > >> > >> > > > > > > > > >>> knowledge is limited and maybe we don't
> >> > > > maintain
> >> > > > >> > >> request
> >> > > > >> > >> > > > order
> >> > > > >> > >> > > > > > for
> >> > > > >> > >> > > > > > > > this
> >> > > > >> > >> > > > > > > > > >>> case, what seems to be an issue IMHO,
> >> and I
> >> > > am
> >> > > > >> > >> wondering
> >> > > > >> > >> > if
> >> > > > >> > >> > > > > > > changing
> >> > > > >> > >> > > > > > > > > >>> request handling to preserve order for
> >> this
> >> > > > case
> >> > > > >> > >> might be
> >> > > > >> > >> > > the
> >> > > > >> > >> > > > > > > cleaner
> >> > > > >> > >> > > > > > > > > >>> solution?)
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>> -Matthias
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>> On 11/30/22 3:28 PM, Artem Livshits
> >> wrote:
> >> > > > >> > >> > > > > > > > > >>>> Hi Justine,
> >> > > > >> > >> > > > > > > > > >>>>
> >> > > > >> > >> > > > > > > > > >>>> I think the interesting part is not in
> >> > this
> >> > > > >> logic
> >> > > > >> > >> > (because
> >> > > > >> > >> > > > it
> >> > > > >> > >> > > > > > > tries
> >> > > > >> > >> > > > > > > > to
> >> > > > >> > >> > > > > > > > > >>>> figure out when UNKNOWN_PRODUCER_ID is
> >> > > > retriable
> >> > > > >> > and
> >> > > > >> > >> if
> >> > > > >> > >> > > it's
> >> > > > >> > >> > > > > > > > > retryable,
> >> > > > >> > >> > > > > > > > > >>>> it's definitely not fatal), but what
> >> > happens
> >> > > > >> when
> >> > > > >> > >> this
> >> > > > >> > >> > > logic
> >> > > > >> > >> > > > > > > doesn't
> >> > > > >> > >> > > > > > > > > >>> return
> >> > > > >> > >> > > > > > > > > >>>> 'true' and falls through.  In the old
> >> > > clients
> >> > > > it
> >> > > > >> > >> seems
> >> > > > >> > >> > to
> >> > > > >> > >> > > be
> >> > > > >> > >> > > > > > > fatal,
> >> > > > >> > >> > > > > > > > if
> >> > > > >> > >> > > > > > > > > >>> we
> >> > > > >> > >> > > > > > > > > >>>> keep the behavior in the new clients,
> >> I'd
> >> > > > >> expect it
> >> > > > >> > >> > would
> >> > > > >> > >> > > be
> >> > > > >> > >> > > > > > fatal
> >> > > > >> > >> > > > > > > > as
> >> > > > >> > >> > > > > > > > > >>> well.
> >> > > > >> > >> > > > > > > > > >>>>
> >> > > > >> > >> > > > > > > > > >>>> -Artem
> >> > > > >> > >> > > > > > > > > >>>>
> >> > > > >> > >> > > > > > > > > >>>> On Tue, Nov 29, 2022 at 11:57 AM
> >> Justine
> >> > > > Olshan
> >> > > > >> > >> > > > > > > > > >>>> <jo...@confluent.io.invalid> wrote:
> >> > > > >> > >> > > > > > > > > >>>>
> >> > > > >> > >> > > > > > > > > >>>>> Hi Artem and Jeff,
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> Thanks for taking a look and sorry for
> >> > the
> >> > > > slow
> >> > > > >> > >> > response.
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> You both mentioned the change to
> >> handle
> >> > > > >> > >> > > UNKNOWN_PRODUCER_ID
> >> > > > >> > >> > > > > > > errors.
> >> > > > >> > >> > > > > > > > > To
> >> > > > >> > >> > > > > > > > > >>> be
> >> > > > >> > >> > > > > > > > > >>>>> clear — this error code will only be
> >> sent
> >> > > > again
> >> > > > >> > when
> >> > > > >> > >> > the
> >> > > > >> > >> > > > > > client's
> >> > > > >> > >> > > > > > > > > >>> request
> >> > > > >> > >> > > > > > > > > >>>>> version is high enough to ensure we
> >> > handle
> >> > > it
> >> > > > >> > >> > correctly.
> >> > > > >> > >> > > > > > > > > >>>>> The current (Java) client handles
> >> this by
> >> > > the
> >> > > > >> > >> following
> >> > > > >> > >> > > > > > (somewhat
> >> > > > >> > >> > > > > > > > > long)
> >> > > > >> > >> > > > > > > > > >>>>> code snippet:
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> // An UNKNOWN_PRODUCER_ID means that
> >> we
> >> > > have
> >> > > > >> lost
> >> > > > >> > >> the
> >> > > > >> > >> > > > > producer
> >> > > > >> > >> > > > > > > > state
> >> > > > >> > >> > > > > > > > > >>> on the
> >> > > > >> > >> > > > > > > > > >>>>> broker. Depending on the log start
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> // offset, we may want to retry
> >> these, as
> >> > > > >> > described
> >> > > > >> > >> for
> >> > > > >> > >> > > > each
> >> > > > >> > >> > > > > > case
> >> > > > >> > >> > > > > > > > > >>> below. If
> >> > > > >> > >> > > > > > > > > >>>>> none of those apply, then for the
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> // idempotent producer, we will
> >> locally
> >> > > bump
> >> > > > >> the
> >> > > > >> > >> epoch
> >> > > > >> > >> > > and
> >> > > > >> > >> > > > > > reset
> >> > > > >> > >> > > > > > > > the
> >> > > > >> > >> > > > > > > > > >>>>> sequence numbers of in-flight batches
> >> > from
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> // sequence 0, then retry the failed
> >> > batch,
> >> > > > >> which
> >> > > > >> > >> > should
> >> > > > >> > >> > > > now
> >> > > > >> > >> > > > > > > > succeed.
> >> > > > >> > >> > > > > > > > > >>> For
> >> > > > >> > >> > > > > > > > > >>>>> the transactional producer, allow the
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> // batch to fail. When processing the
> >> > > failed
> >> > > > >> > batch,
> >> > > > >> > >> we
> >> > > > >> > >> > > will
> >> > > > >> > >> > > > > > > > > transition
> >> > > > >> > >> > > > > > > > > >>> to
> >> > > > >> > >> > > > > > > > > >>>>> an abortable error and set a flag
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> // indicating that we need to bump the
> >> > > epoch
> >> > > > >> (if
> >> > > > >> > >> > > supported
> >> > > > >> > >> > > > by
> >> > > > >> > >> > > > > > the
> >> > > > >> > >> > > > > > > > > >>> broker).
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> if (error ==
> >> > Errors.*UNKNOWN_PRODUCER_ID*)
> >> > > {
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>       if (response.logStartOffset ==
> >> -1)
> >> > {
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // We don't know the log
> >> start
> >> > > > offset
> >> > > > >> > with
> >> > > > >> > >> > this
> >> > > > >> > >> > > > > > > response.
> >> > > > >> > >> > > > > > > > > We
> >> > > > >> > >> > > > > > > > > >>> should
> >> > > > >> > >> > > > > > > > > >>>>> just retry the request until we get
> >> it.
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // The UNKNOWN_PRODUCER_ID
> >> > error
> >> > > > code
> >> > > > >> > was
> >> > > > >> > >> > added
> >> > > > >> > >> > > > > along
> >> > > > >> > >> > > > > > > > with
> >> > > > >> > >> > > > > > > > > >>> the new
> >> > > > >> > >> > > > > > > > > >>>>> ProduceResponse which includes the
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // logStartOffset. So the
> >> '-1'
> >> > > > >> sentinel
> >> > > > >> > is
> >> > > > >> > >> > not
> >> > > > >> > >> > > > for
> >> > > > >> > >> > > > > > > > backward
> >> > > > >> > >> > > > > > > > > >>>>> compatibility. Instead, it is possible
> >> > for
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // a broker to not know the
> >> > > > >> > >> logStartOffset at
> >> > > > >> > >> > > > when
> >> > > > >> > >> > > > > it
> >> > > > >> > >> > > > > > > is
> >> > > > >> > >> > > > > > > > > >>> returning
> >> > > > >> > >> > > > > > > > > >>>>> the response because the partition
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // may have moved away from
> >> the
> >> > > > >> broker
> >> > > > >> > >> from
> >> > > > >> > >> > the
> >> > > > >> > >> > > > > time
> >> > > > >> > >> > > > > > > the
> >> > > > >> > >> > > > > > > > > >>> error was
> >> > > > >> > >> > > > > > > > > >>>>> initially raised to the time the
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // response was being
> >> > > constructed.
> >> > > > In
> >> > > > >> > >> these
> >> > > > >> > >> > > > cases,
> >> > > > >> > >> > > > > we
> >> > > > >> > >> > > > > > > > > should
> >> > > > >> > >> > > > > > > > > >>> just
> >> > > > >> > >> > > > > > > > > >>>>> retry the request: we are guaranteed
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // to eventually get a
> >> > > > logStartOffset
> >> > > > >> > once
> >> > > > >> > >> > > things
> >> > > > >> > >> > > > > > > settle
> >> > > > >> > >> > > > > > > > > down.
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           return true;
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>       }
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>       if
> >> (batch.sequenceHasBeenReset()) {
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // When the first inflight
> >> > batch
> >> > > > >> fails
> >> > > > >> > >> due to
> >> > > > >> > >> > > the
> >> > > > >> > >> > > > > > > > > truncation
> >> > > > >> > >> > > > > > > > > >>> case,
> >> > > > >> > >> > > > > > > > > >>>>> then the sequences of all the other
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // in flight batches would
> >> have
> >> > > > been
> >> > > > >> > >> > restarted
> >> > > > >> > >> > > > from
> >> > > > >> > >> > > > > > the
> >> > > > >> > >> > > > > > > > > >>> beginning.
> >> > > > >> > >> > > > > > > > > >>>>> However, when those responses
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // come back from the
> >> broker,
> >> > > they
> >> > > > >> would
> >> > > > >> > >> also
> >> > > > >> > >> > > > come
> >> > > > >> > >> > > > > > with
> >> > > > >> > >> > > > > > > > an
> >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID error. In this
> >> case,
> >> > we
> >> > > > >> should
> >> > > > >> > >> not
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // reset the sequence
> >> numbers
> >> > to
> >> > > > the
> >> > > > >> > >> > beginning.
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           return true;
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>       } else if
> >> > > > >> > >> > > > > (lastAckedOffset(batch.topicPartition).orElse(
> >> > > > >> > >> > > > > > > > > >>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> >> > > > >> > >> > > > response.logStartOffset) {
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // The head of the log has
> >> been
> >> > > > >> removed,
> >> > > > >> > >> > > probably
> >> > > > >> > >> > > > > due
> >> > > > >> > >> > > > > > > to
> >> > > > >> > >> > > > > > > > > the
> >> > > > >> > >> > > > > > > > > >>>>> retention time elapsing. In this case,
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // we expect to lose the
> >> > producer
> >> > > > >> state.
> >> > > > >> > >> For
> >> > > > >> > >> > > the
> >> > > > >> > >> > > > > > > > > transactional
> >> > > > >> > >> > > > > > > > > >>>>> producer, reset the sequences of all
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // inflight batches to be
> >> from
> >> > > the
> >> > > > >> > >> beginning
> >> > > > >> > >> > > and
> >> > > > >> > >> > > > > > retry
> >> > > > >> > >> > > > > > > > > them,
> >> > > > >> > >> > > > > > > > > >>> so
> >> > > > >> > >> > > > > > > > > >>>>> that the transaction does not need to
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // be aborted. For the
> >> > idempotent
> >> > > > >> > >> producer,
> >> > > > >> > >> > > bump
> >> > > > >> > >> > > > > the
> >> > > > >> > >> > > > > > > > epoch
> >> > > > >> > >> > > > > > > > > to
> >> > > > >> > >> > > > > > > > > >>> avoid
> >> > > > >> > >> > > > > > > > > >>>>> reusing (sequence, epoch) pairs
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           if (isTransactional()) {
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > >
> >> > > > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> >> > > > >> > >> > > > > > > > > >>>>> this.producerIdAndEpoch);
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           } else {
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > >
> >> requestEpochBumpForPartition(batch.topicPartition);
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           }
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           return true;
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>       }
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>       if (!isTransactional()) {
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // For the idempotent
> >> producer,
> >> > > > >> always
> >> > > > >> > >> retry
> >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> >> > > > >> > >> > > > > > > > > >>>>> errors. If the batch has the current
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // producer ID and epoch,
> >> > > request a
> >> > > > >> bump
> >> > > > >> > >> of
> >> > > > >> > >> > the
> >> > > > >> > >> > > > > > epoch.
> >> > > > >> > >> > > > > > > > > >>> Otherwise
> >> > > > >> > >> > > > > > > > > >>>>> just retry the produce.
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > >  requestEpochBumpForPartition(batch.topicPartition);
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           return true;
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>       }
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> }
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> I was considering keeping this
> >> behavior —
> >> > > but
> >> > > > >> am
> >> > > > >> > >> open
> >> > > > >> > >> > to
> >> > > > >> > >> > > > > > > > simplifying
> >> > > > >> > >> > > > > > > > > >>> it.
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> We are leaving changes to older
> >> clients
> >> > off
> >> > > > the
> >> > > > >> > >> table
> >> > > > >> > >> > > here
> >> > > > >> > >> > > > > > since
> >> > > > >> > >> > > > > > > it
> >> > > > >> > >> > > > > > > > > >>> caused
> >> > > > >> > >> > > > > > > > > >>>>> many issues for clients in the past.
> >> > > > Previously
> >> > > > >> > this
> >> > > > >> > >> > was
> >> > > > >> > >> > > a
> >> > > > >> > >> > > > > > fatal
> >> > > > >> > >> > > > > > > > > error
> >> > > > >> > >> > > > > > > > > >>> and
> >> > > > >> > >> > > > > > > > > >>>>> we didn't have the mechanisms in
> >> place to
> >> > > > >> detect
> >> > > > >> > >> when
> >> > > > >> > >> > > this
> >> > > > >> > >> > > > > was
> >> > > > >> > >> > > > > > a
> >> > > > >> > >> > > > > > > > > >>> legitimate
> >> > > > >> > >> > > > > > > > > >>>>> case vs some bug or gap in the
> >> protocol.
> >> > > > >> Ensuring
> >> > > > >> > >> each
> >> > > > >> > >> > > > > > > transaction
> >> > > > >> > >> > > > > > > > > has
> >> > > > >> > >> > > > > > > > > >>> its
> >> > > > >> > >> > > > > > > > > >>>>> own epoch should close this gap.
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> And to address Jeff's second point:
> >> > > > >> > >> > > > > > > > > >>>>> *does the typical produce request path
> >> > > append
> >> > > > >> > >> records
> >> > > > >> > >> > to
> >> > > > >> > >> > > > > local
> >> > > > >> > >> > > > > > > log
> >> > > > >> > >> > > > > > > > > >>> along*
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> *with the currentTxnFirstOffset
> >> > > information?
> >> > > > I
> >> > > > >> > would
> >> > > > >> > >> > like
> >> > > > >> > >> > > > to
> >> > > > >> > >> > > > > > > > > >>> understand*
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> *when the field is written to disk.*
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> Yes, the first produce request
> >> populates
> >> > > this
> >> > > > >> > field
> >> > > > >> > >> and
> >> > > > >> > >> > > > > writes
> >> > > > >> > >> > > > > > > the
> >> > > > >> > >> > > > > > > > > >>> offset
> >> > > > >> > >> > > > > > > > > >>>>> as part of the record batch and also
> >> to
> >> > the
> >> > > > >> > producer
> >> > > > >> > >> > > state
> >> > > > >> > >> > > > > > > > snapshot.
> >> > > > >> > >> > > > > > > > > >>> When
> >> > > > >> > >> > > > > > > > > >>>>> we reload the records on restart
> >> and/or
> >> > > > >> > >> reassignment,
> >> > > > >> > >> > we
> >> > > > >> > >> > > > > > > repopulate
> >> > > > >> > >> > > > > > > > > >>> this
> >> > > > >> > >> > > > > > > > > >>>>> field with the snapshot from disk
> >> along
> >> > > with
> >> > > > >> the
> >> > > > >> > >> rest
> >> > > > >> > >> > of
> >> > > > >> > >> > > > the
> >> > > > >> > >> > > > > > > > producer
> >> > > > >> > >> > > > > > > > > >>>>> state.
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> Let me know if there are further
> >> comments
> >> > > > >> and/or
> >> > > > >> > >> > > questions.
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> Thanks,
> >> > > > >> > >> > > > > > > > > >>>>> Justine
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> On Tue, Nov 22, 2022 at 9:00 PM Jeff
> >> Kim
> >> > > > >> > >> > > > > > > > > <jeff.kim@confluent.io.invalid
> >> > > > >> > >> > > > > > > > > >>>>
> >> > > > >> > >> > > > > > > > > >>>>> wrote:
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>> Hi Justine,
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>> Thanks for the KIP! I have two
> >> > questions:
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>> 1) For new clients, we can once again
> >> > > return
> >> > > > >> an
> >> > > > >> > >> error
> >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> >> > > > >> > >> > > > > > > > > >>>>>> for sequences
> >> > > > >> > >> > > > > > > > > >>>>>> that are non-zero when there is no
> >> > > producer
> >> > > > >> state
> >> > > > >> > >> > > present
> >> > > > >> > >> > > > on
> >> > > > >> > >> > > > > > the
> >> > > > >> > >> > > > > > > > > >>> server.
> >> > > > >> > >> > > > > > > > > >>>>>> This will indicate we missed the 0
> >> > > sequence
> >> > > > >> and
> >> > > > >> > we
> >> > > > >> > >> > don't
> >> > > > >> > >> > > > yet
> >> > > > >> > >> > > > > > > want
> >> > > > >> > >> > > > > > > > to
> >> > > > >> > >> > > > > > > > > >>>>> write
> >> > > > >> > >> > > > > > > > > >>>>>> to the log.
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>> I would like to understand the
> >> current
> >> > > > >> behavior
> >> > > > >> > to
> >> > > > >> > >> > > handle
> >> > > > >> > >> > > > > > older
> >> > > > >> > >> > > > > > > > > >>> clients,
> >> > > > >> > >> > > > > > > > > >>>>>> and if there are any changes we are
> >> > > making.
> >> > > > >> Maybe
> >> > > > >> > >> I'm
> >> > > > >> > >> > > > > missing
> >> > > > >> > >> > > > > > > > > >>> something,
> >> > > > >> > >> > > > > > > > > >>>>>> but we would want to identify
> >> whether we
> >> > > > >> missed
> >> > > > >> > >> the 0
> >> > > > >> > >> > > > > sequence
> >> > > > >> > >> > > > > > > for
> >> > > > >> > >> > > > > > > > > >>> older
> >> > > > >> > >> > > > > > > > > >>>>>> clients, no?
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>> 2) Upon returning from the
> >> transaction
> >> > > > >> > >> coordinator, we
> >> > > > >> > >> > > can
> >> > > > >> > >> > > > > set
> >> > > > >> > >> > > > > > > the
> >> > > > >> > >> > > > > > > > > >>>>>> transaction
> >> > > > >> > >> > > > > > > > > >>>>>> as ongoing on the leader by
> >> populating
> >> > > > >> > >> > > > currentTxnFirstOffset
> >> > > > >> > >> > > > > > > > > >>>>>> through the typical produce request
> >> > > > handling.
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>> does the typical produce request path
> >> > > append
> >> > > > >> > >> records
> >> > > > >> > >> > to
> >> > > > >> > >> > > > > local
> >> > > > >> > >> > > > > > > log
> >> > > > >> > >> > > > > > > > > >>> along
> >> > > > >> > >> > > > > > > > > >>>>>> with the currentTxnFirstOffset
> >> > > information?
> >> > > > I
> >> > > > >> > would
> >> > > > >> > >> > like
> >> > > > >> > >> > > > to
> >> > > > >> > >> > > > > > > > > understand
> >> > > > >> > >> > > > > > > > > >>>>>> when the field is written to disk.
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>> Thanks,
> >> > > > >> > >> > > > > > > > > >>>>>> Jeff
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>> On Tue, Nov 22, 2022 at 4:44 PM Artem
> >> > > > Livshits
> >> > > > >> > >> > > > > > > > > >>>>>> <al...@confluent.io.invalid>
> >> wrote:
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>> Hi Justine,
> >> > > > >> > >> > > > > > > > > >>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>> Thank you for the KIP.  I have one
> >> > > > question.
> >> > > > >> > >> > > > > > > > > >>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>> 5) For new clients, we can once
> >> again
> >> > > > return
> >> > > > >> an
> >> > > > >> > >> error
> >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> >> > > > >> > >> > > > > > > > > >>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>> I believe we had problems in the
> >> past
> >> > > with
> >> > > > >> > >> returning
> >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> >> > > > >> > >> > > > > > > > > >>>>>>> because it was considered fatal and
> >> > > > required
> >> > > > >> > >> client
> >> > > > >> > >> > > > > restart.
> >> > > > >> > >> > > > > > > It
> >> > > > >> > >> > > > > > > > > >>> would
> >> > > > >> > >> > > > > > > > > >>>>> be
> >> > > > >> > >> > > > > > > > > >>>>>>> good to spell out the new client
> >> > behavior
> >> > > > >> when
> >> > > > >> > it
> >> > > > >> > >> > > > receives
> >> > > > >> > >> > > > > > the
> >> > > > >> > >> > > > > > > > > error.
> >> > > > >> > >> > > > > > > > > >>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>> -Artem
> >> > > > >> > >> > > > > > > > > >>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>> On Tue, Nov 22, 2022 at 10:00 AM
> >> > Justine
> >> > > > >> Olshan
> >> > > > >> > >> > > > > > > > > >>>>>>> <jo...@confluent.io.invalid>
> >> wrote:
> >> > > > >> > >> > > > > > > > > >>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks for taking a look Matthias.
> >> > I've
> >> > > > >> tried
> >> > > > >> > to
> >> > > > >> > >> > > answer
> >> > > > >> > >> > > > > your
> >> > > > >> > >> > > > > > > > > >>>>> questions
> >> > > > >> > >> > > > > > > > > >>>>>>>> below:
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> 10)
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> Right — so the hanging transaction
> >> > only
> >> > > > >> occurs
> >> > > > >> > >> when
> >> > > > >> > >> > we
> >> > > > >> > >> > > > > have
> >> > > > >> > >> > > > > > a
> >> > > > >> > >> > > > > > > > late
> >> > > > >> > >> > > > > > > > > >>>>>>> message
> >> > > > >> > >> > > > > > > > > >>>>>>>> come in and the partition is never
> >> > added
> >> > > > to
> >> > > > >> a
> >> > > > >> > >> > > > transaction
> >> > > > >> > >> > > > > > > again.
> >> > > > >> > >> > > > > > > > > If
> >> > > > >> > >> > > > > > > > > >>>>> we
> >> > > > >> > >> > > > > > > > > >>>>>>>> never add the partition to a
> >> > > transaction,
> >> > > > we
> >> > > > >> > will
> >> > > > >> > >> > > never
> >> > > > >> > >> > > > > > write
> >> > > > >> > >> > > > > > > a
> >> > > > >> > >> > > > > > > > > >>>>> marker
> >> > > > >> > >> > > > > > > > > >>>>>>> and
> >> > > > >> > >> > > > > > > > > >>>>>>>> never advance the LSO.
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> If we do end up adding the
> >> partition
> >> > to
> >> > > > the
> >> > > > >> > >> > > transaction
> >> > > > >> > >> > > > (I
> >> > > > >> > >> > > > > > > > suppose
> >> > > > >> > >> > > > > > > > > >>>>> this
> >> > > > >> > >> > > > > > > > > >>>>>>> can
> >> > > > >> > >> > > > > > > > > >>>>>>>> happen before or after the late
> >> > message
> >> > > > >> comes
> >> > > > >> > in)
> >> > > > >> > >> > then
> >> > > > >> > >> > > > we
> >> > > > >> > >> > > > > > will
> >> > > > >> > >> > > > > > > > > >>>>> include
> >> > > > >> > >> > > > > > > > > >>>>>>> the
> >> > > > >> > >> > > > > > > > > >>>>>>>> late message in the next
> >> (incorrect)
> >> > > > >> > transaction.
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> So perhaps it is clearer to make
> >> the
> >> > > > >> > distinction
> >> > > > >> > >> > > between
> >> > > > >> > >> > > > > > > > messages
> >> > > > >> > >> > > > > > > > > >>>>> that
> >> > > > >> > >> > > > > > > > > >>>>>>>> eventually get added to the
> >> > transaction
> >> > > > (but
> >> > > > >> > the
> >> > > > >> > >> > wrong
> >> > > > >> > >> > > > > one)
> >> > > > >> > >> > > > > > or
> >> > > > >> > >> > > > > > > > > >>>>> messages
> >> > > > >> > >> > > > > > > > > >>>>>>>> that never get added and become
> >> > hanging.
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> 20)
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> The client side change for 2 is
> >> > removing
> >> > > > the
> >> > > > >> > >> > > > addPartitions
> >> > > > >> > >> > > > > > to
> >> > > > >> > >> > > > > > > > > >>>>>> transaction
> >> > > > >> > >> > > > > > > > > >>>>>>>> call. We don't need to make this
> >> from
> >> > > the
> >> > > > >> > >> producer
> >> > > > >> > >> > to
> >> > > > >> > >> > > > the
> >> > > > >> > >> > > > > > txn
> >> > > > >> > >> > > > > > > > > >>>>>>> coordinator,
> >> > > > >> > >> > > > > > > > > >>>>>>>> only server side.
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> In my opinion, the issue with the
> >> > > > >> > >> addPartitionsToTxn
> >> > > > >> > >> > > > call
> >> > > > >> > >> > > > > > for
> >> > > > >> > >> > > > > > > > > older
> >> > > > >> > >> > > > > > > > > >>>>>>> clients
> >> > > > >> > >> > > > > > > > > >>>>>>>> is that we don't have the epoch
> >> bump,
> >> > so
> >> > > > we
> >> > > > >> > don't
> >> > > > >> > >> > know
> >> > > > >> > >> > > > if
> >> > > > >> > >> > > > > > the
> >> > > > >> > >> > > > > > > > > >>> message
> >> > > > >> > >> > > > > > > > > >>>>>>>> belongs to the previous
> >> transaction or
> >> > > > this
> >> > > > >> > one.
> >> > > > >> > >> We
> >> > > > >> > >> > > need
> >> > > > >> > >> > > > > to
> >> > > > >> > >> > > > > > > > check
> >> > > > >> > >> > > > > > > > > if
> >> > > > >> > >> > > > > > > > > >>>>>> the
> >> > > > >> > >> > > > > > > > > >>>>>>>> partition has been added to this
> >> > > > >> transaction.
> >> > > > >> > Of
> >> > > > >> > >> > > course,
> >> > > > >> > >> > > > > > this
> >> > > > >> > >> > > > > > > > > means
> >> > > > >> > >> > > > > > > > > >>>>> we
> >> > > > >> > >> > > > > > > > > >>>>>>>> won't completely cover the case
> >> where
> >> > we
> >> > > > >> have a
> >> > > > >> > >> > really
> >> > > > >> > >> > > > > late
> >> > > > >> > >> > > > > > > > > message
> >> > > > >> > >> > > > > > > > > >>>>> and
> >> > > > >> > >> > > > > > > > > >>>>>>> we
> >> > > > >> > >> > > > > > > > > >>>>>>>> have added the partition to the new
> >> > > > >> > transaction,
> >> > > > >> > >> but
> >> > > > >> > >> > > > > that's
> >> > > > >> > >> > > > > > > > > >>>>>> unfortunately
> >> > > > >> > >> > > > > > > > > >>>>>>>> something we will need the new
> >> clients
> >> > > to
> >> > > > >> > cover.
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> 30)
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> Transaction is ongoing = partition
> >> was
> >> > > > >> added to
> >> > > > >> > >> > > > > transaction
> >> > > > >> > >> > > > > > > via
> >> > > > >> > >> > > > > > > > > >>>>>>>> addPartitionsToTxn. We check this
> >> with
> >> > > the
> >> > > > >> > >> > > > > > > DescribeTransactions
> >> > > > >> > >> > > > > > > > > >>> call.
> >> > > > >> > >> > > > > > > > > >>>>>> Let
> >> > > > >> > >> > > > > > > > > >>>>>>>> me know if this wasn't sufficiently
> >> > > > >> explained
> >> > > > >> > >> here:
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > >
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > >
> >> > > > >> > >> > > > >
> >> > > > >> > >> > > >
> >> > > > >> > >> > >
> >> > > > >> > >> >
> >> > > > >> > >>
> >> > > > >> >
> >> > > > >>
> >> > > >
> >> > >
> >> >
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> 40)
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> The idea here is that if any
> >> messages
> >> > > > >> somehow
> >> > > > >> > >> come
> >> > > > >> > >> > in
> >> > > > >> > >> > > > > before
> >> > > > >> > >> > > > > > > we
> >> > > > >> > >> > > > > > > > > get
> >> > > > >> > >> > > > > > > > > >>>>> the
> >> > > > >> > >> > > > > > > > > >>>>>>> new
> >> > > > >> > >> > > > > > > > > >>>>>>>> epoch to the producer, they will be
> >> > > > fenced.
> >> > > > >> > >> However,
> >> > > > >> > >> > > if
> >> > > > >> > >> > > > we
> >> > > > >> > >> > > > > > > don't
> >> > > > >> > >> > > > > > > > > >>>>> think
> >> > > > >> > >> > > > > > > > > >>>>>>> this
> >> > > > >> > >> > > > > > > > > >>>>>>>> is necessary, it can be discussed
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> 50)
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> It should be synchronous because
> >> if we
> >> > > > have
> >> > > > >> an
> >> > > > >> > >> event
> >> > > > >> > >> > > > (ie,
> >> > > > >> > >> > > > > an
> >> > > > >> > >> > > > > > > > > error)
> >> > > > >> > >> > > > > > > > > >>>>>> that
> >> > > > >> > >> > > > > > > > > >>>>>>>> causes us to need to abort the
> >> > > > transaction,
> >> > > > >> we
> >> > > > >> > >> need
> >> > > > >> > >> > to
> >> > > > >> > >> > > > > know
> >> > > > >> > >> > > > > > > > which
> >> > > > >> > >> > > > > > > > > >>>>>>>> partitions to send transaction
> >> markers
> >> > > to.
> >> > > > >> We
> >> > > > >> > >> know
> >> > > > >> > >> > the
> >> > > > >> > >> > > > > > > > partitions
> >> > > > >> > >> > > > > > > > > >>>>>> because
> >> > > > >> > >> > > > > > > > > >>>>>>>> we added them to the coordinator
> >> via
> >> > the
> >> > > > >> > >> > > > > addPartitionsToTxn
> >> > > > >> > >> > > > > > > > call.
> >> > > > >> > >> > > > > > > > > >>>>>>>> Previously we have had asynchronous
> >> > > calls
> >> > > > in
> >> > > > >> > the
> >> > > > >> > >> > past
> >> > > > >> > >> > > > (ie,
> >> > > > >> > >> > > > > > > > writing
> >> > > > >> > >> > > > > > > > > >>>>> the
> >> > > > >> > >> > > > > > > > > >>>>>>>> commit markers when the
> >> transaction is
> >> > > > >> > completed)
> >> > > > >> > >> > but
> >> > > > >> > >> > > > > often
> >> > > > >> > >> > > > > > > this
> >> > > > >> > >> > > > > > > > > >>> just
> >> > > > >> > >> > > > > > > > > >>>>>>>> causes confusion as we need to wait
> >> > for
> >> > > > some
> >> > > > >> > >> > > operations
> >> > > > >> > >> > > > to
> >> > > > >> > >> > > > > > > > > complete.
> >> > > > >> > >> > > > > > > > > >>>>> In
> >> > > > >> > >> > > > > > > > > >>>>>>> the
> >> > > > >> > >> > > > > > > > > >>>>>>>> writing commit markers case,
> >> clients
> >> > > often
> >> > > > >> see
> >> > > > >> > >> > > > > > > > > >>>>> CONCURRENT_TRANSACTIONs
> >> > > > >> > >> > > > > > > > > >>>>>>>> error messages and that can be
> >> > > confusing.
> >> > > > >> For
> >> > > > >> > >> that
> >> > > > >> > >> > > > reason,
> >> > > > >> > >> > > > > > it
> >> > > > >> > >> > > > > > > > may
> >> > > > >> > >> > > > > > > > > be
> >> > > > >> > >> > > > > > > > > >>>>>>>> simpler to just have synchronous
> >> > calls —
> >> > > > >> > >> especially
> >> > > > >> > >> > if
> >> > > > >> > >> > > > we
> >> > > > >> > >> > > > > > need
> >> > > > >> > >> > > > > > > > to
> >> > > > >> > >> > > > > > > > > >>>>> block
> >> > > > >> > >> > > > > > > > > >>>>>>> on
> >> > > > >> > >> > > > > > > > > >>>>>>>> some operation's completion anyway
> >> > > before
> >> > > > we
> >> > > > >> > can
> >> > > > >> > >> > start
> >> > > > >> > >> > > > the
> >> > > > >> > >> > > > > > > next
> >> > > > >> > >> > > > > > > > > >>>>>>>> transaction. And yes, I meant
> >> > > > coordinator. I
> >> > > > >> > will
> >> > > > >> > >> > fix
> >> > > > >> > >> > > > > that.
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> 60)
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> When we are checking if the
> >> > transaction
> >> > > is
> >> > > > >> > >> ongoing,
> >> > > > >> > >> > we
> >> > > > >> > >> > > > > need
> >> > > > >> > >> > > > > > to
> >> > > > >> > >> > > > > > > > > make
> >> > > > >> > >> > > > > > > > > >>> a
> >> > > > >> > >> > > > > > > > > >>>>>>> round
> >> > > > >> > >> > > > > > > > > >>>>>>>> trip from the leader partition to
> >> the
> >> > > > >> > transaction
> >> > > > >> > >> > > > > > coordinator.
> >> > > > >> > >> > > > > > > > In
> >> > > > >> > >> > > > > > > > > >>> the
> >> > > > >> > >> > > > > > > > > >>>>>>> time
> >> > > > >> > >> > > > > > > > > >>>>>>>> we are waiting for this message to
> >> > come
> >> > > > >> back,
> >> > > > >> > in
> >> > > > >> > >> > > theory
> >> > > > >> > >> > > > we
> >> > > > >> > >> > > > > > > could
> >> > > > >> > >> > > > > > > > > >>> have
> >> > > > >> > >> > > > > > > > > >>>>>>> sent
> >> > > > >> > >> > > > > > > > > >>>>>>>> a commit/abort call that would make
> >> > the
> >> > > > >> > original
> >> > > > >> > >> > > result
> >> > > > >> > >> > > > of
> >> > > > >> > >> > > > > > the
> >> > > > >> > >> > > > > > > > > check
> >> > > > >> > >> > > > > > > > > >>>>>> out
> >> > > > >> > >> > > > > > > > > >>>>>>> of
> >> > > > >> > >> > > > > > > > > >>>>>>>> date. That is why we can check the
> >> > > leader
> >> > > > >> state
> >> > > > >> > >> > before
> >> > > > >> > >> > > > we
> >> > > > >> > >> > > > > > > write
> >> > > > >> > >> > > > > > > > to
> >> > > > >> > >> > > > > > > > > >>>>> the
> >> > > > >> > >> > > > > > > > > >>>>>>> log.
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> I'm happy to update the KIP if
> >> some of
> >> > > > these
> >> > > > >> > >> things
> >> > > > >> > >> > > were
> >> > > > >> > >> > > > > not
> >> > > > >> > >> > > > > > > > > clear.
> >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks,
> >> > > > >> > >> > > > > > > > > >>>>>>>> Justine
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> On Mon, Nov 21, 2022 at 7:11 PM
> >> > Matthias
> >> > > > J.
> >> > > > >> > Sax <
> >> > > > >> > >> > > > > > > > mjsax@apache.org
> >> > > > >> > >> > > > > > > > > >
> >> > > > >> > >> > > > > > > > > >>>>>>> wrote:
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> Thanks for the KIP.
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> Couple of clarification questions
> >> (I
> >> > am
> >> > > > >> not a
> >> > > > >> > >> > broker
> >> > > > >> > >> > > > > expert
> >> > > > >> > >> > > > > > > do
> >> > > > >> > >> > > > > > > > > >>>>> maybe
> >> > > > >> > >> > > > > > > > > >>>>>>>>> some question are obvious for
> >> others,
> >> > > but
> >> > > > >> not
> >> > > > >> > >> for
> >> > > > >> > >> > me
> >> > > > >> > >> > > > with
> >> > > > >> > >> > > > > > my
> >> > > > >> > >> > > > > > > > lack
> >> > > > >> > >> > > > > > > > > >>>>> of
> >> > > > >> > >> > > > > > > > > >>>>>>>>> broker knowledge).
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> (10)
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> 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.
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> What happens if the message come
> >> in
> >> > > > before
> >> > > > >> the
> >> > > > >> > >> next
> >> > > > >> > >> > > > > > > > > >>>>>> addPartitionsToTxn
> >> > > > >> > >> > > > > > > > > >>>>>>>>> request? It seems the broker
> >> hosting
> >> > > the
> >> > > > >> data
> >> > > > >> > >> > > > partitions
> >> > > > >> > >> > > > > > > won't
> >> > > > >> > >> > > > > > > > > know
> >> > > > >> > >> > > > > > > > > >>>>>>>>> anything about it and append it to
> >> > the
> >> > > > >> > >> partition,
> >> > > > >> > >> > > too?
> >> > > > >> > >> > > > > What
> >> > > > >> > >> > > > > > > is
> >> > > > >> > >> > > > > > > > > the
> >> > > > >> > >> > > > > > > > > >>>>>>>>> difference between both cases?
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> Also, it seems a TX would only
> >> hang,
> >> > if
> >> > > > >> there
> >> > > > >> > >> is no
> >> > > > >> > >> > > > > > following
> >> > > > >> > >> > > > > > > > TX
> >> > > > >> > >> > > > > > > > > >>>>> that
> >> > > > >> > >> > > > > > > > > >>>>>>> is
> >> > > > >> > >> > > > > > > > > >>>>>>>>> either committer or aborted? Thus,
> >> > for
> >> > > > the
> >> > > > >> > case
> >> > > > >> > >> > > above,
> >> > > > >> > >> > > > > the
> >> > > > >> > >> > > > > > TX
> >> > > > >> > >> > > > > > > > > might
> >> > > > >> > >> > > > > > > > > >>>>>>>>> actually not hang (of course, we
> >> > might
> >> > > > get
> >> > > > >> an
> >> > > > >> > >> EOS
> >> > > > >> > >> > > > > violation
> >> > > > >> > >> > > > > > > if
> >> > > > >> > >> > > > > > > > > the
> >> > > > >> > >> > > > > > > > > >>>>>>> first
> >> > > > >> > >> > > > > > > > > >>>>>>>>> TX was aborted and the second
> >> > > committed,
> >> > > > or
> >> > > > >> > the
> >> > > > >> > >> > other
> >> > > > >> > >> > > > way
> >> > > > >> > >> > > > > > > > > around).
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> (20)
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> Of course, 1 and 2 require
> >> > client-side
> >> > > > >> > >> changes, so
> >> > > > >> > >> > > for
> >> > > > >> > >> > > > > > older
> >> > > > >> > >> > > > > > > > > >>>>>> clients,
> >> > > > >> > >> > > > > > > > > >>>>>>>>> those approaches won’t apply.
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> For (1) I understand why a client
> >> > > change
> >> > > > is
> >> > > > >> > >> > > necessary,
> >> > > > >> > >> > > > > but
> >> > > > >> > >> > > > > > > not
> >> > > > >> > >> > > > > > > > > sure
> >> > > > >> > >> > > > > > > > > >>>>>> why
> >> > > > >> > >> > > > > > > > > >>>>>>>>> we need a client change for (2).
> >> Can
> >> > > you
> >> > > > >> > >> elaborate?
> >> > > > >> > >> > > --
> >> > > > >> > >> > > > > > Later
> >> > > > >> > >> > > > > > > > you
> >> > > > >> > >> > > > > > > > > >>>>>>> explain
> >> > > > >> > >> > > > > > > > > >>>>>>>>> that we should send a
> >> > > > >> > >> DescribeTransactionRequest,
> >> > > > >> > >> > > but I
> >> > > > >> > >> > > > > am
> >> > > > >> > >> > > > > > > not
> >> > > > >> > >> > > > > > > > > sure
> >> > > > >> > >> > > > > > > > > >>>>>>> why?
> >> > > > >> > >> > > > > > > > > >>>>>>>>> Can't we not just do an implicit
> >> > > > >> > >> AddPartiitonToTx,
> >> > > > >> > >> > > too?
> >> > > > >> > >> > > > > If
> >> > > > >> > >> > > > > > > the
> >> > > > >> > >> > > > > > > > > old
> >> > > > >> > >> > > > > > > > > >>>>>>>>> producer correctly registered the
> >> > > > partition
> >> > > > >> > >> > already,
> >> > > > >> > >> > > > the
> >> > > > >> > >> > > > > > > > > >>>>>> TX-coordinator
> >> > > > >> > >> > > > > > > > > >>>>>>>>> can just ignore it as it's an
> >> > > idempotent
> >> > > > >> > >> operation?
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> (30)
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> To cover older clients, we will
> >> > > ensure a
> >> > > > >> > >> > transaction
> >> > > > >> > >> > > > is
> >> > > > >> > >> > > > > > > > ongoing
> >> > > > >> > >> > > > > > > > > >>>>>>> before
> >> > > > >> > >> > > > > > > > > >>>>>>>>> we write to a transaction
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> Not sure what you mean by this?
> >> Can
> >> > you
> >> > > > >> > >> elaborate?
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> (40)
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> [the TX-coordinator] will write
> >> the
> >> > > > >> prepare
> >> > > > >> > >> commit
> >> > > > >> > >> > > > > message
> >> > > > >> > >> > > > > > > > with
> >> > > > >> > >> > > > > > > > > a
> >> > > > >> > >> > > > > > > > > >>>>>>>> bumped
> >> > > > >> > >> > > > > > > > > >>>>>>>>> epoch and send
> >> WriteTxnMarkerRequests
> >> > > > with
> >> > > > >> the
> >> > > > >> > >> > bumped
> >> > > > >> > >> > > > > > epoch.
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> Why do we use the bumped epoch for
> >> > > both?
> >> > > > It
> >> > > > >> > >> seems
> >> > > > >> > >> > > more
> >> > > > >> > >> > > > > > > > intuitive
> >> > > > >> > >> > > > > > > > > to
> >> > > > >> > >> > > > > > > > > >>>>>> use
> >> > > > >> > >> > > > > > > > > >>>>>>>>> the current epoch, and only return
> >> > the
> >> > > > >> bumped
> >> > > > >> > >> epoch
> >> > > > >> > >> > > to
> >> > > > >> > >> > > > > the
> >> > > > >> > >> > > > > > > > > >>>>> producer?
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> (50) "Implicit
> >> > > AddPartitionToTransaction"
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> Why does the implicitly sent
> >> request
> >> > > need
> >> > > > >> to
> >> > > > >> > be
> >> > > > >> > >> > > > > > synchronous?
> >> > > > >> > >> > > > > > > > The
> >> > > > >> > >> > > > > > > > > >>>>> KIP
> >> > > > >> > >> > > > > > > > > >>>>>>>>> also says
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> in case we need to abort and
> >> need to
> >> > > > know
> >> > > > >> > which
> >> > > > >> > >> > > > > partitions
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> What do you mean by this?
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> we don’t want to write to it
> >> before
> >> > we
> >> > > > >> store
> >> > > > >> > in
> >> > > > >> > >> > the
> >> > > > >> > >> > > > > > > > transaction
> >> > > > >> > >> > > > > > > > > >>>>>>> manager
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> Do you mean TX-coordinator
> >> instead of
> >> > > > >> > "manager"?
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> (60)
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> For older clients and ensuring
> >> that
> >> > the
> >> > > > TX
> >> > > > >> is
> >> > > > >> > >> > > ongoing,
> >> > > > >> > >> > > > > you
> >> > > > >> > >> > > > > > > > > >>>>> describe a
> >> > > > >> > >> > > > > > > > > >>>>>>>>> race condition. I am not sure if I
> >> > can
> >> > > > >> follow
> >> > > > >> > >> here.
> >> > > > >> > >> > > Can
> >> > > > >> > >> > > > > you
> >> > > > >> > >> > > > > > > > > >>>>>> elaborate?
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> -Matthias
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> On 11/18/22 1:21 PM, Justine
> >> Olshan
> >> > > > wrote:
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> Hey all!
> >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> I'd like to start a discussion
> >> on my
> >> > > > >> proposal
> >> > > > >> > >> to
> >> > > > >> > >> > add
> >> > > > >> > >> > > > > some
> >> > > > >> > >> > > > > > > > > >>>>>> server-side
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> checks on transactions to avoid
> >> > > hanging
> >> > > > >> > >> > > transactions.
> >> > > > >> > >> > > > I
> >> > > > >> > >> > > > > > know
> >> > > > >> > >> > > > > > > > > this
> >> > > > >> > >> > > > > > > > > >>>>>> has
> >> > > > >> > >> > > > > > > > > >>>>>>>>> been
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> an issue for some time, so I
> >> really
> >> > > hope
> >> > > > >> this
> >> > > > >> > >> KIP
> >> > > > >> > >> > > will
> >> > > > >> > >> > > > > be
> >> > > > >> > >> > > > > > > > > helpful
> >> > > > >> > >> > > > > > > > > >>>>>> for
> >> > > > >> > >> > > > > > > > > >>>>>>>>> many
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> users of EOS.
> >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> The KIP includes changes that
> >> will
> >> > be
> >> > > > >> > >> compatible
> >> > > > >> > >> > > with
> >> > > > >> > >> > > > > old
> >> > > > >> > >> > > > > > > > > clients
> >> > > > >> > >> > > > > > > > > >>>>>> and
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> changes to improve performance
> >> and
> >> > > > >> > correctness
> >> > > > >> > >> on
> >> > > > >> > >> > > new
> >> > > > >> > >> > > > > > > clients.
> >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> Please take a look and leave any
> >> > > > comments
> >> > > > >> you
> >> > > > >> > >> may
> >> > > > >> > >> > > > have!
> >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> KIP:
> >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > >
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > >
> >> > > > >> > >> > > > >
> >> > > > >> > >> > > >
> >> > > > >> > >> > >
> >> > > > >> > >> >
> >> > > > >> > >>
> >> > > > >> >
> >> > > > >>
> >> > > >
> >> > >
> >> >
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> JIRA:
> >> > > > >> > >> > > > https://issues.apache.org/jira/browse/KAFKA-14402
> >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> Thanks!
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> Justine
> >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >
> >> > > > >> > >> > > > > > > > >
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > >
> >> > > > >> > >> > > > >
> >> > > > >> > >> > > >
> >> > > > >> > >> > >
> >> > > > >> > >> >
> >> > > > >> > >>
> >> > > > >> > >
> >> > > > >> >
> >> > > > >>
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Artem Livshits <al...@confluent.io.INVALID>.
>  looks like we already have code to handle bumping the epoch and
when the epoch is Short.MAX_VALUE, we get a new producer ID.

My understanding is that this logic is currently encapsulated in the broker
and the client doesn't really know at which epoch value the new producer id
is generated.  With the new protocol, the client would need to be aware.
We don't need to change the logic, just document it.  With our
implementation, once epoch reaches Short.MAX_VALUE it cannot be used
further, but a naïve client implementer may miss this point and it may be
missed in testing if the tests don't overflow the epoch, and then once they
hit the issue, it's not immediately obvious from the KIP how to handle it.
Explicitly documenting this point in the KIP would help to avoid (or
quickly resolve) such issues.

-Artem

On Wed, Jan 18, 2023 at 3:01 PM Justine Olshan <jo...@confluent.io.invalid>
wrote:

> Yeah -- looks like we already have code to handle bumping the epoch and
> when the epoch is Short.MAX_VALUE, we get a new producer ID. Since this is
> already the behavior, do we want to change it further?
>
> Justine
>
> On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <jo...@confluent.io>
> wrote:
>
> > Hey all, just wanted to quickly update and say I've modified the KIP to
> > explicitly mention that AddOffsetCommitsToTxnRequest will be replaced by
> > a coordinator-side (inter-broker) AddPartitionsToTxn implicit request.
> This
> > mirrors the user partitions and will implicitly add offset partitions to
> > transactions when we commit offsets on them. We will deprecate
> AddOffsetCommitsToTxnRequest
> > for new clients.
> >
> > Also to address Artem's comments --
> > I'm a bit unsure if the changes here will change the previous behavior
> for
> > fencing producers. In the case you mention in the first paragraph, are
> you
> > saying we bump the epoch before we try to abort the transaction? I think
> I
> > need to understand the scenarios you mention a bit better.
> >
> > As for the second part -- I think it makes sense to have some sort of
> > "sentinel" epoch to signal epoch is about to overflow (I think we sort of
> > have this value in place in some ways) so we can codify it in the KIP.
> I'll
> > look into that and try to update soon.
> >
> > Thanks,
> > Justine.
> >
> > On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
> > <al...@confluent.io.invalid> wrote:
> >
> >> It's good to know that KIP-588 addressed some of the issues.  Looking at
> >> the code, it still looks like there are some cases that would result in
> >> fatal error, e.g. PRODUCER_FENCED is issued by the transaction
> coordinator
> >> if epoch doesn't match, and the client treats it as a fatal error (code
> in
> >> TransactionManager request handling).  If we consider, for example,
> >> committing a transaction that returns a timeout, but actually succeeds,
> >> trying to abort it or re-commit may result in PRODUCER_FENCED error
> >> (because of epoch bump).
> >>
> >> For failed commits, specifically, we need to know the actual outcome,
> >> because if we return an error the application may think that the
> >> transaction is aborted and redo the work, leading to duplicates.
> >>
> >> Re: overflowing epoch.  We could either do it on the TC and return both
> >> producer id and epoch (e.g. change the protocol), or signal the client
> >> that
> >> it needs to get a new producer id.  Checking for max epoch could be a
> >> reasonable signal, the value to check should probably be present in the
> >> KIP
> >> as this is effectively a part of the contract.  Also, the TC should
> >> probably return an error if the client didn't change producer id after
> >> hitting max epoch.
> >>
> >> -Artem
> >>
> >>
> >> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
> >> <jo...@confluent.io.invalid> wrote:
> >>
> >> > Thanks for the discussion Artem.
> >> >
> >> > With respect to the handling of fenced producers, we have some
> behavior
> >> > already in place. As of KIP-588:
> >> >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> >> > ,
> >> > we handle timeouts more gracefully. The producer can recover.
> >> >
> >> > Produce requests can also recover from epoch fencing by aborting the
> >> > transaction and starting over.
> >> >
> >> > What other cases were you considering that would cause us to have a
> >> fenced
> >> > epoch but we'd want to recover?
> >> >
> >> > The first point about handling epoch overflows is fair. I think there
> is
> >> > some logic we'd need to consider. (ie, if we are one away from the max
> >> > epoch, we need to reset the producer ID.) I'm still wondering if there
> >> is a
> >> > way to direct this from the response, or if everything should be done
> on
> >> > the client side. Let me know if you have any thoughts here.
> >> >
> >> > Thanks,
> >> > Justine
> >> >
> >> > On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
> >> > <al...@confluent.io.invalid> wrote:
> >> >
> >> > > There are some workflows in the client that are implied by protocol
> >> > > changes, e.g.:
> >> > >
> >> > > - for new clients, epoch changes with every transaction and can
> >> overflow,
> >> > > in old clients this condition was handled transparently, because
> epoch
> >> > was
> >> > > bumped in InitProducerId and it would return a new producer id if
> >> epoch
> >> > > overflows, the new clients would need to implement some workflow to
> >> > refresh
> >> > > producer id
> >> > > - how to handle fenced producers, for new clients epoch changes with
> >> > every
> >> > > transaction, so in presence of failures during commits / aborts, the
> >> > > producer could get easily fenced, old clients would pretty much
> would
> >> get
> >> > > fenced when a new incarnation of the producer was initialized with
> >> > > InitProducerId so it's ok to treat as a fatal error, the new clients
> >> > would
> >> > > need to implement some workflow to handle that error, otherwise they
> >> > could
> >> > > get fenced by themselves
> >> > > - in particular (as a subset of the previous issue), what would the
> >> > client
> >> > > do if it got a timeout during commit?  commit could've succeeded or
> >> > failed
> >> > >
> >> > > Not sure if this has to be defined in the KIP as implementing those
> >> > > probably wouldn't require protocol changes, but we have multiple
> >> > > implementations of Kafka clients, so probably would be good to have
> >> some
> >> > > client implementation guidance.  Could also be done as a separate
> doc.
> >> > >
> >> > > -Artem
> >> > >
> >> > > On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
> >> > <jolshan@confluent.io.invalid
> >> > > >
> >> > > wrote:
> >> > >
> >> > > > Hey all, I've updated the KIP to incorporate Jason's suggestions.
> >> > > >
> >> > > >
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> >> > > >
> >> > > >
> >> > > > 1. Use AddPartitionsToTxn + verify flag to check on old clients
> >> > > > 2. Updated AddPartitionsToTxn API to support transaction batching
> >> > > > 3. Mention IBP bump
> >> > > > 4. Mention auth change on new AddPartitionsToTxn version.
> >> > > >
> >> > > > I'm planning on opening a vote soon.
> >> > > > Thanks,
> >> > > > Justine
> >> > > >
> >> > > > On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <
> jolshan@confluent.io
> >> >
> >> > > > wrote:
> >> > > >
> >> > > > > Thanks Jason. Those changes make sense to me. I will update the
> >> KIP.
> >> > > > >
> >> > > > >
> >> > > > >
> >> > > > > On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
> >> > > > <ja...@confluent.io.invalid>
> >> > > > > wrote:
> >> > > > >
> >> > > > >> Hey Justine,
> >> > > > >>
> >> > > > >> > I was wondering about compatibility here. When we send
> requests
> >> > > > >> between brokers, we want to ensure that the receiving broker
> >> > > understands
> >> > > > >> the request (specifically the new fields). Typically this is
> done
> >> > via
> >> > > > >> IBP/metadata version.
> >> > > > >> I'm trying to think if there is a way around it but I'm not
> sure
> >> > there
> >> > > > is.
> >> > > > >>
> >> > > > >> Yes. I think we would gate usage of this behind an IBP bump.
> Does
> >> > that
> >> > > > >> seem
> >> > > > >> reasonable?
> >> > > > >>
> >> > > > >> > As for the improvements -- can you clarify how the multiple
> >> > > > >> transactional
> >> > > > >> IDs would help here? Were you thinking of a case where we
> >> wait/batch
> >> > > > >> multiple produce requests together? My understanding for now
> was
> >> 1
> >> > > > >> transactional ID and one validation per 1 produce request.
> >> > > > >>
> >> > > > >> Each call to `AddPartitionsToTxn` is essentially a write to the
> >> > > > >> transaction
> >> > > > >> log and must block on replication. The more we can fit into a
> >> single
> >> > > > >> request, the more writes we can do in parallel. The alternative
> >> is
> >> > to
> >> > > > make
> >> > > > >> use of more connections, but usually we prefer batching since
> the
> >> > > > network
> >> > > > >> stack is not really optimized for high connection/request
> loads.
> >> > > > >>
> >> > > > >> > Finally with respect to the authorizations, I think it makes
> >> sense
> >> > > to
> >> > > > >> skip
> >> > > > >> topic authorizations, but I'm a bit confused by the "leader ID"
> >> > field.
> >> > > > >> Wouldn't we just want to flag the request as from a broker
> (does
> >> it
> >> > > > matter
> >> > > > >> which one?).
> >> > > > >>
> >> > > > >> We could also make it version-based. For the next version, we
> >> could
> >> > > > >> require
> >> > > > >> CLUSTER auth. So clients would not be able to use the API
> >> anymore,
> >> > > which
> >> > > > >> is
> >> > > > >> probably what we want.
> >> > > > >>
> >> > > > >> -Jason
> >> > > > >>
> >> > > > >> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> >> > > > >> <jo...@confluent.io.invalid>
> >> > > > >> wrote:
> >> > > > >>
> >> > > > >> > As a follow up, I was just thinking about the batching a bit
> >> more.
> >> > > > >> > I suppose if we have one request in flight and we queue up
> the
> >> > other
> >> > > > >> > produce requests in some sort of purgatory, we could send
> >> > > information
> >> > > > >> out
> >> > > > >> > for all of them rather than one by one. So that would be a
> >> benefit
> >> > > of
> >> > > > >> > batching partitions to add per transaction.
> >> > > > >> >
> >> > > > >> > I'll need to think a bit more on the design of this part of
> the
> >> > KIP,
> >> > > > and
> >> > > > >> > will update the KIP in the next few days.
> >> > > > >> >
> >> > > > >> > Thanks,
> >> > > > >> > Justine
> >> > > > >> >
> >> > > > >> > On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan <
> >> > > jolshan@confluent.io>
> >> > > > >> > wrote:
> >> > > > >> >
> >> > > > >> > > Hey Jason -- thanks for the input -- I was just digging a
> bit
> >> > > deeper
> >> > > > >> into
> >> > > > >> > > the design + implementation of the validation calls here
> and
> >> > what
> >> > > > you
> >> > > > >> say
> >> > > > >> > > makes sense.
> >> > > > >> > >
> >> > > > >> > > I was wondering about compatibility here. When we send
> >> requests
> >> > > > >> > > between brokers, we want to ensure that the receiving
> broker
> >> > > > >> understands
> >> > > > >> > > the request (specifically the new fields). Typically this
> is
> >> > done
> >> > > > via
> >> > > > >> > > IBP/metadata version.
> >> > > > >> > > I'm trying to think if there is a way around it but I'm not
> >> sure
> >> > > > there
> >> > > > >> > is.
> >> > > > >> > >
> >> > > > >> > > As for the improvements -- can you clarify how the multiple
> >> > > > >> transactional
> >> > > > >> > > IDs would help here? Were you thinking of a case where we
> >> > > wait/batch
> >> > > > >> > > multiple produce requests together? My understanding for
> now
> >> > was 1
> >> > > > >> > > transactional ID and one validation per 1 produce request.
> >> > > > >> > >
> >> > > > >> > > Finally with respect to the authorizations, I think it
> makes
> >> > sense
> >> > > > to
> >> > > > >> > skip
> >> > > > >> > > topic authorizations, but I'm a bit confused by the "leader
> >> ID"
> >> > > > field.
> >> > > > >> > > Wouldn't we just want to flag the request as from a broker
> >> (does
> >> > > it
> >> > > > >> > matter
> >> > > > >> > > which one?).
> >> > > > >> > >
> >> > > > >> > > I think I want to adopt these suggestions, just had a few
> >> > > questions
> >> > > > on
> >> > > > >> > the
> >> > > > >> > > details.
> >> > > > >> > >
> >> > > > >> > > Thanks,
> >> > > > >> > > Justine
> >> > > > >> > >
> >> > > > >> > > On Thu, Jan 5, 2023 at 5:05 PM Jason Gustafson
> >> > > > >> > <ja...@confluent.io.invalid>
> >> > > > >> > > wrote:
> >> > > > >> > >
> >> > > > >> > >> Hi Justine,
> >> > > > >> > >>
> >> > > > >> > >> Thanks for the proposal.
> >> > > > >> > >>
> >> > > > >> > >> I was thinking about the implementation a little bit. In
> the
> >> > > > current
> >> > > > >> > >> proposal, the behavior depends on whether we have an old
> or
> >> new
> >> > > > >> client.
> >> > > > >> > >> For
> >> > > > >> > >> old clients, we send `DescribeTransactions` and verify the
> >> > result
> >> > > > and
> >> > > > >> > for
> >> > > > >> > >> new clients, we send `AddPartitionsToTxn`. We might be
> able
> >> to
> >> > > > >> simplify
> >> > > > >> > >> the
> >> > > > >> > >> implementation if we can use the same request type. For
> >> > example,
> >> > > > >> what if
> >> > > > >> > >> we
> >> > > > >> > >> bump the protocol version for `AddPartitionsToTxn` and
> add a
> >> > > > >> > >> `validateOnly`
> >> > > > >> > >> flag? For older versions, we can set `validateOnly=true`
> so
> >> > that
> >> > > > the
> >> > > > >> > >> request only returns successfully if the partition had
> >> already
> >> > > been
> >> > > > >> > added.
> >> > > > >> > >> For new versions, we can set `validateOnly=false` and the
> >> > > partition
> >> > > > >> will
> >> > > > >> > >> be
> >> > > > >> > >> added to the transaction. The other slightly annoying
> thing
> >> > that
> >> > > > this
> >> > > > >> > >> would
> >> > > > >> > >> get around is the need to collect the transaction state
> for
> >> all
> >> > > > >> > partitions
> >> > > > >> > >> even when we only care about a subset.
> >> > > > >> > >>
> >> > > > >> > >> Some additional improvements to consider:
> >> > > > >> > >>
> >> > > > >> > >> - We can give `AddPartitionsToTxn` better batch support
> for
> >> > > > >> inter-broker
> >> > > > >> > >> usage. Currently we only allow one `TransactionalId` to be
> >> > > > specified,
> >> > > > >> > but
> >> > > > >> > >> the broker may get some benefit being able to batch across
> >> > > multiple
> >> > > > >> > >> transactions.
> >> > > > >> > >> - Another small improvement is skipping topic
> authorization
> >> > > checks
> >> > > > >> for
> >> > > > >> > >> `AddPartitionsToTxn` when the request is from a broker.
> >> Perhaps
> >> > > we
> >> > > > >> can
> >> > > > >> > add
> >> > > > >> > >> a field for the `LeaderId` or something like that and
> >> require
> >> > > > CLUSTER
> >> > > > >> > >> permission when set.
> >> > > > >> > >>
> >> > > > >> > >> Best,
> >> > > > >> > >> Jason
> >> > > > >> > >>
> >> > > > >> > >>
> >> > > > >> > >>
> >> > > > >> > >> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
> >> > <jun@confluent.io.invalid
> >> > > >
> >> > > > >> > wrote:
> >> > > > >> > >>
> >> > > > >> > >> > Hi, Justine,
> >> > > > >> > >> >
> >> > > > >> > >> > Thanks for the explanation. It makes sense to me now.
> >> > > > >> > >> >
> >> > > > >> > >> > Jun
> >> > > > >> > >> >
> >> > > > >> > >> > On Mon, Dec 19, 2022 at 1:42 PM Justine Olshan
> >> > > > >> > >> > <jo...@confluent.io.invalid>
> >> > > > >> > >> > wrote:
> >> > > > >> > >> >
> >> > > > >> > >> > > Hi Jun,
> >> > > > >> > >> > >
> >> > > > >> > >> > > My understanding of the mechanism is that when we get
> to
> >> > the
> >> > > > last
> >> > > > >> > >> epoch,
> >> > > > >> > >> > we
> >> > > > >> > >> > > increment to the fencing/last epoch and if any further
> >> > > requests
> >> > > > >> come
> >> > > > >> > >> in
> >> > > > >> > >> > for
> >> > > > >> > >> > > this producer ID they are fenced. Then the producer
> >> gets a
> >> > > new
> >> > > > ID
> >> > > > >> > and
> >> > > > >> > >> > > restarts with epoch/sequence 0. The fenced epoch
> sticks
> >> > > around
> >> > > > >> for
> >> > > > >> > the
> >> > > > >> > >> > > duration of producer.id.expiration.ms and blocks any
> >> late
> >> > > > >> messages
> >> > > > >> > >> > there.
> >> > > > >> > >> > > The new ID will get to take advantage of the improved
> >> > > semantics
> >> > > > >> > around
> >> > > > >> > >> > > non-zero start sequences. So I think we are covered.
> >> > > > >> > >> > >
> >> > > > >> > >> > > The only potential issue is overloading the cache, but
> >> > > > hopefully
> >> > > > >> the
> >> > > > >> > >> > > improvements (lowered producer.id.expiration.ms) will
> >> help
> >> > > > with
> >> > > > >> > that.
> >> > > > >> > >> > Let
> >> > > > >> > >> > > me know if you still have concerns.
> >> > > > >> > >> > >
> >> > > > >> > >> > > Thanks,
> >> > > > >> > >> > > Justine
> >> > > > >> > >> > >
> >> > > > >> > >> > > On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
> >> > > > >> <ju...@confluent.io.invalid>
> >> > > > >> > >> > wrote:
> >> > > > >> > >> > >
> >> > > > >> > >> > > > Hi, Justine,
> >> > > > >> > >> > > >
> >> > > > >> > >> > > > Thanks for the explanation.
> >> > > > >> > >> > > >
> >> > > > >> > >> > > > 70. The proposed fencing logic doesn't apply when
> pid
> >> > > > changes,
> >> > > > >> is
> >> > > > >> > >> that
> >> > > > >> > >> > > > right? If so, I am not sure how complete we are
> >> > addressing
> >> > > > this
> >> > > > >> > >> issue
> >> > > > >> > >> > if
> >> > > > >> > >> > > > the pid changes more frequently.
> >> > > > >> > >> > > >
> >> > > > >> > >> > > > Thanks,
> >> > > > >> > >> > > >
> >> > > > >> > >> > > > Jun
> >> > > > >> > >> > > >
> >> > > > >> > >> > > >
> >> > > > >> > >> > > >
> >> > > > >> > >> > > > On Fri, Dec 16, 2022 at 9:16 AM Justine Olshan
> >> > > > >> > >> > > > <jo...@confluent.io.invalid>
> >> > > > >> > >> > > > wrote:
> >> > > > >> > >> > > >
> >> > > > >> > >> > > > > Hi Jun,
> >> > > > >> > >> > > > >
> >> > > > >> > >> > > > > Thanks for replying!
> >> > > > >> > >> > > > >
> >> > > > >> > >> > > > > 70.We already do the overflow mechanism, so my
> >> change
> >> > > would
> >> > > > >> just
> >> > > > >> > >> make
> >> > > > >> > >> > > it
> >> > > > >> > >> > > > > happen more often.
> >> > > > >> > >> > > > > I was also not suggesting a new field in the log,
> >> but
> >> > in
> >> > > > the
> >> > > > >> > >> > response,
> >> > > > >> > >> > > > > which would be gated by the client version. Sorry
> if
> >> > > > >> something
> >> > > > >> > >> there
> >> > > > >> > >> > is
> >> > > > >> > >> > > > > unclear. I think we are starting to diverge.
> >> > > > >> > >> > > > > The goal of this KIP is to not change to the
> marker
> >> > > format
> >> > > > at
> >> > > > >> > all.
> >> > > > >> > >> > > > >
> >> > > > >> > >> > > > > 71. Yes, I guess I was going under the assumption
> >> that
> >> > > the
> >> > > > >> log
> >> > > > >> > >> would
> >> > > > >> > >> > > just
> >> > > > >> > >> > > > > look at its last epoch and treat it as the current
> >> > > epoch. I
> >> > > > >> > >> suppose
> >> > > > >> > >> > we
> >> > > > >> > >> > > > can
> >> > > > >> > >> > > > > have some special logic that if the last epoch was
> >> on a
> >> > > > >> marker
> >> > > > >> > we
> >> > > > >> > >> > > > actually
> >> > > > >> > >> > > > > expect the next epoch or something like that. We
> >> just
> >> > > need
> >> > > > to
> >> > > > >> > >> > > distinguish
> >> > > > >> > >> > > > > based on whether we had a commit/abort marker.
> >> > > > >> > >> > > > >
> >> > > > >> > >> > > > > 72.
> >> > > > >> > >> > > > > > if the producer epoch hasn't been bumped on the
> >> > > > >> > >> > > > > broker, it seems that the stucked message will
> fail
> >> the
> >> > > > >> sequence
> >> > > > >> > >> > > > validation
> >> > > > >> > >> > > > > and will be ignored. If the producer epoch has
> been
> >> > > bumped,
> >> > > > >> we
> >> > > > >> > >> ignore
> >> > > > >> > >> > > the
> >> > > > >> > >> > > > > sequence check and the stuck message could be
> >> appended
> >> > to
> >> > > > the
> >> > > > >> > log.
> >> > > > >> > >> > So,
> >> > > > >> > >> > > is
> >> > > > >> > >> > > > > the latter case that we want to guard?
> >> > > > >> > >> > > > >
> >> > > > >> > >> > > > > I'm not sure I follow that "the message will fail
> >> the
> >> > > > >> sequence
> >> > > > >> > >> > > > validation".
> >> > > > >> > >> > > > > In some of these cases, we had an abort marker
> (due
> >> to
> >> > an
> >> > > > >> error)
> >> > > > >> > >> and
> >> > > > >> > >> > > then
> >> > > > >> > >> > > > > the late message comes in with the correct
> sequence
> >> > > number.
> >> > > > >> This
> >> > > > >> > >> is a
> >> > > > >> > >> > > > case
> >> > > > >> > >> > > > > covered by the KIP.
> >> > > > >> > >> > > > > The latter case is actually not something we've
> >> > > considered
> >> > > > >> > here. I
> >> > > > >> > >> > > think
> >> > > > >> > >> > > > > generally when we bump the epoch, we are accepting
> >> that
> >> > > the
> >> > > > >> > >> sequence
> >> > > > >> > >> > > does
> >> > > > >> > >> > > > > not need to be checked anymore. My understanding
> is
> >> > also
> >> > > > >> that we
> >> > > > >> > >> > don't
> >> > > > >> > >> > > > > typically bump epoch mid transaction (based on a
> >> quick
> >> > > look
> >> > > > >> at
> >> > > > >> > the
> >> > > > >> > >> > > code)
> >> > > > >> > >> > > > > but let me know if that is the case.
> >> > > > >> > >> > > > >
> >> > > > >> > >> > > > > Thanks,
> >> > > > >> > >> > > > > Justine
> >> > > > >> > >> > > > >
> >> > > > >> > >> > > > > On Thu, Dec 15, 2022 at 12:23 PM Jun Rao
> >> > > > >> > <jun@confluent.io.invalid
> >> > > > >> > >> >
> >> > > > >> > >> > > > wrote:
> >> > > > >> > >> > > > >
> >> > > > >> > >> > > > > > Hi, Justine,
> >> > > > >> > >> > > > > >
> >> > > > >> > >> > > > > > Thanks for the reply.
> >> > > > >> > >> > > > > >
> >> > > > >> > >> > > > > > 70. Assigning a new pid on int overflow seems a
> >> bit
> >> > > > hacky.
> >> > > > >> If
> >> > > > >> > we
> >> > > > >> > >> > > need a
> >> > > > >> > >> > > > > txn
> >> > > > >> > >> > > > > > level id, it will be better to model this
> >> explicitly.
> >> > > > >> Adding a
> >> > > > >> > >> new
> >> > > > >> > >> > > > field
> >> > > > >> > >> > > > > > would require a bit more work since it requires
> a
> >> new
> >> > > txn
> >> > > > >> > marker
> >> > > > >> > >> > > format
> >> > > > >> > >> > > > > in
> >> > > > >> > >> > > > > > the log. So, we probably need to guard it with
> an
> >> IBP
> >> > > or
> >> > > > >> > >> metadata
> >> > > > >> > >> > > > version
> >> > > > >> > >> > > > > > and document the impact on downgrade once the
> new
> >> > > format
> >> > > > is
> >> > > > >> > >> written
> >> > > > >> > >> > > to
> >> > > > >> > >> > > > > the
> >> > > > >> > >> > > > > > log.
> >> > > > >> > >> > > > > >
> >> > > > >> > >> > > > > > 71. Hmm, once the marker is written, the
> partition
> >> > will
> >> > > > >> expect
> >> > > > >> > >> the
> >> > > > >> > >> > > next
> >> > > > >> > >> > > > > > append to be on the next epoch. Does that cover
> >> the
> >> > > case
> >> > > > >> you
> >> > > > >> > >> > > mentioned?
> >> > > > >> > >> > > > > >
> >> > > > >> > >> > > > > > 72. Also, just to be clear on the stucked
> message
> >> > issue
> >> > > > >> > >> described
> >> > > > >> > >> > in
> >> > > > >> > >> > > > the
> >> > > > >> > >> > > > > > motivation. With EoS, we also validate the
> >> sequence
> >> > id
> >> > > > for
> >> > > > >> > >> > > idempotency.
> >> > > > >> > >> > > > > So,
> >> > > > >> > >> > > > > > with the current logic, if the producer epoch
> >> hasn't
> >> > > been
> >> > > > >> > >> bumped on
> >> > > > >> > >> > > the
> >> > > > >> > >> > > > > > broker, it seems that the stucked message will
> >> fail
> >> > the
> >> > > > >> > sequence
> >> > > > >> > >> > > > > validation
> >> > > > >> > >> > > > > > and will be ignored. If the producer epoch has
> >> been
> >> > > > >> bumped, we
> >> > > > >> > >> > ignore
> >> > > > >> > >> > > > the
> >> > > > >> > >> > > > > > sequence check and the stuck message could be
> >> > appended
> >> > > to
> >> > > > >> the
> >> > > > >> > >> log.
> >> > > > >> > >> > > So,
> >> > > > >> > >> > > > is
> >> > > > >> > >> > > > > > the latter case that we want to guard?
> >> > > > >> > >> > > > > >
> >> > > > >> > >> > > > > > Thanks,
> >> > > > >> > >> > > > > >
> >> > > > >> > >> > > > > > Jun
> >> > > > >> > >> > > > > >
> >> > > > >> > >> > > > > > On Wed, Dec 14, 2022 at 10:44 AM Justine Olshan
> >> > > > >> > >> > > > > > <jo...@confluent.io.invalid> wrote:
> >> > > > >> > >> > > > > >
> >> > > > >> > >> > > > > > > Matthias — thanks again for taking time to
> look
> >> a
> >> > > this.
> >> > > > >> You
> >> > > > >> > >> said:
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > > My proposal was only focusing to avoid
> >> dangling
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > transactions if records are added without
> >> > registered
> >> > > > >> > >> partition.
> >> > > > >> > >> > --
> >> > > > >> > >> > > > > Maybe
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > you can add a few more details to the KIP
> about
> >> > this
> >> > > > >> > scenario
> >> > > > >> > >> for
> >> > > > >> > >> > > > > better
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > documentation purpose?
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > I'm not sure I understand what you mean here.
> >> The
> >> > > > >> motivation
> >> > > > >> > >> > > section
> >> > > > >> > >> > > > > > > describes two scenarios about how the record
> >> can be
> >> > > > added
> >> > > > >> > >> > without a
> >> > > > >> > >> > > > > > > registered partition:
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > > 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.
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > > Another way hanging transactions can occur
> is
> >> > that
> >> > > a
> >> > > > >> > client
> >> > > > >> > >> is
> >> > > > >> > >> > > > buggy
> >> > > > >> > >> > > > > > and
> >> > > > >> > >> > > > > > > may somehow try to write to a partition before
> >> it
> >> > > adds
> >> > > > >> the
> >> > > > >> > >> > > partition
> >> > > > >> > >> > > > to
> >> > > > >> > >> > > > > > the
> >> > > > >> > >> > > > > > > transaction.
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > For the first example of this would it be
> >> helpful
> >> > to
> >> > > > say
> >> > > > >> > that
> >> > > > >> > >> > this
> >> > > > >> > >> > > > > > message
> >> > > > >> > >> > > > > > > comes in after the abort, but before the
> >> partition
> >> > is
> >> > > > >> added
> >> > > > >> > to
> >> > > > >> > >> > the
> >> > > > >> > >> > > > next
> >> > > > >> > >> > > > > > > transaction so it becomes "hanging." Perhaps
> the
> >> > next
> >> > > > >> > sentence
> >> > > > >> > >> > > > > describing
> >> > > > >> > >> > > > > > > the message becoming part of the next
> >> transaction
> >> > (a
> >> > > > >> > different
> >> > > > >> > >> > > case)
> >> > > > >> > >> > > > > was
> >> > > > >> > >> > > > > > > not properly differentiated.
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > Jun — thanks for reading the KIP.
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > 70. The int typing was a concern. Currently we
> >> > have a
> >> > > > >> > >> mechanism
> >> > > > >> > >> > in
> >> > > > >> > >> > > > > place
> >> > > > >> > >> > > > > > to
> >> > > > >> > >> > > > > > > fence the final epoch when the epoch is about
> to
> >> > > > overflow
> >> > > > >> > and
> >> > > > >> > >> > > assign
> >> > > > >> > >> > > > a
> >> > > > >> > >> > > > > > new
> >> > > > >> > >> > > > > > > producer ID with epoch 0. Of course, this is a
> >> bit
> >> > > > tricky
> >> > > > >> > >> when it
> >> > > > >> > >> > > > comes
> >> > > > >> > >> > > > > > to
> >> > > > >> > >> > > > > > > the response back to the client.
> >> > > > >> > >> > > > > > > Making this a long could be another option,
> but
> >> I
> >> > > > wonder
> >> > > > >> are
> >> > > > >> > >> > there
> >> > > > >> > >> > > > any
> >> > > > >> > >> > > > > > > implications on changing this field if the
> >> epoch is
> >> > > > >> > persisted
> >> > > > >> > >> to
> >> > > > >> > >> > > > disk?
> >> > > > >> > >> > > > > > I'd
> >> > > > >> > >> > > > > > > need to check the usages.
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > 71.This was something Matthias asked about as
> >> > well. I
> >> > > > was
> >> > > > >> > >> > > > considering a
> >> > > > >> > >> > > > > > > possible edge case where a produce request
> from
> >> a
> >> > new
> >> > > > >> > >> transaction
> >> > > > >> > >> > > > > somehow
> >> > > > >> > >> > > > > > > gets sent right after the marker is written,
> but
> >> > > before
> >> > > > >> the
> >> > > > >> > >> > > producer
> >> > > > >> > >> > > > is
> >> > > > >> > >> > > > > > > alerted of the newly bumped epoch. In this
> >> case, we
> >> > > may
> >> > > > >> > >> include
> >> > > > >> > >> > > this
> >> > > > >> > >> > > > > > record
> >> > > > >> > >> > > > > > > when we don't want to. I suppose we could try
> >> to do
> >> > > > >> > something
> >> > > > >> > >> > > client
> >> > > > >> > >> > > > > side
> >> > > > >> > >> > > > > > > to bump the epoch after sending an endTxn as
> >> well
> >> > in
> >> > > > this
> >> > > > >> > >> > scenario
> >> > > > >> > >> > > —
> >> > > > >> > >> > > > > but
> >> > > > >> > >> > > > > > I
> >> > > > >> > >> > > > > > > wonder how it would work when the server is
> >> > aborting
> >> > > > >> based
> >> > > > >> > on
> >> > > > >> > >> a
> >> > > > >> > >> > > > > > server-side
> >> > > > >> > >> > > > > > > error. I could also be missing something and
> >> this
> >> > > > >> scenario
> >> > > > >> > is
> >> > > > >> > >> > > > actually
> >> > > > >> > >> > > > > > not
> >> > > > >> > >> > > > > > > possible.
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > Thanks again to everyone reading and
> commenting.
> >> > Let
> >> > > me
> >> > > > >> know
> >> > > > >> > >> > about
> >> > > > >> > >> > > > any
> >> > > > >> > >> > > > > > > further questions or comments.
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > Justine
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > On Wed, Dec 14, 2022 at 9:41 AM Jun Rao
> >> > > > >> > >> <jun@confluent.io.invalid
> >> > > > >> > >> > >
> >> > > > >> > >> > > > > > wrote:
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > > Hi, Justine,
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > > > Thanks for the KIP. A couple of comments.
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > > > 70. Currently, the producer epoch is an int.
> >> I am
> >> > > not
> >> > > > >> sure
> >> > > > >> > >> if
> >> > > > >> > >> > > it's
> >> > > > >> > >> > > > > > enough
> >> > > > >> > >> > > > > > > > to accommodate all transactions in the
> >> lifetime
> >> > of
> >> > > a
> >> > > > >> > >> producer.
> >> > > > >> > >> > > > Should
> >> > > > >> > >> > > > > > we
> >> > > > >> > >> > > > > > > > change that to a long or add a new long
> field
> >> > like
> >> > > > >> txnId?
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > > > 71. "it will write the prepare commit
> message
> >> > with
> >> > > a
> >> > > > >> > bumped
> >> > > > >> > >> > epoch
> >> > > > >> > >> > > > and
> >> > > > >> > >> > > > > > > send
> >> > > > >> > >> > > > > > > > WriteTxnMarkerRequests with the bumped
> epoch."
> >> > Hmm,
> >> > > > the
> >> > > > >> > >> epoch
> >> > > > >> > >> > is
> >> > > > >> > >> > > > > > > associated
> >> > > > >> > >> > > > > > > > with the current txn right? So, it seems
> >> weird to
> >> > > > >> write a
> >> > > > >> > >> > commit
> >> > > > >> > >> > > > > > message
> >> > > > >> > >> > > > > > > > with a bumped epoch. Should we only bump up
> >> the
> >> > > epoch
> >> > > > >> in
> >> > > > >> > >> > > > > EndTxnResponse
> >> > > > >> > >> > > > > > > and
> >> > > > >> > >> > > > > > > > rename the field to sth like
> >> nextProducerEpoch?
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > > > Thanks,
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > > > Jun
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > > > On Mon, Dec 12, 2022 at 8:54 PM Matthias J.
> >> Sax <
> >> > > > >> > >> > > mjsax@apache.org>
> >> > > > >> > >> > > > > > > wrote:
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > > > > Thanks for the background.
> >> > > > >> > >> > > > > > > > >
> >> > > > >> > >> > > > > > > > > 20/30: SGTM. My proposal was only focusing
> >> to
> >> > > avoid
> >> > > > >> > >> dangling
> >> > > > >> > >> > > > > > > > > transactions if records are added without
> >> > > > registered
> >> > > > >> > >> > partition.
> >> > > > >> > >> > > > --
> >> > > > >> > >> > > > > > > Maybe
> >> > > > >> > >> > > > > > > > > you can add a few more details to the KIP
> >> about
> >> > > > this
> >> > > > >> > >> scenario
> >> > > > >> > >> > > for
> >> > > > >> > >> > > > > > > better
> >> > > > >> > >> > > > > > > > > documentation purpose?
> >> > > > >> > >> > > > > > > > >
> >> > > > >> > >> > > > > > > > > 40: I think you hit a fair point about
> race
> >> > > > >> conditions
> >> > > > >> > or
> >> > > > >> > >> > > client
> >> > > > >> > >> > > > > bugs
> >> > > > >> > >> > > > > > > > > (incorrectly not bumping the epoch). The
> >> > > > >> > >> complexity/confusion
> >> > > > >> > >> > > for
> >> > > > >> > >> > > > > > using
> >> > > > >> > >> > > > > > > > > the bumped epoch I see, is mainly for
> >> internal
> >> > > > >> > debugging,
> >> > > > >> > >> ie,
> >> > > > >> > >> > > > > > > inspecting
> >> > > > >> > >> > > > > > > > > log segment dumps -- it seems harder to
> >> reason
> >> > > > about
> >> > > > >> the
> >> > > > >> > >> > system
> >> > > > >> > >> > > > for
> >> > > > >> > >> > > > > > us
> >> > > > >> > >> > > > > > > > > humans. But if we get better guarantees,
> it
> >> > would
> >> > > > be
> >> > > > >> > >> worth to
> >> > > > >> > >> > > use
> >> > > > >> > >> > > > > the
> >> > > > >> > >> > > > > > > > > bumped epoch.
> >> > > > >> > >> > > > > > > > >
> >> > > > >> > >> > > > > > > > > 60: as I mentioned already, I don't know
> the
> >> > > broker
> >> > > > >> > >> internals
> >> > > > >> > >> > > to
> >> > > > >> > >> > > > > > > provide
> >> > > > >> > >> > > > > > > > > more input. So if nobody else chimes in,
> we
> >> > > should
> >> > > > >> just
> >> > > > >> > >> move
> >> > > > >> > >> > > > > forward
> >> > > > >> > >> > > > > > > > > with your proposal.
> >> > > > >> > >> > > > > > > > >
> >> > > > >> > >> > > > > > > > >
> >> > > > >> > >> > > > > > > > > -Matthias
> >> > > > >> > >> > > > > > > > >
> >> > > > >> > >> > > > > > > > >
> >> > > > >> > >> > > > > > > > > On 12/6/22 4:22 PM, Justine Olshan wrote:
> >> > > > >> > >> > > > > > > > > > Hi all,
> >> > > > >> > >> > > > > > > > > > After Artem's questions about error
> >> behavior,
> >> > > > I've
> >> > > > >> > >> > > re-evaluated
> >> > > > >> > >> > > > > the
> >> > > > >> > >> > > > > > > > > > unknown producer ID exception and had
> some
> >> > > > >> discussions
> >> > > > >> > >> > > offline.
> >> > > > >> > >> > > > > > > > > >
> >> > > > >> > >> > > > > > > > > > I think generally it makes sense to
> >> simplify
> >> > > > error
> >> > > > >> > >> handling
> >> > > > >> > >> > > in
> >> > > > >> > >> > > > > > cases
> >> > > > >> > >> > > > > > > > like
> >> > > > >> > >> > > > > > > > > > this and the UNKNOWN_PRODUCER_ID error
> >> has a
> >> > > > pretty
> >> > > > >> > long
> >> > > > >> > >> > and
> >> > > > >> > >> > > > > > > > complicated
> >> > > > >> > >> > > > > > > > > > history. Because of this, I propose
> >> adding a
> >> > > new
> >> > > > >> error
> >> > > > >> > >> code
> >> > > > >> > >> > > > > > > > > ABORTABLE_ERROR
> >> > > > >> > >> > > > > > > > > > that when encountered by new clients
> >> (gated
> >> > by
> >> > > > the
> >> > > > >> > >> produce
> >> > > > >> > >> > > > > request
> >> > > > >> > >> > > > > > > > > version)
> >> > > > >> > >> > > > > > > > > > will simply abort the transaction. This
> >> > allows
> >> > > > the
> >> > > > >> > >> server
> >> > > > >> > >> > to
> >> > > > >> > >> > > > have
> >> > > > >> > >> > > > > > > some
> >> > > > >> > >> > > > > > > > > say
> >> > > > >> > >> > > > > > > > > > in whether the client aborts and makes
> >> > handling
> >> > > > >> much
> >> > > > >> > >> > simpler.
> >> > > > >> > >> > > > In
> >> > > > >> > >> > > > > > the
> >> > > > >> > >> > > > > > > > > > future, we can also use this error in
> >> other
> >> > > > >> situations
> >> > > > >> > >> > where
> >> > > > >> > >> > > we
> >> > > > >> > >> > > > > > want
> >> > > > >> > >> > > > > > > to
> >> > > > >> > >> > > > > > > > > > abort the transactions. We can even use
> on
> >> > > other
> >> > > > >> apis.
> >> > > > >> > >> > > > > > > > > >
> >> > > > >> > >> > > > > > > > > > I've added this to the KIP. Let me know
> if
> >> > > there
> >> > > > >> are
> >> > > > >> > any
> >> > > > >> > >> > > > > questions
> >> > > > >> > >> > > > > > or
> >> > > > >> > >> > > > > > > > > > issues.
> >> > > > >> > >> > > > > > > > > >
> >> > > > >> > >> > > > > > > > > > Justine
> >> > > > >> > >> > > > > > > > > >
> >> > > > >> > >> > > > > > > > > > On Fri, Dec 2, 2022 at 10:22 AM Justine
> >> > Olshan
> >> > > <
> >> > > > >> > >> > > > > > jolshan@confluent.io
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > > > > wrote:
> >> > > > >> > >> > > > > > > > > >
> >> > > > >> > >> > > > > > > > > >> Hey Matthias,
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >> 20/30 — Maybe I also didn't express
> >> myself
> >> > > > >> clearly.
> >> > > > >> > For
> >> > > > >> > >> > > older
> >> > > > >> > >> > > > > > > clients
> >> > > > >> > >> > > > > > > > we
> >> > > > >> > >> > > > > > > > > >> don't have a way to distinguish
> between a
> >> > > > previous
> >> > > > >> > and
> >> > > > >> > >> the
> >> > > > >> > >> > > > > current
> >> > > > >> > >> > > > > > > > > >> transaction since we don't have the
> epoch
> >> > > bump.
> >> > > > >> This
> >> > > > >> > >> means
> >> > > > >> > >> > > > that
> >> > > > >> > >> > > > > a
> >> > > > >> > >> > > > > > > late
> >> > > > >> > >> > > > > > > > > >> message from the previous transaction
> >> may be
> >> > > > >> added to
> >> > > > >> > >> the
> >> > > > >> > >> > > new
> >> > > > >> > >> > > > > one.
> >> > > > >> > >> > > > > > > > With
> >> > > > >> > >> > > > > > > > > >> older clients — we can't guarantee this
> >> > won't
> >> > > > >> happen
> >> > > > >> > >> if we
> >> > > > >> > >> > > > > already
> >> > > > >> > >> > > > > > > > sent
> >> > > > >> > >> > > > > > > > > the
> >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call (why we make
> >> changes
> >> > > for
> >> > > > >> the
> >> > > > >> > >> newer
> >> > > > >> > >> > > > > client)
> >> > > > >> > >> > > > > > > but
> >> > > > >> > >> > > > > > > > > we
> >> > > > >> > >> > > > > > > > > >> can at least gate some by ensuring that
> >> the
> >> > > > >> partition
> >> > > > >> > >> has
> >> > > > >> > >> > > been
> >> > > > >> > >> > > > > > added
> >> > > > >> > >> > > > > > > > to
> >> > > > >> > >> > > > > > > > > the
> >> > > > >> > >> > > > > > > > > >> transaction. The rationale here is that
> >> > there
> >> > > > are
> >> > > > >> > >> likely
> >> > > > >> > >> > > LESS
> >> > > > >> > >> > > > > late
> >> > > > >> > >> > > > > > > > > arrivals
> >> > > > >> > >> > > > > > > > > >> as time goes on, so hopefully most late
> >> > > arrivals
> >> > > > >> will
> >> > > > >> > >> come
> >> > > > >> > >> > > in
> >> > > > >> > >> > > > > > BEFORE
> >> > > > >> > >> > > > > > > > the
> >> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call. Those that
> >> arrive
> >> > > > before
> >> > > > >> > will
> >> > > > >> > >> be
> >> > > > >> > >> > > > > properly
> >> > > > >> > >> > > > > > > > gated
> >> > > > >> > >> > > > > > > > > >> with the describeTransactions approach.
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >> If we take the approach you suggested,
> >> ANY
> >> > > late
> >> > > > >> > arrival
> >> > > > >> > >> > > from a
> >> > > > >> > >> > > > > > > > previous
> >> > > > >> > >> > > > > > > > > >> transaction will be added. And we don't
> >> want
> >> > > > >> that. I
> >> > > > >> > >> also
> >> > > > >> > >> > > > don't
> >> > > > >> > >> > > > > > see
> >> > > > >> > >> > > > > > > > any
> >> > > > >> > >> > > > > > > > > >> benefit in sending addPartitionsToTxn
> >> over
> >> > the
> >> > > > >> > >> > describeTxns
> >> > > > >> > >> > > > > call.
> >> > > > >> > >> > > > > > > They
> >> > > > >> > >> > > > > > > > > will
> >> > > > >> > >> > > > > > > > > >> both be one extra RPC to the Txn
> >> > coordinator.
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >> To be clear — newer clients will use
> >> > > > >> > addPartitionsToTxn
> >> > > > >> > >> > > > instead
> >> > > > >> > >> > > > > of
> >> > > > >> > >> > > > > > > the
> >> > > > >> > >> > > > > > > > > >> DescribeTxns.
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >> 40)
> >> > > > >> > >> > > > > > > > > >> My concern is that if we have some
> delay
> >> in
> >> > > the
> >> > > > >> > client
> >> > > > >> > >> to
> >> > > > >> > >> > > bump
> >> > > > >> > >> > > > > the
> >> > > > >> > >> > > > > > > > > epoch,
> >> > > > >> > >> > > > > > > > > >> it could continue to send epoch 73 and
> >> those
> >> > > > >> records
> >> > > > >> > >> would
> >> > > > >> > >> > > not
> >> > > > >> > >> > > > > be
> >> > > > >> > >> > > > > > > > > fenced.
> >> > > > >> > >> > > > > > > > > >> Perhaps this is not an issue if we
> don't
> >> > allow
> >> > > > the
> >> > > > >> > next
> >> > > > >> > >> > > > produce
> >> > > > >> > >> > > > > to
> >> > > > >> > >> > > > > > > go
> >> > > > >> > >> > > > > > > > > >> through before the EndTxn request
> >> returns.
> >> > I'm
> >> > > > >> also
> >> > > > >> > >> > thinking
> >> > > > >> > >> > > > > about
> >> > > > >> > >> > > > > > > > > cases of
> >> > > > >> > >> > > > > > > > > >> failure. I will need to think on this a
> >> bit.
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >> I wasn't sure if it was that confusing.
> >> But
> >> > if
> >> > > > we
> >> > > > >> > >> think it
> >> > > > >> > >> > > is,
> >> > > > >> > >> > > > > we
> >> > > > >> > >> > > > > > > can
> >> > > > >> > >> > > > > > > > > >> investigate other ways.
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >> 60)
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >> I'm not sure these are the same
> >> purgatories
> >> > > > since
> >> > > > >> one
> >> > > > >> > >> is a
> >> > > > >> > >> > > > > produce
> >> > > > >> > >> > > > > > > > > >> purgatory (I was planning on using a
> >> > callback
> >> > > > >> rather
> >> > > > >> > >> than
> >> > > > >> > >> > > > > > purgatory)
> >> > > > >> > >> > > > > > > > and
> >> > > > >> > >> > > > > > > > > >> the other is simply a request to append
> >> to
> >> > the
> >> > > > >> log.
> >> > > > >> > Not
> >> > > > >> > >> > sure
> >> > > > >> > >> > > > we
> >> > > > >> > >> > > > > > have
> >> > > > >> > >> > > > > > > > any
> >> > > > >> > >> > > > > > > > > >> structure here for ordering, but my
> >> > > > understanding
> >> > > > >> is
> >> > > > >> > >> that
> >> > > > >> > >> > > the
> >> > > > >> > >> > > > > > broker
> >> > > > >> > >> > > > > > > > > could
> >> > > > >> > >> > > > > > > > > >> handle the write request before it
> hears
> >> > back
> >> > > > from
> >> > > > >> > the
> >> > > > >> > >> Txn
> >> > > > >> > >> > > > > > > > Coordinator.
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >> Let me know if I misunderstood
> something
> >> or
> >> > > > >> something
> >> > > > >> > >> was
> >> > > > >> > >> > > > > unclear.
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >> Justine
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >> On Thu, Dec 1, 2022 at 12:15 PM
> Matthias
> >> J.
> >> > > Sax
> >> > > > <
> >> > > > >> > >> > > > > mjsax@apache.org
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > > > > > wrote:
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >>> Thanks for the details Justine!
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>> 20)
> >> > > > >> > >> > > > > > > > > >>>>
> >> > > > >> > >> > > > > > > > > >>>> The client side change for 2 is
> >> removing
> >> > the
> >> > > > >> > >> > addPartitions
> >> > > > >> > >> > > > to
> >> > > > >> > >> > > > > > > > > >>> transaction
> >> > > > >> > >> > > > > > > > > >>>> call. We don't need to make this from
> >> the
> >> > > > >> producer
> >> > > > >> > to
> >> > > > >> > >> > the
> >> > > > >> > >> > > > txn
> >> > > > >> > >> > > > > > > > > >>> coordinator,
> >> > > > >> > >> > > > > > > > > >>>> only server side.
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>> I think I did not express myself
> >> clearly. I
> >> > > > >> > understand
> >> > > > >> > >> > that
> >> > > > >> > >> > > > we
> >> > > > >> > >> > > > > > can
> >> > > > >> > >> > > > > > > > (and
> >> > > > >> > >> > > > > > > > > >>> should) change the producer to not
> send
> >> the
> >> > > > >> > >> > `addPartitions`
> >> > > > >> > >> > > > > > request
> >> > > > >> > >> > > > > > > > any
> >> > > > >> > >> > > > > > > > > >>> longer. But I don't thinks it's
> >> requirement
> >> > > to
> >> > > > >> > change
> >> > > > >> > >> the
> >> > > > >> > >> > > > > broker?
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>> What I am trying to say is: as a
> >> safe-guard
> >> > > and
> >> > > > >> > >> > improvement
> >> > > > >> > >> > > > for
> >> > > > >> > >> > > > > > > older
> >> > > > >> > >> > > > > > > > > >>> producers, the partition leader can
> just
> >> > send
> >> > > > the
> >> > > > >> > >> > > > > `addPartitions`
> >> > > > >> > >> > > > > > > > > >>> request to the TX-coordinator in any
> >> case
> >> > --
> >> > > if
> >> > > > >> the
> >> > > > >> > >> old
> >> > > > >> > >> > > > > producer
> >> > > > >> > >> > > > > > > > > >>> correctly did send the `addPartition`
> >> > request
> >> > > > to
> >> > > > >> the
> >> > > > >> > >> > > > > > TX-coordinator
> >> > > > >> > >> > > > > > > > > >>> already, the TX-coordinator can just
> >> > "ignore"
> >> > > > is
> >> > > > >> as
> >> > > > >> > >> > > > idempotent.
> >> > > > >> > >> > > > > > > > > However,
> >> > > > >> > >> > > > > > > > > >>> if the old producer has a bug and did
> >> > forget
> >> > > to
> >> > > > >> sent
> >> > > > >> > >> the
> >> > > > >> > >> > > > > > > > `addPartition`
> >> > > > >> > >> > > > > > > > > >>> request, we would now ensure that the
> >> > > partition
> >> > > > >> is
> >> > > > >> > >> indeed
> >> > > > >> > >> > > > added
> >> > > > >> > >> > > > > > to
> >> > > > >> > >> > > > > > > > the
> >> > > > >> > >> > > > > > > > > >>> TX and thus fix a potential producer
> bug
> >> > > (even
> >> > > > >> if we
> >> > > > >> > >> > don't
> >> > > > >> > >> > > > get
> >> > > > >> > >> > > > > > the
> >> > > > >> > >> > > > > > > > > >>> fencing via the bump epoch). -- It
> >> seems to
> >> > > be
> >> > > > a
> >> > > > >> > good
> >> > > > >> > >> > > > > > improvement?
> >> > > > >> > >> > > > > > > Or
> >> > > > >> > >> > > > > > > > > is
> >> > > > >> > >> > > > > > > > > >>> there a reason to not do this?
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>> 30)
> >> > > > >> > >> > > > > > > > > >>>>
> >> > > > >> > >> > > > > > > > > >>>> Transaction is ongoing = partition
> was
> >> > added
> >> > > > to
> >> > > > >> > >> > > transaction
> >> > > > >> > >> > > > > via
> >> > > > >> > >> > > > > > > > > >>>> addPartitionsToTxn. We check this
> with
> >> the
> >> > > > >> > >> > > > > DescribeTransactions
> >> > > > >> > >> > > > > > > > call.
> >> > > > >> > >> > > > > > > > > >>> Let
> >> > > > >> > >> > > > > > > > > >>>> me know if this wasn't sufficiently
> >> > > explained
> >> > > > >> here:
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>> If we do what I propose in (20), we
> >> don't
> >> > > > really
> >> > > > >> > need
> >> > > > >> > >> to
> >> > > > >> > >> > > make
> >> > > > >> > >> > > > > > this
> >> > > > >> > >> > > > > > > > > >>> `DescribeTransaction` call, as the
> >> > partition
> >> > > > >> leader
> >> > > > >> > >> adds
> >> > > > >> > >> > > the
> >> > > > >> > >> > > > > > > > partition
> >> > > > >> > >> > > > > > > > > >>> for older clients and we get this
> check
> >> for
> >> > > > free.
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>> 40)
> >> > > > >> > >> > > > > > > > > >>>>
> >> > > > >> > >> > > > > > > > > >>>> The idea here is that if any messages
> >> > > somehow
> >> > > > >> come
> >> > > > >> > in
> >> > > > >> > >> > > before
> >> > > > >> > >> > > > > we
> >> > > > >> > >> > > > > > > get
> >> > > > >> > >> > > > > > > > > the
> >> > > > >> > >> > > > > > > > > >>> new
> >> > > > >> > >> > > > > > > > > >>>> epoch to the producer, they will be
> >> > fenced.
> >> > > > >> > However,
> >> > > > >> > >> if
> >> > > > >> > >> > we
> >> > > > >> > >> > > > > don't
> >> > > > >> > >> > > > > > > > think
> >> > > > >> > >> > > > > > > > > >>> this
> >> > > > >> > >> > > > > > > > > >>>> is necessary, it can be discussed
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>> I agree that we should have epoch
> >> fencing.
> >> > My
> >> > > > >> > >> question is
> >> > > > >> > >> > > > > > > different:
> >> > > > >> > >> > > > > > > > > >>> Assume we are at epoch 73, and we have
> >> an
> >> > > > ongoing
> >> > > > >> > >> > > > transaction,
> >> > > > >> > >> > > > > > that
> >> > > > >> > >> > > > > > > > is
> >> > > > >> > >> > > > > > > > > >>> committed. It seems natural to write
> the
> >> > > > "prepare
> >> > > > >> > >> commit"
> >> > > > >> > >> > > > > marker
> >> > > > >> > >> > > > > > > and
> >> > > > >> > >> > > > > > > > > the
> >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` both with epoch
> >> 73,
> >> > > too,
> >> > > > >> as
> >> > > > >> > it
> >> > > > >> > >> > > belongs
> >> > > > >> > >> > > > > to
> >> > > > >> > >> > > > > > > the
> >> > > > >> > >> > > > > > > > > >>> current transaction. Of course, we now
> >> also
> >> > > > bump
> >> > > > >> the
> >> > > > >> > >> > epoch
> >> > > > >> > >> > > > and
> >> > > > >> > >> > > > > > > expect
> >> > > > >> > >> > > > > > > > > >>> the next requests to have epoch 74,
> and
> >> > would
> >> > > > >> reject
> >> > > > >> > >> an
> >> > > > >> > >> > > > request
> >> > > > >> > >> > > > > > > with
> >> > > > >> > >> > > > > > > > > >>> epoch 73, as the corresponding TX for
> >> epoch
> >> > > 73
> >> > > > >> was
> >> > > > >> > >> > already
> >> > > > >> > >> > > > > > > committed.
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>> It seems you propose to write the
> >> "prepare
> >> > > > commit
> >> > > > >> > >> marker"
> >> > > > >> > >> > > and
> >> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` with epoch 74
> >> > though,
> >> > > > what
> >> > > > >> > >> would
> >> > > > >> > >> > > work,
> >> > > > >> > >> > > > > but
> >> > > > >> > >> > > > > > > it
> >> > > > >> > >> > > > > > > > > >>> seems confusing. Is there a reason why
> >> we
> >> > > would
> >> > > > >> use
> >> > > > >> > >> the
> >> > > > >> > >> > > > bumped
> >> > > > >> > >> > > > > > > epoch
> >> > > > >> > >> > > > > > > > 74
> >> > > > >> > >> > > > > > > > > >>> instead of the current epoch 73?
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>> 60)
> >> > > > >> > >> > > > > > > > > >>>>
> >> > > > >> > >> > > > > > > > > >>>> When we are checking if the
> >> transaction is
> >> > > > >> ongoing,
> >> > > > >> > >> we
> >> > > > >> > >> > > need
> >> > > > >> > >> > > > to
> >> > > > >> > >> > > > > > > make
> >> > > > >> > >> > > > > > > > a
> >> > > > >> > >> > > > > > > > > >>> round
> >> > > > >> > >> > > > > > > > > >>>> trip from the leader partition to the
> >> > > > >> transaction
> >> > > > >> > >> > > > coordinator.
> >> > > > >> > >> > > > > > In
> >> > > > >> > >> > > > > > > > the
> >> > > > >> > >> > > > > > > > > >>> time
> >> > > > >> > >> > > > > > > > > >>>> we are waiting for this message to
> come
> >> > > back,
> >> > > > in
> >> > > > >> > >> theory
> >> > > > >> > >> > we
> >> > > > >> > >> > > > > could
> >> > > > >> > >> > > > > > > > have
> >> > > > >> > >> > > > > > > > > >>> sent
> >> > > > >> > >> > > > > > > > > >>>> a commit/abort call that would make
> the
> >> > > > original
> >> > > > >> > >> result
> >> > > > >> > >> > of
> >> > > > >> > >> > > > the
> >> > > > >> > >> > > > > > > check
> >> > > > >> > >> > > > > > > > > >>> out of
> >> > > > >> > >> > > > > > > > > >>>> date. That is why we can check the
> >> leader
> >> > > > state
> >> > > > >> > >> before
> >> > > > >> > >> > we
> >> > > > >> > >> > > > > write
> >> > > > >> > >> > > > > > to
> >> > > > >> > >> > > > > > > > the
> >> > > > >> > >> > > > > > > > > >>> log.
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>> Thanks. Got it.
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>> However, is this really an issue? We
> put
> >> > the
> >> > > > >> produce
> >> > > > >> > >> > > request
> >> > > > >> > >> > > > in
> >> > > > >> > >> > > > > > > > > >>> purgatory, so how could we process the
> >> > > > >> > >> > > > `WriteTxnMarkerRequest`
> >> > > > >> > >> > > > > > > first?
> >> > > > >> > >> > > > > > > > > >>> Don't we need to put the
> >> > > > `WriteTxnMarkerRequest`
> >> > > > >> > into
> >> > > > >> > >> > > > > purgatory,
> >> > > > >> > >> > > > > > > too,
> >> > > > >> > >> > > > > > > > > >>> for this case, and process both
> request
> >> > > > in-order?
> >> > > > >> > >> (Again,
> >> > > > >> > >> > > my
> >> > > > >> > >> > > > > > broker
> >> > > > >> > >> > > > > > > > > >>> knowledge is limited and maybe we
> don't
> >> > > > maintain
> >> > > > >> > >> request
> >> > > > >> > >> > > > order
> >> > > > >> > >> > > > > > for
> >> > > > >> > >> > > > > > > > this
> >> > > > >> > >> > > > > > > > > >>> case, what seems to be an issue IMHO,
> >> and I
> >> > > am
> >> > > > >> > >> wondering
> >> > > > >> > >> > if
> >> > > > >> > >> > > > > > > changing
> >> > > > >> > >> > > > > > > > > >>> request handling to preserve order for
> >> this
> >> > > > case
> >> > > > >> > >> might be
> >> > > > >> > >> > > the
> >> > > > >> > >> > > > > > > cleaner
> >> > > > >> > >> > > > > > > > > >>> solution?)
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>> -Matthias
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>> On 11/30/22 3:28 PM, Artem Livshits
> >> wrote:
> >> > > > >> > >> > > > > > > > > >>>> Hi Justine,
> >> > > > >> > >> > > > > > > > > >>>>
> >> > > > >> > >> > > > > > > > > >>>> I think the interesting part is not
> in
> >> > this
> >> > > > >> logic
> >> > > > >> > >> > (because
> >> > > > >> > >> > > > it
> >> > > > >> > >> > > > > > > tries
> >> > > > >> > >> > > > > > > > to
> >> > > > >> > >> > > > > > > > > >>>> figure out when UNKNOWN_PRODUCER_ID
> is
> >> > > > retriable
> >> > > > >> > and
> >> > > > >> > >> if
> >> > > > >> > >> > > it's
> >> > > > >> > >> > > > > > > > > retryable,
> >> > > > >> > >> > > > > > > > > >>>> it's definitely not fatal), but what
> >> > happens
> >> > > > >> when
> >> > > > >> > >> this
> >> > > > >> > >> > > logic
> >> > > > >> > >> > > > > > > doesn't
> >> > > > >> > >> > > > > > > > > >>> return
> >> > > > >> > >> > > > > > > > > >>>> 'true' and falls through.  In the old
> >> > > clients
> >> > > > it
> >> > > > >> > >> seems
> >> > > > >> > >> > to
> >> > > > >> > >> > > be
> >> > > > >> > >> > > > > > > fatal,
> >> > > > >> > >> > > > > > > > if
> >> > > > >> > >> > > > > > > > > >>> we
> >> > > > >> > >> > > > > > > > > >>>> keep the behavior in the new clients,
> >> I'd
> >> > > > >> expect it
> >> > > > >> > >> > would
> >> > > > >> > >> > > be
> >> > > > >> > >> > > > > > fatal
> >> > > > >> > >> > > > > > > > as
> >> > > > >> > >> > > > > > > > > >>> well.
> >> > > > >> > >> > > > > > > > > >>>>
> >> > > > >> > >> > > > > > > > > >>>> -Artem
> >> > > > >> > >> > > > > > > > > >>>>
> >> > > > >> > >> > > > > > > > > >>>> On Tue, Nov 29, 2022 at 11:57 AM
> >> Justine
> >> > > > Olshan
> >> > > > >> > >> > > > > > > > > >>>> <jo...@confluent.io.invalid>
> wrote:
> >> > > > >> > >> > > > > > > > > >>>>
> >> > > > >> > >> > > > > > > > > >>>>> Hi Artem and Jeff,
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> Thanks for taking a look and sorry
> for
> >> > the
> >> > > > slow
> >> > > > >> > >> > response.
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> You both mentioned the change to
> >> handle
> >> > > > >> > >> > > UNKNOWN_PRODUCER_ID
> >> > > > >> > >> > > > > > > errors.
> >> > > > >> > >> > > > > > > > > To
> >> > > > >> > >> > > > > > > > > >>> be
> >> > > > >> > >> > > > > > > > > >>>>> clear — this error code will only be
> >> sent
> >> > > > again
> >> > > > >> > when
> >> > > > >> > >> > the
> >> > > > >> > >> > > > > > client's
> >> > > > >> > >> > > > > > > > > >>> request
> >> > > > >> > >> > > > > > > > > >>>>> version is high enough to ensure we
> >> > handle
> >> > > it
> >> > > > >> > >> > correctly.
> >> > > > >> > >> > > > > > > > > >>>>> The current (Java) client handles
> >> this by
> >> > > the
> >> > > > >> > >> following
> >> > > > >> > >> > > > > > (somewhat
> >> > > > >> > >> > > > > > > > > long)
> >> > > > >> > >> > > > > > > > > >>>>> code snippet:
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> // An UNKNOWN_PRODUCER_ID means that
> >> we
> >> > > have
> >> > > > >> lost
> >> > > > >> > >> the
> >> > > > >> > >> > > > > producer
> >> > > > >> > >> > > > > > > > state
> >> > > > >> > >> > > > > > > > > >>> on the
> >> > > > >> > >> > > > > > > > > >>>>> broker. Depending on the log start
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> // offset, we may want to retry
> >> these, as
> >> > > > >> > described
> >> > > > >> > >> for
> >> > > > >> > >> > > > each
> >> > > > >> > >> > > > > > case
> >> > > > >> > >> > > > > > > > > >>> below. If
> >> > > > >> > >> > > > > > > > > >>>>> none of those apply, then for the
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> // idempotent producer, we will
> >> locally
> >> > > bump
> >> > > > >> the
> >> > > > >> > >> epoch
> >> > > > >> > >> > > and
> >> > > > >> > >> > > > > > reset
> >> > > > >> > >> > > > > > > > the
> >> > > > >> > >> > > > > > > > > >>>>> sequence numbers of in-flight
> batches
> >> > from
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> // sequence 0, then retry the failed
> >> > batch,
> >> > > > >> which
> >> > > > >> > >> > should
> >> > > > >> > >> > > > now
> >> > > > >> > >> > > > > > > > succeed.
> >> > > > >> > >> > > > > > > > > >>> For
> >> > > > >> > >> > > > > > > > > >>>>> the transactional producer, allow
> the
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> // batch to fail. When processing
> the
> >> > > failed
> >> > > > >> > batch,
> >> > > > >> > >> we
> >> > > > >> > >> > > will
> >> > > > >> > >> > > > > > > > > transition
> >> > > > >> > >> > > > > > > > > >>> to
> >> > > > >> > >> > > > > > > > > >>>>> an abortable error and set a flag
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> // indicating that we need to bump
> the
> >> > > epoch
> >> > > > >> (if
> >> > > > >> > >> > > supported
> >> > > > >> > >> > > > by
> >> > > > >> > >> > > > > > the
> >> > > > >> > >> > > > > > > > > >>> broker).
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> if (error ==
> >> > Errors.*UNKNOWN_PRODUCER_ID*)
> >> > > {
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>       if (response.logStartOffset ==
> >> -1)
> >> > {
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // We don't know the log
> >> start
> >> > > > offset
> >> > > > >> > with
> >> > > > >> > >> > this
> >> > > > >> > >> > > > > > > response.
> >> > > > >> > >> > > > > > > > > We
> >> > > > >> > >> > > > > > > > > >>> should
> >> > > > >> > >> > > > > > > > > >>>>> just retry the request until we get
> >> it.
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // The UNKNOWN_PRODUCER_ID
> >> > error
> >> > > > code
> >> > > > >> > was
> >> > > > >> > >> > added
> >> > > > >> > >> > > > > along
> >> > > > >> > >> > > > > > > > with
> >> > > > >> > >> > > > > > > > > >>> the new
> >> > > > >> > >> > > > > > > > > >>>>> ProduceResponse which includes the
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // logStartOffset. So the
> >> '-1'
> >> > > > >> sentinel
> >> > > > >> > is
> >> > > > >> > >> > not
> >> > > > >> > >> > > > for
> >> > > > >> > >> > > > > > > > backward
> >> > > > >> > >> > > > > > > > > >>>>> compatibility. Instead, it is
> possible
> >> > for
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // a broker to not know
> the
> >> > > > >> > >> logStartOffset at
> >> > > > >> > >> > > > when
> >> > > > >> > >> > > > > it
> >> > > > >> > >> > > > > > > is
> >> > > > >> > >> > > > > > > > > >>> returning
> >> > > > >> > >> > > > > > > > > >>>>> the response because the partition
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // may have moved away
> from
> >> the
> >> > > > >> broker
> >> > > > >> > >> from
> >> > > > >> > >> > the
> >> > > > >> > >> > > > > time
> >> > > > >> > >> > > > > > > the
> >> > > > >> > >> > > > > > > > > >>> error was
> >> > > > >> > >> > > > > > > > > >>>>> initially raised to the time the
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // response was being
> >> > > constructed.
> >> > > > In
> >> > > > >> > >> these
> >> > > > >> > >> > > > cases,
> >> > > > >> > >> > > > > we
> >> > > > >> > >> > > > > > > > > should
> >> > > > >> > >> > > > > > > > > >>> just
> >> > > > >> > >> > > > > > > > > >>>>> retry the request: we are guaranteed
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // to eventually get a
> >> > > > logStartOffset
> >> > > > >> > once
> >> > > > >> > >> > > things
> >> > > > >> > >> > > > > > > settle
> >> > > > >> > >> > > > > > > > > down.
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           return true;
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>       }
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>       if
> >> (batch.sequenceHasBeenReset()) {
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // When the first inflight
> >> > batch
> >> > > > >> fails
> >> > > > >> > >> due to
> >> > > > >> > >> > > the
> >> > > > >> > >> > > > > > > > > truncation
> >> > > > >> > >> > > > > > > > > >>> case,
> >> > > > >> > >> > > > > > > > > >>>>> then the sequences of all the other
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // in flight batches would
> >> have
> >> > > > been
> >> > > > >> > >> > restarted
> >> > > > >> > >> > > > from
> >> > > > >> > >> > > > > > the
> >> > > > >> > >> > > > > > > > > >>> beginning.
> >> > > > >> > >> > > > > > > > > >>>>> However, when those responses
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // come back from the
> >> broker,
> >> > > they
> >> > > > >> would
> >> > > > >> > >> also
> >> > > > >> > >> > > > come
> >> > > > >> > >> > > > > > with
> >> > > > >> > >> > > > > > > > an
> >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID error. In this
> >> case,
> >> > we
> >> > > > >> should
> >> > > > >> > >> not
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // reset the sequence
> >> numbers
> >> > to
> >> > > > the
> >> > > > >> > >> > beginning.
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           return true;
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>       } else if
> >> > > > >> > >> > > > > (lastAckedOffset(batch.topicPartition).orElse(
> >> > > > >> > >> > > > > > > > > >>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> >> > > > >> > >> > > > response.logStartOffset) {
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // The head of the log has
> >> been
> >> > > > >> removed,
> >> > > > >> > >> > > probably
> >> > > > >> > >> > > > > due
> >> > > > >> > >> > > > > > > to
> >> > > > >> > >> > > > > > > > > the
> >> > > > >> > >> > > > > > > > > >>>>> retention time elapsing. In this
> case,
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // we expect to lose the
> >> > producer
> >> > > > >> state.
> >> > > > >> > >> For
> >> > > > >> > >> > > the
> >> > > > >> > >> > > > > > > > > transactional
> >> > > > >> > >> > > > > > > > > >>>>> producer, reset the sequences of all
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // inflight batches to be
> >> from
> >> > > the
> >> > > > >> > >> beginning
> >> > > > >> > >> > > and
> >> > > > >> > >> > > > > > retry
> >> > > > >> > >> > > > > > > > > them,
> >> > > > >> > >> > > > > > > > > >>> so
> >> > > > >> > >> > > > > > > > > >>>>> that the transaction does not need
> to
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // be aborted. For the
> >> > idempotent
> >> > > > >> > >> producer,
> >> > > > >> > >> > > bump
> >> > > > >> > >> > > > > the
> >> > > > >> > >> > > > > > > > epoch
> >> > > > >> > >> > > > > > > > > to
> >> > > > >> > >> > > > > > > > > >>> avoid
> >> > > > >> > >> > > > > > > > > >>>>> reusing (sequence, epoch) pairs
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           if (isTransactional()) {
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > >
> >> > > > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> >> > > > >> > >> > > > > > > > > >>>>> this.producerIdAndEpoch);
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           } else {
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > >
> >> requestEpochBumpForPartition(batch.topicPartition);
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           }
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           return true;
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>       }
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>       if (!isTransactional()) {
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // For the idempotent
> >> producer,
> >> > > > >> always
> >> > > > >> > >> retry
> >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> >> > > > >> > >> > > > > > > > > >>>>> errors. If the batch has the current
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           // producer ID and epoch,
> >> > > request a
> >> > > > >> bump
> >> > > > >> > >> of
> >> > > > >> > >> > the
> >> > > > >> > >> > > > > > epoch.
> >> > > > >> > >> > > > > > > > > >>> Otherwise
> >> > > > >> > >> > > > > > > > > >>>>> just retry the produce.
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > >  requestEpochBumpForPartition(batch.topicPartition);
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>           return true;
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>       }
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> }
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> I was considering keeping this
> >> behavior —
> >> > > but
> >> > > > >> am
> >> > > > >> > >> open
> >> > > > >> > >> > to
> >> > > > >> > >> > > > > > > > simplifying
> >> > > > >> > >> > > > > > > > > >>> it.
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> We are leaving changes to older
> >> clients
> >> > off
> >> > > > the
> >> > > > >> > >> table
> >> > > > >> > >> > > here
> >> > > > >> > >> > > > > > since
> >> > > > >> > >> > > > > > > it
> >> > > > >> > >> > > > > > > > > >>> caused
> >> > > > >> > >> > > > > > > > > >>>>> many issues for clients in the past.
> >> > > > Previously
> >> > > > >> > this
> >> > > > >> > >> > was
> >> > > > >> > >> > > a
> >> > > > >> > >> > > > > > fatal
> >> > > > >> > >> > > > > > > > > error
> >> > > > >> > >> > > > > > > > > >>> and
> >> > > > >> > >> > > > > > > > > >>>>> we didn't have the mechanisms in
> >> place to
> >> > > > >> detect
> >> > > > >> > >> when
> >> > > > >> > >> > > this
> >> > > > >> > >> > > > > was
> >> > > > >> > >> > > > > > a
> >> > > > >> > >> > > > > > > > > >>> legitimate
> >> > > > >> > >> > > > > > > > > >>>>> case vs some bug or gap in the
> >> protocol.
> >> > > > >> Ensuring
> >> > > > >> > >> each
> >> > > > >> > >> > > > > > > transaction
> >> > > > >> > >> > > > > > > > > has
> >> > > > >> > >> > > > > > > > > >>> its
> >> > > > >> > >> > > > > > > > > >>>>> own epoch should close this gap.
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> And to address Jeff's second point:
> >> > > > >> > >> > > > > > > > > >>>>> *does the typical produce request
> path
> >> > > append
> >> > > > >> > >> records
> >> > > > >> > >> > to
> >> > > > >> > >> > > > > local
> >> > > > >> > >> > > > > > > log
> >> > > > >> > >> > > > > > > > > >>> along*
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> *with the currentTxnFirstOffset
> >> > > information?
> >> > > > I
> >> > > > >> > would
> >> > > > >> > >> > like
> >> > > > >> > >> > > > to
> >> > > > >> > >> > > > > > > > > >>> understand*
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> *when the field is written to disk.*
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> Yes, the first produce request
> >> populates
> >> > > this
> >> > > > >> > field
> >> > > > >> > >> and
> >> > > > >> > >> > > > > writes
> >> > > > >> > >> > > > > > > the
> >> > > > >> > >> > > > > > > > > >>> offset
> >> > > > >> > >> > > > > > > > > >>>>> as part of the record batch and also
> >> to
> >> > the
> >> > > > >> > producer
> >> > > > >> > >> > > state
> >> > > > >> > >> > > > > > > > snapshot.
> >> > > > >> > >> > > > > > > > > >>> When
> >> > > > >> > >> > > > > > > > > >>>>> we reload the records on restart
> >> and/or
> >> > > > >> > >> reassignment,
> >> > > > >> > >> > we
> >> > > > >> > >> > > > > > > repopulate
> >> > > > >> > >> > > > > > > > > >>> this
> >> > > > >> > >> > > > > > > > > >>>>> field with the snapshot from disk
> >> along
> >> > > with
> >> > > > >> the
> >> > > > >> > >> rest
> >> > > > >> > >> > of
> >> > > > >> > >> > > > the
> >> > > > >> > >> > > > > > > > producer
> >> > > > >> > >> > > > > > > > > >>>>> state.
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> Let me know if there are further
> >> comments
> >> > > > >> and/or
> >> > > > >> > >> > > questions.
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> Thanks,
> >> > > > >> > >> > > > > > > > > >>>>> Justine
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>> On Tue, Nov 22, 2022 at 9:00 PM Jeff
> >> Kim
> >> > > > >> > >> > > > > > > > > <jeff.kim@confluent.io.invalid
> >> > > > >> > >> > > > > > > > > >>>>
> >> > > > >> > >> > > > > > > > > >>>>> wrote:
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>>> Hi Justine,
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>> Thanks for the KIP! I have two
> >> > questions:
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>> 1) For new clients, we can once
> again
> >> > > return
> >> > > > >> an
> >> > > > >> > >> error
> >> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> >> > > > >> > >> > > > > > > > > >>>>>> for sequences
> >> > > > >> > >> > > > > > > > > >>>>>> that are non-zero when there is no
> >> > > producer
> >> > > > >> state
> >> > > > >> > >> > > present
> >> > > > >> > >> > > > on
> >> > > > >> > >> > > > > > the
> >> > > > >> > >> > > > > > > > > >>> server.
> >> > > > >> > >> > > > > > > > > >>>>>> This will indicate we missed the 0
> >> > > sequence
> >> > > > >> and
> >> > > > >> > we
> >> > > > >> > >> > don't
> >> > > > >> > >> > > > yet
> >> > > > >> > >> > > > > > > want
> >> > > > >> > >> > > > > > > > to
> >> > > > >> > >> > > > > > > > > >>>>> write
> >> > > > >> > >> > > > > > > > > >>>>>> to the log.
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>> I would like to understand the
> >> current
> >> > > > >> behavior
> >> > > > >> > to
> >> > > > >> > >> > > handle
> >> > > > >> > >> > > > > > older
> >> > > > >> > >> > > > > > > > > >>> clients,
> >> > > > >> > >> > > > > > > > > >>>>>> and if there are any changes we are
> >> > > making.
> >> > > > >> Maybe
> >> > > > >> > >> I'm
> >> > > > >> > >> > > > > missing
> >> > > > >> > >> > > > > > > > > >>> something,
> >> > > > >> > >> > > > > > > > > >>>>>> but we would want to identify
> >> whether we
> >> > > > >> missed
> >> > > > >> > >> the 0
> >> > > > >> > >> > > > > sequence
> >> > > > >> > >> > > > > > > for
> >> > > > >> > >> > > > > > > > > >>> older
> >> > > > >> > >> > > > > > > > > >>>>>> clients, no?
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>> 2) Upon returning from the
> >> transaction
> >> > > > >> > >> coordinator, we
> >> > > > >> > >> > > can
> >> > > > >> > >> > > > > set
> >> > > > >> > >> > > > > > > the
> >> > > > >> > >> > > > > > > > > >>>>>> transaction
> >> > > > >> > >> > > > > > > > > >>>>>> as ongoing on the leader by
> >> populating
> >> > > > >> > >> > > > currentTxnFirstOffset
> >> > > > >> > >> > > > > > > > > >>>>>> through the typical produce request
> >> > > > handling.
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>> does the typical produce request
> path
> >> > > append
> >> > > > >> > >> records
> >> > > > >> > >> > to
> >> > > > >> > >> > > > > local
> >> > > > >> > >> > > > > > > log
> >> > > > >> > >> > > > > > > > > >>> along
> >> > > > >> > >> > > > > > > > > >>>>>> with the currentTxnFirstOffset
> >> > > information?
> >> > > > I
> >> > > > >> > would
> >> > > > >> > >> > like
> >> > > > >> > >> > > > to
> >> > > > >> > >> > > > > > > > > understand
> >> > > > >> > >> > > > > > > > > >>>>>> when the field is written to disk.
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>> Thanks,
> >> > > > >> > >> > > > > > > > > >>>>>> Jeff
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>> On Tue, Nov 22, 2022 at 4:44 PM
> Artem
> >> > > > Livshits
> >> > > > >> > >> > > > > > > > > >>>>>> <al...@confluent.io.invalid>
> >> wrote:
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>> Hi Justine,
> >> > > > >> > >> > > > > > > > > >>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>> Thank you for the KIP.  I have one
> >> > > > question.
> >> > > > >> > >> > > > > > > > > >>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>> 5) For new clients, we can once
> >> again
> >> > > > return
> >> > > > >> an
> >> > > > >> > >> error
> >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> >> > > > >> > >> > > > > > > > > >>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>> I believe we had problems in the
> >> past
> >> > > with
> >> > > > >> > >> returning
> >> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> >> > > > >> > >> > > > > > > > > >>>>>>> because it was considered fatal
> and
> >> > > > required
> >> > > > >> > >> client
> >> > > > >> > >> > > > > restart.
> >> > > > >> > >> > > > > > > It
> >> > > > >> > >> > > > > > > > > >>> would
> >> > > > >> > >> > > > > > > > > >>>>> be
> >> > > > >> > >> > > > > > > > > >>>>>>> good to spell out the new client
> >> > behavior
> >> > > > >> when
> >> > > > >> > it
> >> > > > >> > >> > > > receives
> >> > > > >> > >> > > > > > the
> >> > > > >> > >> > > > > > > > > error.
> >> > > > >> > >> > > > > > > > > >>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>> -Artem
> >> > > > >> > >> > > > > > > > > >>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>> On Tue, Nov 22, 2022 at 10:00 AM
> >> > Justine
> >> > > > >> Olshan
> >> > > > >> > >> > > > > > > > > >>>>>>> <jo...@confluent.io.invalid>
> >> wrote:
> >> > > > >> > >> > > > > > > > > >>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks for taking a look
> Matthias.
> >> > I've
> >> > > > >> tried
> >> > > > >> > to
> >> > > > >> > >> > > answer
> >> > > > >> > >> > > > > your
> >> > > > >> > >> > > > > > > > > >>>>> questions
> >> > > > >> > >> > > > > > > > > >>>>>>>> below:
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> 10)
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> Right — so the hanging
> transaction
> >> > only
> >> > > > >> occurs
> >> > > > >> > >> when
> >> > > > >> > >> > we
> >> > > > >> > >> > > > > have
> >> > > > >> > >> > > > > > a
> >> > > > >> > >> > > > > > > > late
> >> > > > >> > >> > > > > > > > > >>>>>>> message
> >> > > > >> > >> > > > > > > > > >>>>>>>> come in and the partition is
> never
> >> > added
> >> > > > to
> >> > > > >> a
> >> > > > >> > >> > > > transaction
> >> > > > >> > >> > > > > > > again.
> >> > > > >> > >> > > > > > > > > If
> >> > > > >> > >> > > > > > > > > >>>>> we
> >> > > > >> > >> > > > > > > > > >>>>>>>> never add the partition to a
> >> > > transaction,
> >> > > > we
> >> > > > >> > will
> >> > > > >> > >> > > never
> >> > > > >> > >> > > > > > write
> >> > > > >> > >> > > > > > > a
> >> > > > >> > >> > > > > > > > > >>>>> marker
> >> > > > >> > >> > > > > > > > > >>>>>>> and
> >> > > > >> > >> > > > > > > > > >>>>>>>> never advance the LSO.
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> If we do end up adding the
> >> partition
> >> > to
> >> > > > the
> >> > > > >> > >> > > transaction
> >> > > > >> > >> > > > (I
> >> > > > >> > >> > > > > > > > suppose
> >> > > > >> > >> > > > > > > > > >>>>> this
> >> > > > >> > >> > > > > > > > > >>>>>>> can
> >> > > > >> > >> > > > > > > > > >>>>>>>> happen before or after the late
> >> > message
> >> > > > >> comes
> >> > > > >> > in)
> >> > > > >> > >> > then
> >> > > > >> > >> > > > we
> >> > > > >> > >> > > > > > will
> >> > > > >> > >> > > > > > > > > >>>>> include
> >> > > > >> > >> > > > > > > > > >>>>>>> the
> >> > > > >> > >> > > > > > > > > >>>>>>>> late message in the next
> >> (incorrect)
> >> > > > >> > transaction.
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> So perhaps it is clearer to make
> >> the
> >> > > > >> > distinction
> >> > > > >> > >> > > between
> >> > > > >> > >> > > > > > > > messages
> >> > > > >> > >> > > > > > > > > >>>>> that
> >> > > > >> > >> > > > > > > > > >>>>>>>> eventually get added to the
> >> > transaction
> >> > > > (but
> >> > > > >> > the
> >> > > > >> > >> > wrong
> >> > > > >> > >> > > > > one)
> >> > > > >> > >> > > > > > or
> >> > > > >> > >> > > > > > > > > >>>>> messages
> >> > > > >> > >> > > > > > > > > >>>>>>>> that never get added and become
> >> > hanging.
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> 20)
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> The client side change for 2 is
> >> > removing
> >> > > > the
> >> > > > >> > >> > > > addPartitions
> >> > > > >> > >> > > > > > to
> >> > > > >> > >> > > > > > > > > >>>>>> transaction
> >> > > > >> > >> > > > > > > > > >>>>>>>> call. We don't need to make this
> >> from
> >> > > the
> >> > > > >> > >> producer
> >> > > > >> > >> > to
> >> > > > >> > >> > > > the
> >> > > > >> > >> > > > > > txn
> >> > > > >> > >> > > > > > > > > >>>>>>> coordinator,
> >> > > > >> > >> > > > > > > > > >>>>>>>> only server side.
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> In my opinion, the issue with the
> >> > > > >> > >> addPartitionsToTxn
> >> > > > >> > >> > > > call
> >> > > > >> > >> > > > > > for
> >> > > > >> > >> > > > > > > > > older
> >> > > > >> > >> > > > > > > > > >>>>>>> clients
> >> > > > >> > >> > > > > > > > > >>>>>>>> is that we don't have the epoch
> >> bump,
> >> > so
> >> > > > we
> >> > > > >> > don't
> >> > > > >> > >> > know
> >> > > > >> > >> > > > if
> >> > > > >> > >> > > > > > the
> >> > > > >> > >> > > > > > > > > >>> message
> >> > > > >> > >> > > > > > > > > >>>>>>>> belongs to the previous
> >> transaction or
> >> > > > this
> >> > > > >> > one.
> >> > > > >> > >> We
> >> > > > >> > >> > > need
> >> > > > >> > >> > > > > to
> >> > > > >> > >> > > > > > > > check
> >> > > > >> > >> > > > > > > > > if
> >> > > > >> > >> > > > > > > > > >>>>>> the
> >> > > > >> > >> > > > > > > > > >>>>>>>> partition has been added to this
> >> > > > >> transaction.
> >> > > > >> > Of
> >> > > > >> > >> > > course,
> >> > > > >> > >> > > > > > this
> >> > > > >> > >> > > > > > > > > means
> >> > > > >> > >> > > > > > > > > >>>>> we
> >> > > > >> > >> > > > > > > > > >>>>>>>> won't completely cover the case
> >> where
> >> > we
> >> > > > >> have a
> >> > > > >> > >> > really
> >> > > > >> > >> > > > > late
> >> > > > >> > >> > > > > > > > > message
> >> > > > >> > >> > > > > > > > > >>>>> and
> >> > > > >> > >> > > > > > > > > >>>>>>> we
> >> > > > >> > >> > > > > > > > > >>>>>>>> have added the partition to the
> new
> >> > > > >> > transaction,
> >> > > > >> > >> but
> >> > > > >> > >> > > > > that's
> >> > > > >> > >> > > > > > > > > >>>>>> unfortunately
> >> > > > >> > >> > > > > > > > > >>>>>>>> something we will need the new
> >> clients
> >> > > to
> >> > > > >> > cover.
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> 30)
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> Transaction is ongoing =
> partition
> >> was
> >> > > > >> added to
> >> > > > >> > >> > > > > transaction
> >> > > > >> > >> > > > > > > via
> >> > > > >> > >> > > > > > > > > >>>>>>>> addPartitionsToTxn. We check this
> >> with
> >> > > the
> >> > > > >> > >> > > > > > > DescribeTransactions
> >> > > > >> > >> > > > > > > > > >>> call.
> >> > > > >> > >> > > > > > > > > >>>>>> Let
> >> > > > >> > >> > > > > > > > > >>>>>>>> me know if this wasn't
> sufficiently
> >> > > > >> explained
> >> > > > >> > >> here:
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > >
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > >
> >> > > > >> > >> > > > >
> >> > > > >> > >> > > >
> >> > > > >> > >> > >
> >> > > > >> > >> >
> >> > > > >> > >>
> >> > > > >> >
> >> > > > >>
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> 40)
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> The idea here is that if any
> >> messages
> >> > > > >> somehow
> >> > > > >> > >> come
> >> > > > >> > >> > in
> >> > > > >> > >> > > > > before
> >> > > > >> > >> > > > > > > we
> >> > > > >> > >> > > > > > > > > get
> >> > > > >> > >> > > > > > > > > >>>>> the
> >> > > > >> > >> > > > > > > > > >>>>>>> new
> >> > > > >> > >> > > > > > > > > >>>>>>>> epoch to the producer, they will
> be
> >> > > > fenced.
> >> > > > >> > >> However,
> >> > > > >> > >> > > if
> >> > > > >> > >> > > > we
> >> > > > >> > >> > > > > > > don't
> >> > > > >> > >> > > > > > > > > >>>>> think
> >> > > > >> > >> > > > > > > > > >>>>>>> this
> >> > > > >> > >> > > > > > > > > >>>>>>>> is necessary, it can be discussed
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> 50)
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> It should be synchronous because
> >> if we
> >> > > > have
> >> > > > >> an
> >> > > > >> > >> event
> >> > > > >> > >> > > > (ie,
> >> > > > >> > >> > > > > an
> >> > > > >> > >> > > > > > > > > error)
> >> > > > >> > >> > > > > > > > > >>>>>> that
> >> > > > >> > >> > > > > > > > > >>>>>>>> causes us to need to abort the
> >> > > > transaction,
> >> > > > >> we
> >> > > > >> > >> need
> >> > > > >> > >> > to
> >> > > > >> > >> > > > > know
> >> > > > >> > >> > > > > > > > which
> >> > > > >> > >> > > > > > > > > >>>>>>>> partitions to send transaction
> >> markers
> >> > > to.
> >> > > > >> We
> >> > > > >> > >> know
> >> > > > >> > >> > the
> >> > > > >> > >> > > > > > > > partitions
> >> > > > >> > >> > > > > > > > > >>>>>> because
> >> > > > >> > >> > > > > > > > > >>>>>>>> we added them to the coordinator
> >> via
> >> > the
> >> > > > >> > >> > > > > addPartitionsToTxn
> >> > > > >> > >> > > > > > > > call.
> >> > > > >> > >> > > > > > > > > >>>>>>>> Previously we have had
> asynchronous
> >> > > calls
> >> > > > in
> >> > > > >> > the
> >> > > > >> > >> > past
> >> > > > >> > >> > > > (ie,
> >> > > > >> > >> > > > > > > > writing
> >> > > > >> > >> > > > > > > > > >>>>> the
> >> > > > >> > >> > > > > > > > > >>>>>>>> commit markers when the
> >> transaction is
> >> > > > >> > completed)
> >> > > > >> > >> > but
> >> > > > >> > >> > > > > often
> >> > > > >> > >> > > > > > > this
> >> > > > >> > >> > > > > > > > > >>> just
> >> > > > >> > >> > > > > > > > > >>>>>>>> causes confusion as we need to
> wait
> >> > for
> >> > > > some
> >> > > > >> > >> > > operations
> >> > > > >> > >> > > > to
> >> > > > >> > >> > > > > > > > > complete.
> >> > > > >> > >> > > > > > > > > >>>>> In
> >> > > > >> > >> > > > > > > > > >>>>>>> the
> >> > > > >> > >> > > > > > > > > >>>>>>>> writing commit markers case,
> >> clients
> >> > > often
> >> > > > >> see
> >> > > > >> > >> > > > > > > > > >>>>> CONCURRENT_TRANSACTIONs
> >> > > > >> > >> > > > > > > > > >>>>>>>> error messages and that can be
> >> > > confusing.
> >> > > > >> For
> >> > > > >> > >> that
> >> > > > >> > >> > > > reason,
> >> > > > >> > >> > > > > > it
> >> > > > >> > >> > > > > > > > may
> >> > > > >> > >> > > > > > > > > be
> >> > > > >> > >> > > > > > > > > >>>>>>>> simpler to just have synchronous
> >> > calls —
> >> > > > >> > >> especially
> >> > > > >> > >> > if
> >> > > > >> > >> > > > we
> >> > > > >> > >> > > > > > need
> >> > > > >> > >> > > > > > > > to
> >> > > > >> > >> > > > > > > > > >>>>> block
> >> > > > >> > >> > > > > > > > > >>>>>>> on
> >> > > > >> > >> > > > > > > > > >>>>>>>> some operation's completion
> anyway
> >> > > before
> >> > > > we
> >> > > > >> > can
> >> > > > >> > >> > start
> >> > > > >> > >> > > > the
> >> > > > >> > >> > > > > > > next
> >> > > > >> > >> > > > > > > > > >>>>>>>> transaction. And yes, I meant
> >> > > > coordinator. I
> >> > > > >> > will
> >> > > > >> > >> > fix
> >> > > > >> > >> > > > > that.
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> 60)
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> When we are checking if the
> >> > transaction
> >> > > is
> >> > > > >> > >> ongoing,
> >> > > > >> > >> > we
> >> > > > >> > >> > > > > need
> >> > > > >> > >> > > > > > to
> >> > > > >> > >> > > > > > > > > make
> >> > > > >> > >> > > > > > > > > >>> a
> >> > > > >> > >> > > > > > > > > >>>>>>> round
> >> > > > >> > >> > > > > > > > > >>>>>>>> trip from the leader partition to
> >> the
> >> > > > >> > transaction
> >> > > > >> > >> > > > > > coordinator.
> >> > > > >> > >> > > > > > > > In
> >> > > > >> > >> > > > > > > > > >>> the
> >> > > > >> > >> > > > > > > > > >>>>>>> time
> >> > > > >> > >> > > > > > > > > >>>>>>>> we are waiting for this message
> to
> >> > come
> >> > > > >> back,
> >> > > > >> > in
> >> > > > >> > >> > > theory
> >> > > > >> > >> > > > we
> >> > > > >> > >> > > > > > > could
> >> > > > >> > >> > > > > > > > > >>> have
> >> > > > >> > >> > > > > > > > > >>>>>>> sent
> >> > > > >> > >> > > > > > > > > >>>>>>>> a commit/abort call that would
> make
> >> > the
> >> > > > >> > original
> >> > > > >> > >> > > result
> >> > > > >> > >> > > > of
> >> > > > >> > >> > > > > > the
> >> > > > >> > >> > > > > > > > > check
> >> > > > >> > >> > > > > > > > > >>>>>> out
> >> > > > >> > >> > > > > > > > > >>>>>>> of
> >> > > > >> > >> > > > > > > > > >>>>>>>> date. That is why we can check
> the
> >> > > leader
> >> > > > >> state
> >> > > > >> > >> > before
> >> > > > >> > >> > > > we
> >> > > > >> > >> > > > > > > write
> >> > > > >> > >> > > > > > > > to
> >> > > > >> > >> > > > > > > > > >>>>> the
> >> > > > >> > >> > > > > > > > > >>>>>>> log.
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> I'm happy to update the KIP if
> >> some of
> >> > > > these
> >> > > > >> > >> things
> >> > > > >> > >> > > were
> >> > > > >> > >> > > > > not
> >> > > > >> > >> > > > > > > > > clear.
> >> > > > >> > >> > > > > > > > > >>>>>>>> Thanks,
> >> > > > >> > >> > > > > > > > > >>>>>>>> Justine
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>> On Mon, Nov 21, 2022 at 7:11 PM
> >> > Matthias
> >> > > > J.
> >> > > > >> > Sax <
> >> > > > >> > >> > > > > > > > mjsax@apache.org
> >> > > > >> > >> > > > > > > > > >
> >> > > > >> > >> > > > > > > > > >>>>>>> wrote:
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> Thanks for the KIP.
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> Couple of clarification
> questions
> >> (I
> >> > am
> >> > > > >> not a
> >> > > > >> > >> > broker
> >> > > > >> > >> > > > > expert
> >> > > > >> > >> > > > > > > do
> >> > > > >> > >> > > > > > > > > >>>>> maybe
> >> > > > >> > >> > > > > > > > > >>>>>>>>> some question are obvious for
> >> others,
> >> > > but
> >> > > > >> not
> >> > > > >> > >> for
> >> > > > >> > >> > me
> >> > > > >> > >> > > > with
> >> > > > >> > >> > > > > > my
> >> > > > >> > >> > > > > > > > lack
> >> > > > >> > >> > > > > > > > > >>>>> of
> >> > > > >> > >> > > > > > > > > >>>>>>>>> broker knowledge).
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> (10)
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> 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.
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> What happens if the message come
> >> in
> >> > > > before
> >> > > > >> the
> >> > > > >> > >> next
> >> > > > >> > >> > > > > > > > > >>>>>> addPartitionsToTxn
> >> > > > >> > >> > > > > > > > > >>>>>>>>> request? It seems the broker
> >> hosting
> >> > > the
> >> > > > >> data
> >> > > > >> > >> > > > partitions
> >> > > > >> > >> > > > > > > won't
> >> > > > >> > >> > > > > > > > > know
> >> > > > >> > >> > > > > > > > > >>>>>>>>> anything about it and append it
> to
> >> > the
> >> > > > >> > >> partition,
> >> > > > >> > >> > > too?
> >> > > > >> > >> > > > > What
> >> > > > >> > >> > > > > > > is
> >> > > > >> > >> > > > > > > > > the
> >> > > > >> > >> > > > > > > > > >>>>>>>>> difference between both cases?
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> Also, it seems a TX would only
> >> hang,
> >> > if
> >> > > > >> there
> >> > > > >> > >> is no
> >> > > > >> > >> > > > > > following
> >> > > > >> > >> > > > > > > > TX
> >> > > > >> > >> > > > > > > > > >>>>> that
> >> > > > >> > >> > > > > > > > > >>>>>>> is
> >> > > > >> > >> > > > > > > > > >>>>>>>>> either committer or aborted?
> Thus,
> >> > for
> >> > > > the
> >> > > > >> > case
> >> > > > >> > >> > > above,
> >> > > > >> > >> > > > > the
> >> > > > >> > >> > > > > > TX
> >> > > > >> > >> > > > > > > > > might
> >> > > > >> > >> > > > > > > > > >>>>>>>>> actually not hang (of course, we
> >> > might
> >> > > > get
> >> > > > >> an
> >> > > > >> > >> EOS
> >> > > > >> > >> > > > > violation
> >> > > > >> > >> > > > > > > if
> >> > > > >> > >> > > > > > > > > the
> >> > > > >> > >> > > > > > > > > >>>>>>> first
> >> > > > >> > >> > > > > > > > > >>>>>>>>> TX was aborted and the second
> >> > > committed,
> >> > > > or
> >> > > > >> > the
> >> > > > >> > >> > other
> >> > > > >> > >> > > > way
> >> > > > >> > >> > > > > > > > > around).
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> (20)
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> Of course, 1 and 2 require
> >> > client-side
> >> > > > >> > >> changes, so
> >> > > > >> > >> > > for
> >> > > > >> > >> > > > > > older
> >> > > > >> > >> > > > > > > > > >>>>>> clients,
> >> > > > >> > >> > > > > > > > > >>>>>>>>> those approaches won’t apply.
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> For (1) I understand why a
> client
> >> > > change
> >> > > > is
> >> > > > >> > >> > > necessary,
> >> > > > >> > >> > > > > but
> >> > > > >> > >> > > > > > > not
> >> > > > >> > >> > > > > > > > > sure
> >> > > > >> > >> > > > > > > > > >>>>>> why
> >> > > > >> > >> > > > > > > > > >>>>>>>>> we need a client change for (2).
> >> Can
> >> > > you
> >> > > > >> > >> elaborate?
> >> > > > >> > >> > > --
> >> > > > >> > >> > > > > > Later
> >> > > > >> > >> > > > > > > > you
> >> > > > >> > >> > > > > > > > > >>>>>>> explain
> >> > > > >> > >> > > > > > > > > >>>>>>>>> that we should send a
> >> > > > >> > >> DescribeTransactionRequest,
> >> > > > >> > >> > > but I
> >> > > > >> > >> > > > > am
> >> > > > >> > >> > > > > > > not
> >> > > > >> > >> > > > > > > > > sure
> >> > > > >> > >> > > > > > > > > >>>>>>> why?
> >> > > > >> > >> > > > > > > > > >>>>>>>>> Can't we not just do an implicit
> >> > > > >> > >> AddPartiitonToTx,
> >> > > > >> > >> > > too?
> >> > > > >> > >> > > > > If
> >> > > > >> > >> > > > > > > the
> >> > > > >> > >> > > > > > > > > old
> >> > > > >> > >> > > > > > > > > >>>>>>>>> producer correctly registered
> the
> >> > > > partition
> >> > > > >> > >> > already,
> >> > > > >> > >> > > > the
> >> > > > >> > >> > > > > > > > > >>>>>> TX-coordinator
> >> > > > >> > >> > > > > > > > > >>>>>>>>> can just ignore it as it's an
> >> > > idempotent
> >> > > > >> > >> operation?
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> (30)
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> To cover older clients, we will
> >> > > ensure a
> >> > > > >> > >> > transaction
> >> > > > >> > >> > > > is
> >> > > > >> > >> > > > > > > > ongoing
> >> > > > >> > >> > > > > > > > > >>>>>>> before
> >> > > > >> > >> > > > > > > > > >>>>>>>>> we write to a transaction
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> Not sure what you mean by this?
> >> Can
> >> > you
> >> > > > >> > >> elaborate?
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> (40)
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> [the TX-coordinator] will write
> >> the
> >> > > > >> prepare
> >> > > > >> > >> commit
> >> > > > >> > >> > > > > message
> >> > > > >> > >> > > > > > > > with
> >> > > > >> > >> > > > > > > > > a
> >> > > > >> > >> > > > > > > > > >>>>>>>> bumped
> >> > > > >> > >> > > > > > > > > >>>>>>>>> epoch and send
> >> WriteTxnMarkerRequests
> >> > > > with
> >> > > > >> the
> >> > > > >> > >> > bumped
> >> > > > >> > >> > > > > > epoch.
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> Why do we use the bumped epoch
> for
> >> > > both?
> >> > > > It
> >> > > > >> > >> seems
> >> > > > >> > >> > > more
> >> > > > >> > >> > > > > > > > intuitive
> >> > > > >> > >> > > > > > > > > to
> >> > > > >> > >> > > > > > > > > >>>>>> use
> >> > > > >> > >> > > > > > > > > >>>>>>>>> the current epoch, and only
> return
> >> > the
> >> > > > >> bumped
> >> > > > >> > >> epoch
> >> > > > >> > >> > > to
> >> > > > >> > >> > > > > the
> >> > > > >> > >> > > > > > > > > >>>>> producer?
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> (50) "Implicit
> >> > > AddPartitionToTransaction"
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> Why does the implicitly sent
> >> request
> >> > > need
> >> > > > >> to
> >> > > > >> > be
> >> > > > >> > >> > > > > > synchronous?
> >> > > > >> > >> > > > > > > > The
> >> > > > >> > >> > > > > > > > > >>>>> KIP
> >> > > > >> > >> > > > > > > > > >>>>>>>>> also says
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> in case we need to abort and
> >> need to
> >> > > > know
> >> > > > >> > which
> >> > > > >> > >> > > > > partitions
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> What do you mean by this?
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> we don’t want to write to it
> >> before
> >> > we
> >> > > > >> store
> >> > > > >> > in
> >> > > > >> > >> > the
> >> > > > >> > >> > > > > > > > transaction
> >> > > > >> > >> > > > > > > > > >>>>>>> manager
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> Do you mean TX-coordinator
> >> instead of
> >> > > > >> > "manager"?
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> (60)
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> For older clients and ensuring
> >> that
> >> > the
> >> > > > TX
> >> > > > >> is
> >> > > > >> > >> > > ongoing,
> >> > > > >> > >> > > > > you
> >> > > > >> > >> > > > > > > > > >>>>> describe a
> >> > > > >> > >> > > > > > > > > >>>>>>>>> race condition. I am not sure
> if I
> >> > can
> >> > > > >> follow
> >> > > > >> > >> here.
> >> > > > >> > >> > > Can
> >> > > > >> > >> > > > > you
> >> > > > >> > >> > > > > > > > > >>>>>> elaborate?
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> -Matthias
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>> On 11/18/22 1:21 PM, Justine
> >> Olshan
> >> > > > wrote:
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> Hey all!
> >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> I'd like to start a discussion
> >> on my
> >> > > > >> proposal
> >> > > > >> > >> to
> >> > > > >> > >> > add
> >> > > > >> > >> > > > > some
> >> > > > >> > >> > > > > > > > > >>>>>> server-side
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> checks on transactions to avoid
> >> > > hanging
> >> > > > >> > >> > > transactions.
> >> > > > >> > >> > > > I
> >> > > > >> > >> > > > > > know
> >> > > > >> > >> > > > > > > > > this
> >> > > > >> > >> > > > > > > > > >>>>>> has
> >> > > > >> > >> > > > > > > > > >>>>>>>>> been
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> an issue for some time, so I
> >> really
> >> > > hope
> >> > > > >> this
> >> > > > >> > >> KIP
> >> > > > >> > >> > > will
> >> > > > >> > >> > > > > be
> >> > > > >> > >> > > > > > > > > helpful
> >> > > > >> > >> > > > > > > > > >>>>>> for
> >> > > > >> > >> > > > > > > > > >>>>>>>>> many
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> users of EOS.
> >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> The KIP includes changes that
> >> will
> >> > be
> >> > > > >> > >> compatible
> >> > > > >> > >> > > with
> >> > > > >> > >> > > > > old
> >> > > > >> > >> > > > > > > > > clients
> >> > > > >> > >> > > > > > > > > >>>>>> and
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> changes to improve performance
> >> and
> >> > > > >> > correctness
> >> > > > >> > >> on
> >> > > > >> > >> > > new
> >> > > > >> > >> > > > > > > clients.
> >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> Please take a look and leave
> any
> >> > > > comments
> >> > > > >> you
> >> > > > >> > >> may
> >> > > > >> > >> > > > have!
> >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> KIP:
> >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > >
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > >
> >> > > > >> > >> > > > >
> >> > > > >> > >> > > >
> >> > > > >> > >> > >
> >> > > > >> > >> >
> >> > > > >> > >>
> >> > > > >> >
> >> > > > >>
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> JIRA:
> >> > > > >> > >> > > > https://issues.apache.org/jira/browse/KAFKA-14402
> >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> Thanks!
> >> > > > >> > >> > > > > > > > > >>>>>>>>>> Justine
> >> > > > >> > >> > > > > > > > > >>>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>>
> >> > > > >> > >> > > > > > > > > >>>>>
> >> > > > >> > >> > > > > > > > > >>>>
> >> > > > >> > >> > > > > > > > > >>>
> >> > > > >> > >> > > > > > > > > >>
> >> > > > >> > >> > > > > > > > > >
> >> > > > >> > >> > > > > > > > >
> >> > > > >> > >> > > > > > > >
> >> > > > >> > >> > > > > > >
> >> > > > >> > >> > > > > >
> >> > > > >> > >> > > > >
> >> > > > >> > >> > > >
> >> > > > >> > >> > >
> >> > > > >> > >> >
> >> > > > >> > >>
> >> > > > >> > >
> >> > > > >> >
> >> > > > >>
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
Yeah -- looks like we already have code to handle bumping the epoch and
when the epoch is Short.MAX_VALUE, we get a new producer ID. Since this is
already the behavior, do we want to change it further?

Justine

On Wed, Jan 18, 2023 at 1:12 PM Justine Olshan <jo...@confluent.io> wrote:

> Hey all, just wanted to quickly update and say I've modified the KIP to
> explicitly mention that AddOffsetCommitsToTxnRequest will be replaced by
> a coordinator-side (inter-broker) AddPartitionsToTxn implicit request. This
> mirrors the user partitions and will implicitly add offset partitions to
> transactions when we commit offsets on them. We will deprecate AddOffsetCommitsToTxnRequest
> for new clients.
>
> Also to address Artem's comments --
> I'm a bit unsure if the changes here will change the previous behavior for
> fencing producers. In the case you mention in the first paragraph, are you
> saying we bump the epoch before we try to abort the transaction? I think I
> need to understand the scenarios you mention a bit better.
>
> As for the second part -- I think it makes sense to have some sort of
> "sentinel" epoch to signal epoch is about to overflow (I think we sort of
> have this value in place in some ways) so we can codify it in the KIP. I'll
> look into that and try to update soon.
>
> Thanks,
> Justine.
>
> On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
> <al...@confluent.io.invalid> wrote:
>
>> It's good to know that KIP-588 addressed some of the issues.  Looking at
>> the code, it still looks like there are some cases that would result in
>> fatal error, e.g. PRODUCER_FENCED is issued by the transaction coordinator
>> if epoch doesn't match, and the client treats it as a fatal error (code in
>> TransactionManager request handling).  If we consider, for example,
>> committing a transaction that returns a timeout, but actually succeeds,
>> trying to abort it or re-commit may result in PRODUCER_FENCED error
>> (because of epoch bump).
>>
>> For failed commits, specifically, we need to know the actual outcome,
>> because if we return an error the application may think that the
>> transaction is aborted and redo the work, leading to duplicates.
>>
>> Re: overflowing epoch.  We could either do it on the TC and return both
>> producer id and epoch (e.g. change the protocol), or signal the client
>> that
>> it needs to get a new producer id.  Checking for max epoch could be a
>> reasonable signal, the value to check should probably be present in the
>> KIP
>> as this is effectively a part of the contract.  Also, the TC should
>> probably return an error if the client didn't change producer id after
>> hitting max epoch.
>>
>> -Artem
>>
>>
>> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
>> <jo...@confluent.io.invalid> wrote:
>>
>> > Thanks for the discussion Artem.
>> >
>> > With respect to the handling of fenced producers, we have some behavior
>> > already in place. As of KIP-588:
>> >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
>> > ,
>> > we handle timeouts more gracefully. The producer can recover.
>> >
>> > Produce requests can also recover from epoch fencing by aborting the
>> > transaction and starting over.
>> >
>> > What other cases were you considering that would cause us to have a
>> fenced
>> > epoch but we'd want to recover?
>> >
>> > The first point about handling epoch overflows is fair. I think there is
>> > some logic we'd need to consider. (ie, if we are one away from the max
>> > epoch, we need to reset the producer ID.) I'm still wondering if there
>> is a
>> > way to direct this from the response, or if everything should be done on
>> > the client side. Let me know if you have any thoughts here.
>> >
>> > Thanks,
>> > Justine
>> >
>> > On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
>> > <al...@confluent.io.invalid> wrote:
>> >
>> > > There are some workflows in the client that are implied by protocol
>> > > changes, e.g.:
>> > >
>> > > - for new clients, epoch changes with every transaction and can
>> overflow,
>> > > in old clients this condition was handled transparently, because epoch
>> > was
>> > > bumped in InitProducerId and it would return a new producer id if
>> epoch
>> > > overflows, the new clients would need to implement some workflow to
>> > refresh
>> > > producer id
>> > > - how to handle fenced producers, for new clients epoch changes with
>> > every
>> > > transaction, so in presence of failures during commits / aborts, the
>> > > producer could get easily fenced, old clients would pretty much would
>> get
>> > > fenced when a new incarnation of the producer was initialized with
>> > > InitProducerId so it's ok to treat as a fatal error, the new clients
>> > would
>> > > need to implement some workflow to handle that error, otherwise they
>> > could
>> > > get fenced by themselves
>> > > - in particular (as a subset of the previous issue), what would the
>> > client
>> > > do if it got a timeout during commit?  commit could've succeeded or
>> > failed
>> > >
>> > > Not sure if this has to be defined in the KIP as implementing those
>> > > probably wouldn't require protocol changes, but we have multiple
>> > > implementations of Kafka clients, so probably would be good to have
>> some
>> > > client implementation guidance.  Could also be done as a separate doc.
>> > >
>> > > -Artem
>> > >
>> > > On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
>> > <jolshan@confluent.io.invalid
>> > > >
>> > > wrote:
>> > >
>> > > > Hey all, I've updated the KIP to incorporate Jason's suggestions.
>> > > >
>> > > >
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
>> > > >
>> > > >
>> > > > 1. Use AddPartitionsToTxn + verify flag to check on old clients
>> > > > 2. Updated AddPartitionsToTxn API to support transaction batching
>> > > > 3. Mention IBP bump
>> > > > 4. Mention auth change on new AddPartitionsToTxn version.
>> > > >
>> > > > I'm planning on opening a vote soon.
>> > > > Thanks,
>> > > > Justine
>> > > >
>> > > > On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <jolshan@confluent.io
>> >
>> > > > wrote:
>> > > >
>> > > > > Thanks Jason. Those changes make sense to me. I will update the
>> KIP.
>> > > > >
>> > > > >
>> > > > >
>> > > > > On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
>> > > > <ja...@confluent.io.invalid>
>> > > > > wrote:
>> > > > >
>> > > > >> Hey Justine,
>> > > > >>
>> > > > >> > I was wondering about compatibility here. When we send requests
>> > > > >> between brokers, we want to ensure that the receiving broker
>> > > understands
>> > > > >> the request (specifically the new fields). Typically this is done
>> > via
>> > > > >> IBP/metadata version.
>> > > > >> I'm trying to think if there is a way around it but I'm not sure
>> > there
>> > > > is.
>> > > > >>
>> > > > >> Yes. I think we would gate usage of this behind an IBP bump. Does
>> > that
>> > > > >> seem
>> > > > >> reasonable?
>> > > > >>
>> > > > >> > As for the improvements -- can you clarify how the multiple
>> > > > >> transactional
>> > > > >> IDs would help here? Were you thinking of a case where we
>> wait/batch
>> > > > >> multiple produce requests together? My understanding for now was
>> 1
>> > > > >> transactional ID and one validation per 1 produce request.
>> > > > >>
>> > > > >> Each call to `AddPartitionsToTxn` is essentially a write to the
>> > > > >> transaction
>> > > > >> log and must block on replication. The more we can fit into a
>> single
>> > > > >> request, the more writes we can do in parallel. The alternative
>> is
>> > to
>> > > > make
>> > > > >> use of more connections, but usually we prefer batching since the
>> > > > network
>> > > > >> stack is not really optimized for high connection/request loads.
>> > > > >>
>> > > > >> > Finally with respect to the authorizations, I think it makes
>> sense
>> > > to
>> > > > >> skip
>> > > > >> topic authorizations, but I'm a bit confused by the "leader ID"
>> > field.
>> > > > >> Wouldn't we just want to flag the request as from a broker (does
>> it
>> > > > matter
>> > > > >> which one?).
>> > > > >>
>> > > > >> We could also make it version-based. For the next version, we
>> could
>> > > > >> require
>> > > > >> CLUSTER auth. So clients would not be able to use the API
>> anymore,
>> > > which
>> > > > >> is
>> > > > >> probably what we want.
>> > > > >>
>> > > > >> -Jason
>> > > > >>
>> > > > >> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
>> > > > >> <jo...@confluent.io.invalid>
>> > > > >> wrote:
>> > > > >>
>> > > > >> > As a follow up, I was just thinking about the batching a bit
>> more.
>> > > > >> > I suppose if we have one request in flight and we queue up the
>> > other
>> > > > >> > produce requests in some sort of purgatory, we could send
>> > > information
>> > > > >> out
>> > > > >> > for all of them rather than one by one. So that would be a
>> benefit
>> > > of
>> > > > >> > batching partitions to add per transaction.
>> > > > >> >
>> > > > >> > I'll need to think a bit more on the design of this part of the
>> > KIP,
>> > > > and
>> > > > >> > will update the KIP in the next few days.
>> > > > >> >
>> > > > >> > Thanks,
>> > > > >> > Justine
>> > > > >> >
>> > > > >> > On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan <
>> > > jolshan@confluent.io>
>> > > > >> > wrote:
>> > > > >> >
>> > > > >> > > Hey Jason -- thanks for the input -- I was just digging a bit
>> > > deeper
>> > > > >> into
>> > > > >> > > the design + implementation of the validation calls here and
>> > what
>> > > > you
>> > > > >> say
>> > > > >> > > makes sense.
>> > > > >> > >
>> > > > >> > > I was wondering about compatibility here. When we send
>> requests
>> > > > >> > > between brokers, we want to ensure that the receiving broker
>> > > > >> understands
>> > > > >> > > the request (specifically the new fields). Typically this is
>> > done
>> > > > via
>> > > > >> > > IBP/metadata version.
>> > > > >> > > I'm trying to think if there is a way around it but I'm not
>> sure
>> > > > there
>> > > > >> > is.
>> > > > >> > >
>> > > > >> > > As for the improvements -- can you clarify how the multiple
>> > > > >> transactional
>> > > > >> > > IDs would help here? Were you thinking of a case where we
>> > > wait/batch
>> > > > >> > > multiple produce requests together? My understanding for now
>> > was 1
>> > > > >> > > transactional ID and one validation per 1 produce request.
>> > > > >> > >
>> > > > >> > > Finally with respect to the authorizations, I think it makes
>> > sense
>> > > > to
>> > > > >> > skip
>> > > > >> > > topic authorizations, but I'm a bit confused by the "leader
>> ID"
>> > > > field.
>> > > > >> > > Wouldn't we just want to flag the request as from a broker
>> (does
>> > > it
>> > > > >> > matter
>> > > > >> > > which one?).
>> > > > >> > >
>> > > > >> > > I think I want to adopt these suggestions, just had a few
>> > > questions
>> > > > on
>> > > > >> > the
>> > > > >> > > details.
>> > > > >> > >
>> > > > >> > > Thanks,
>> > > > >> > > Justine
>> > > > >> > >
>> > > > >> > > On Thu, Jan 5, 2023 at 5:05 PM Jason Gustafson
>> > > > >> > <ja...@confluent.io.invalid>
>> > > > >> > > wrote:
>> > > > >> > >
>> > > > >> > >> Hi Justine,
>> > > > >> > >>
>> > > > >> > >> Thanks for the proposal.
>> > > > >> > >>
>> > > > >> > >> I was thinking about the implementation a little bit. In the
>> > > > current
>> > > > >> > >> proposal, the behavior depends on whether we have an old or
>> new
>> > > > >> client.
>> > > > >> > >> For
>> > > > >> > >> old clients, we send `DescribeTransactions` and verify the
>> > result
>> > > > and
>> > > > >> > for
>> > > > >> > >> new clients, we send `AddPartitionsToTxn`. We might be able
>> to
>> > > > >> simplify
>> > > > >> > >> the
>> > > > >> > >> implementation if we can use the same request type. For
>> > example,
>> > > > >> what if
>> > > > >> > >> we
>> > > > >> > >> bump the protocol version for `AddPartitionsToTxn` and add a
>> > > > >> > >> `validateOnly`
>> > > > >> > >> flag? For older versions, we can set `validateOnly=true` so
>> > that
>> > > > the
>> > > > >> > >> request only returns successfully if the partition had
>> already
>> > > been
>> > > > >> > added.
>> > > > >> > >> For new versions, we can set `validateOnly=false` and the
>> > > partition
>> > > > >> will
>> > > > >> > >> be
>> > > > >> > >> added to the transaction. The other slightly annoying thing
>> > that
>> > > > this
>> > > > >> > >> would
>> > > > >> > >> get around is the need to collect the transaction state for
>> all
>> > > > >> > partitions
>> > > > >> > >> even when we only care about a subset.
>> > > > >> > >>
>> > > > >> > >> Some additional improvements to consider:
>> > > > >> > >>
>> > > > >> > >> - We can give `AddPartitionsToTxn` better batch support for
>> > > > >> inter-broker
>> > > > >> > >> usage. Currently we only allow one `TransactionalId` to be
>> > > > specified,
>> > > > >> > but
>> > > > >> > >> the broker may get some benefit being able to batch across
>> > > multiple
>> > > > >> > >> transactions.
>> > > > >> > >> - Another small improvement is skipping topic authorization
>> > > checks
>> > > > >> for
>> > > > >> > >> `AddPartitionsToTxn` when the request is from a broker.
>> Perhaps
>> > > we
>> > > > >> can
>> > > > >> > add
>> > > > >> > >> a field for the `LeaderId` or something like that and
>> require
>> > > > CLUSTER
>> > > > >> > >> permission when set.
>> > > > >> > >>
>> > > > >> > >> Best,
>> > > > >> > >> Jason
>> > > > >> > >>
>> > > > >> > >>
>> > > > >> > >>
>> > > > >> > >> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
>> > <jun@confluent.io.invalid
>> > > >
>> > > > >> > wrote:
>> > > > >> > >>
>> > > > >> > >> > Hi, Justine,
>> > > > >> > >> >
>> > > > >> > >> > Thanks for the explanation. It makes sense to me now.
>> > > > >> > >> >
>> > > > >> > >> > Jun
>> > > > >> > >> >
>> > > > >> > >> > On Mon, Dec 19, 2022 at 1:42 PM Justine Olshan
>> > > > >> > >> > <jo...@confluent.io.invalid>
>> > > > >> > >> > wrote:
>> > > > >> > >> >
>> > > > >> > >> > > Hi Jun,
>> > > > >> > >> > >
>> > > > >> > >> > > My understanding of the mechanism is that when we get to
>> > the
>> > > > last
>> > > > >> > >> epoch,
>> > > > >> > >> > we
>> > > > >> > >> > > increment to the fencing/last epoch and if any further
>> > > requests
>> > > > >> come
>> > > > >> > >> in
>> > > > >> > >> > for
>> > > > >> > >> > > this producer ID they are fenced. Then the producer
>> gets a
>> > > new
>> > > > ID
>> > > > >> > and
>> > > > >> > >> > > restarts with epoch/sequence 0. The fenced epoch sticks
>> > > around
>> > > > >> for
>> > > > >> > the
>> > > > >> > >> > > duration of producer.id.expiration.ms and blocks any
>> late
>> > > > >> messages
>> > > > >> > >> > there.
>> > > > >> > >> > > The new ID will get to take advantage of the improved
>> > > semantics
>> > > > >> > around
>> > > > >> > >> > > non-zero start sequences. So I think we are covered.
>> > > > >> > >> > >
>> > > > >> > >> > > The only potential issue is overloading the cache, but
>> > > > hopefully
>> > > > >> the
>> > > > >> > >> > > improvements (lowered producer.id.expiration.ms) will
>> help
>> > > > with
>> > > > >> > that.
>> > > > >> > >> > Let
>> > > > >> > >> > > me know if you still have concerns.
>> > > > >> > >> > >
>> > > > >> > >> > > Thanks,
>> > > > >> > >> > > Justine
>> > > > >> > >> > >
>> > > > >> > >> > > On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
>> > > > >> <ju...@confluent.io.invalid>
>> > > > >> > >> > wrote:
>> > > > >> > >> > >
>> > > > >> > >> > > > Hi, Justine,
>> > > > >> > >> > > >
>> > > > >> > >> > > > Thanks for the explanation.
>> > > > >> > >> > > >
>> > > > >> > >> > > > 70. The proposed fencing logic doesn't apply when pid
>> > > > changes,
>> > > > >> is
>> > > > >> > >> that
>> > > > >> > >> > > > right? If so, I am not sure how complete we are
>> > addressing
>> > > > this
>> > > > >> > >> issue
>> > > > >> > >> > if
>> > > > >> > >> > > > the pid changes more frequently.
>> > > > >> > >> > > >
>> > > > >> > >> > > > Thanks,
>> > > > >> > >> > > >
>> > > > >> > >> > > > Jun
>> > > > >> > >> > > >
>> > > > >> > >> > > >
>> > > > >> > >> > > >
>> > > > >> > >> > > > On Fri, Dec 16, 2022 at 9:16 AM Justine Olshan
>> > > > >> > >> > > > <jo...@confluent.io.invalid>
>> > > > >> > >> > > > wrote:
>> > > > >> > >> > > >
>> > > > >> > >> > > > > Hi Jun,
>> > > > >> > >> > > > >
>> > > > >> > >> > > > > Thanks for replying!
>> > > > >> > >> > > > >
>> > > > >> > >> > > > > 70.We already do the overflow mechanism, so my
>> change
>> > > would
>> > > > >> just
>> > > > >> > >> make
>> > > > >> > >> > > it
>> > > > >> > >> > > > > happen more often.
>> > > > >> > >> > > > > I was also not suggesting a new field in the log,
>> but
>> > in
>> > > > the
>> > > > >> > >> > response,
>> > > > >> > >> > > > > which would be gated by the client version. Sorry if
>> > > > >> something
>> > > > >> > >> there
>> > > > >> > >> > is
>> > > > >> > >> > > > > unclear. I think we are starting to diverge.
>> > > > >> > >> > > > > The goal of this KIP is to not change to the marker
>> > > format
>> > > > at
>> > > > >> > all.
>> > > > >> > >> > > > >
>> > > > >> > >> > > > > 71. Yes, I guess I was going under the assumption
>> that
>> > > the
>> > > > >> log
>> > > > >> > >> would
>> > > > >> > >> > > just
>> > > > >> > >> > > > > look at its last epoch and treat it as the current
>> > > epoch. I
>> > > > >> > >> suppose
>> > > > >> > >> > we
>> > > > >> > >> > > > can
>> > > > >> > >> > > > > have some special logic that if the last epoch was
>> on a
>> > > > >> marker
>> > > > >> > we
>> > > > >> > >> > > > actually
>> > > > >> > >> > > > > expect the next epoch or something like that. We
>> just
>> > > need
>> > > > to
>> > > > >> > >> > > distinguish
>> > > > >> > >> > > > > based on whether we had a commit/abort marker.
>> > > > >> > >> > > > >
>> > > > >> > >> > > > > 72.
>> > > > >> > >> > > > > > if the producer epoch hasn't been bumped on the
>> > > > >> > >> > > > > broker, it seems that the stucked message will fail
>> the
>> > > > >> sequence
>> > > > >> > >> > > > validation
>> > > > >> > >> > > > > and will be ignored. If the producer epoch has been
>> > > bumped,
>> > > > >> we
>> > > > >> > >> ignore
>> > > > >> > >> > > the
>> > > > >> > >> > > > > sequence check and the stuck message could be
>> appended
>> > to
>> > > > the
>> > > > >> > log.
>> > > > >> > >> > So,
>> > > > >> > >> > > is
>> > > > >> > >> > > > > the latter case that we want to guard?
>> > > > >> > >> > > > >
>> > > > >> > >> > > > > I'm not sure I follow that "the message will fail
>> the
>> > > > >> sequence
>> > > > >> > >> > > > validation".
>> > > > >> > >> > > > > In some of these cases, we had an abort marker (due
>> to
>> > an
>> > > > >> error)
>> > > > >> > >> and
>> > > > >> > >> > > then
>> > > > >> > >> > > > > the late message comes in with the correct sequence
>> > > number.
>> > > > >> This
>> > > > >> > >> is a
>> > > > >> > >> > > > case
>> > > > >> > >> > > > > covered by the KIP.
>> > > > >> > >> > > > > The latter case is actually not something we've
>> > > considered
>> > > > >> > here. I
>> > > > >> > >> > > think
>> > > > >> > >> > > > > generally when we bump the epoch, we are accepting
>> that
>> > > the
>> > > > >> > >> sequence
>> > > > >> > >> > > does
>> > > > >> > >> > > > > not need to be checked anymore. My understanding is
>> > also
>> > > > >> that we
>> > > > >> > >> > don't
>> > > > >> > >> > > > > typically bump epoch mid transaction (based on a
>> quick
>> > > look
>> > > > >> at
>> > > > >> > the
>> > > > >> > >> > > code)
>> > > > >> > >> > > > > but let me know if that is the case.
>> > > > >> > >> > > > >
>> > > > >> > >> > > > > Thanks,
>> > > > >> > >> > > > > Justine
>> > > > >> > >> > > > >
>> > > > >> > >> > > > > On Thu, Dec 15, 2022 at 12:23 PM Jun Rao
>> > > > >> > <jun@confluent.io.invalid
>> > > > >> > >> >
>> > > > >> > >> > > > wrote:
>> > > > >> > >> > > > >
>> > > > >> > >> > > > > > Hi, Justine,
>> > > > >> > >> > > > > >
>> > > > >> > >> > > > > > Thanks for the reply.
>> > > > >> > >> > > > > >
>> > > > >> > >> > > > > > 70. Assigning a new pid on int overflow seems a
>> bit
>> > > > hacky.
>> > > > >> If
>> > > > >> > we
>> > > > >> > >> > > need a
>> > > > >> > >> > > > > txn
>> > > > >> > >> > > > > > level id, it will be better to model this
>> explicitly.
>> > > > >> Adding a
>> > > > >> > >> new
>> > > > >> > >> > > > field
>> > > > >> > >> > > > > > would require a bit more work since it requires a
>> new
>> > > txn
>> > > > >> > marker
>> > > > >> > >> > > format
>> > > > >> > >> > > > > in
>> > > > >> > >> > > > > > the log. So, we probably need to guard it with an
>> IBP
>> > > or
>> > > > >> > >> metadata
>> > > > >> > >> > > > version
>> > > > >> > >> > > > > > and document the impact on downgrade once the new
>> > > format
>> > > > is
>> > > > >> > >> written
>> > > > >> > >> > > to
>> > > > >> > >> > > > > the
>> > > > >> > >> > > > > > log.
>> > > > >> > >> > > > > >
>> > > > >> > >> > > > > > 71. Hmm, once the marker is written, the partition
>> > will
>> > > > >> expect
>> > > > >> > >> the
>> > > > >> > >> > > next
>> > > > >> > >> > > > > > append to be on the next epoch. Does that cover
>> the
>> > > case
>> > > > >> you
>> > > > >> > >> > > mentioned?
>> > > > >> > >> > > > > >
>> > > > >> > >> > > > > > 72. Also, just to be clear on the stucked message
>> > issue
>> > > > >> > >> described
>> > > > >> > >> > in
>> > > > >> > >> > > > the
>> > > > >> > >> > > > > > motivation. With EoS, we also validate the
>> sequence
>> > id
>> > > > for
>> > > > >> > >> > > idempotency.
>> > > > >> > >> > > > > So,
>> > > > >> > >> > > > > > with the current logic, if the producer epoch
>> hasn't
>> > > been
>> > > > >> > >> bumped on
>> > > > >> > >> > > the
>> > > > >> > >> > > > > > broker, it seems that the stucked message will
>> fail
>> > the
>> > > > >> > sequence
>> > > > >> > >> > > > > validation
>> > > > >> > >> > > > > > and will be ignored. If the producer epoch has
>> been
>> > > > >> bumped, we
>> > > > >> > >> > ignore
>> > > > >> > >> > > > the
>> > > > >> > >> > > > > > sequence check and the stuck message could be
>> > appended
>> > > to
>> > > > >> the
>> > > > >> > >> log.
>> > > > >> > >> > > So,
>> > > > >> > >> > > > is
>> > > > >> > >> > > > > > the latter case that we want to guard?
>> > > > >> > >> > > > > >
>> > > > >> > >> > > > > > Thanks,
>> > > > >> > >> > > > > >
>> > > > >> > >> > > > > > Jun
>> > > > >> > >> > > > > >
>> > > > >> > >> > > > > > On Wed, Dec 14, 2022 at 10:44 AM Justine Olshan
>> > > > >> > >> > > > > > <jo...@confluent.io.invalid> wrote:
>> > > > >> > >> > > > > >
>> > > > >> > >> > > > > > > Matthias — thanks again for taking time to look
>> a
>> > > this.
>> > > > >> You
>> > > > >> > >> said:
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > > > > My proposal was only focusing to avoid
>> dangling
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > > > transactions if records are added without
>> > registered
>> > > > >> > >> partition.
>> > > > >> > >> > --
>> > > > >> > >> > > > > Maybe
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > > > you can add a few more details to the KIP about
>> > this
>> > > > >> > scenario
>> > > > >> > >> for
>> > > > >> > >> > > > > better
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > > > documentation purpose?
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > > > I'm not sure I understand what you mean here.
>> The
>> > > > >> motivation
>> > > > >> > >> > > section
>> > > > >> > >> > > > > > > describes two scenarios about how the record
>> can be
>> > > > added
>> > > > >> > >> > without a
>> > > > >> > >> > > > > > > registered partition:
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > > > > 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.
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > > > > Another way hanging transactions can occur is
>> > that
>> > > a
>> > > > >> > client
>> > > > >> > >> is
>> > > > >> > >> > > > buggy
>> > > > >> > >> > > > > > and
>> > > > >> > >> > > > > > > may somehow try to write to a partition before
>> it
>> > > adds
>> > > > >> the
>> > > > >> > >> > > partition
>> > > > >> > >> > > > to
>> > > > >> > >> > > > > > the
>> > > > >> > >> > > > > > > transaction.
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > > > For the first example of this would it be
>> helpful
>> > to
>> > > > say
>> > > > >> > that
>> > > > >> > >> > this
>> > > > >> > >> > > > > > message
>> > > > >> > >> > > > > > > comes in after the abort, but before the
>> partition
>> > is
>> > > > >> added
>> > > > >> > to
>> > > > >> > >> > the
>> > > > >> > >> > > > next
>> > > > >> > >> > > > > > > transaction so it becomes "hanging." Perhaps the
>> > next
>> > > > >> > sentence
>> > > > >> > >> > > > > describing
>> > > > >> > >> > > > > > > the message becoming part of the next
>> transaction
>> > (a
>> > > > >> > different
>> > > > >> > >> > > case)
>> > > > >> > >> > > > > was
>> > > > >> > >> > > > > > > not properly differentiated.
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > > > Jun — thanks for reading the KIP.
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > > > 70. The int typing was a concern. Currently we
>> > have a
>> > > > >> > >> mechanism
>> > > > >> > >> > in
>> > > > >> > >> > > > > place
>> > > > >> > >> > > > > > to
>> > > > >> > >> > > > > > > fence the final epoch when the epoch is about to
>> > > > overflow
>> > > > >> > and
>> > > > >> > >> > > assign
>> > > > >> > >> > > > a
>> > > > >> > >> > > > > > new
>> > > > >> > >> > > > > > > producer ID with epoch 0. Of course, this is a
>> bit
>> > > > tricky
>> > > > >> > >> when it
>> > > > >> > >> > > > comes
>> > > > >> > >> > > > > > to
>> > > > >> > >> > > > > > > the response back to the client.
>> > > > >> > >> > > > > > > Making this a long could be another option, but
>> I
>> > > > wonder
>> > > > >> are
>> > > > >> > >> > there
>> > > > >> > >> > > > any
>> > > > >> > >> > > > > > > implications on changing this field if the
>> epoch is
>> > > > >> > persisted
>> > > > >> > >> to
>> > > > >> > >> > > > disk?
>> > > > >> > >> > > > > > I'd
>> > > > >> > >> > > > > > > need to check the usages.
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > > > 71.This was something Matthias asked about as
>> > well. I
>> > > > was
>> > > > >> > >> > > > considering a
>> > > > >> > >> > > > > > > possible edge case where a produce request from
>> a
>> > new
>> > > > >> > >> transaction
>> > > > >> > >> > > > > somehow
>> > > > >> > >> > > > > > > gets sent right after the marker is written, but
>> > > before
>> > > > >> the
>> > > > >> > >> > > producer
>> > > > >> > >> > > > is
>> > > > >> > >> > > > > > > alerted of the newly bumped epoch. In this
>> case, we
>> > > may
>> > > > >> > >> include
>> > > > >> > >> > > this
>> > > > >> > >> > > > > > record
>> > > > >> > >> > > > > > > when we don't want to. I suppose we could try
>> to do
>> > > > >> > something
>> > > > >> > >> > > client
>> > > > >> > >> > > > > side
>> > > > >> > >> > > > > > > to bump the epoch after sending an endTxn as
>> well
>> > in
>> > > > this
>> > > > >> > >> > scenario
>> > > > >> > >> > > —
>> > > > >> > >> > > > > but
>> > > > >> > >> > > > > > I
>> > > > >> > >> > > > > > > wonder how it would work when the server is
>> > aborting
>> > > > >> based
>> > > > >> > on
>> > > > >> > >> a
>> > > > >> > >> > > > > > server-side
>> > > > >> > >> > > > > > > error. I could also be missing something and
>> this
>> > > > >> scenario
>> > > > >> > is
>> > > > >> > >> > > > actually
>> > > > >> > >> > > > > > not
>> > > > >> > >> > > > > > > possible.
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > > > Thanks again to everyone reading and commenting.
>> > Let
>> > > me
>> > > > >> know
>> > > > >> > >> > about
>> > > > >> > >> > > > any
>> > > > >> > >> > > > > > > further questions or comments.
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > > > Justine
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > > > On Wed, Dec 14, 2022 at 9:41 AM Jun Rao
>> > > > >> > >> <jun@confluent.io.invalid
>> > > > >> > >> > >
>> > > > >> > >> > > > > > wrote:
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > > > > Hi, Justine,
>> > > > >> > >> > > > > > > >
>> > > > >> > >> > > > > > > > Thanks for the KIP. A couple of comments.
>> > > > >> > >> > > > > > > >
>> > > > >> > >> > > > > > > > 70. Currently, the producer epoch is an int.
>> I am
>> > > not
>> > > > >> sure
>> > > > >> > >> if
>> > > > >> > >> > > it's
>> > > > >> > >> > > > > > enough
>> > > > >> > >> > > > > > > > to accommodate all transactions in the
>> lifetime
>> > of
>> > > a
>> > > > >> > >> producer.
>> > > > >> > >> > > > Should
>> > > > >> > >> > > > > > we
>> > > > >> > >> > > > > > > > change that to a long or add a new long field
>> > like
>> > > > >> txnId?
>> > > > >> > >> > > > > > > >
>> > > > >> > >> > > > > > > > 71. "it will write the prepare commit message
>> > with
>> > > a
>> > > > >> > bumped
>> > > > >> > >> > epoch
>> > > > >> > >> > > > and
>> > > > >> > >> > > > > > > send
>> > > > >> > >> > > > > > > > WriteTxnMarkerRequests with the bumped epoch."
>> > Hmm,
>> > > > the
>> > > > >> > >> epoch
>> > > > >> > >> > is
>> > > > >> > >> > > > > > > associated
>> > > > >> > >> > > > > > > > with the current txn right? So, it seems
>> weird to
>> > > > >> write a
>> > > > >> > >> > commit
>> > > > >> > >> > > > > > message
>> > > > >> > >> > > > > > > > with a bumped epoch. Should we only bump up
>> the
>> > > epoch
>> > > > >> in
>> > > > >> > >> > > > > EndTxnResponse
>> > > > >> > >> > > > > > > and
>> > > > >> > >> > > > > > > > rename the field to sth like
>> nextProducerEpoch?
>> > > > >> > >> > > > > > > >
>> > > > >> > >> > > > > > > > Thanks,
>> > > > >> > >> > > > > > > >
>> > > > >> > >> > > > > > > > Jun
>> > > > >> > >> > > > > > > >
>> > > > >> > >> > > > > > > >
>> > > > >> > >> > > > > > > >
>> > > > >> > >> > > > > > > > On Mon, Dec 12, 2022 at 8:54 PM Matthias J.
>> Sax <
>> > > > >> > >> > > mjsax@apache.org>
>> > > > >> > >> > > > > > > wrote:
>> > > > >> > >> > > > > > > >
>> > > > >> > >> > > > > > > > > Thanks for the background.
>> > > > >> > >> > > > > > > > >
>> > > > >> > >> > > > > > > > > 20/30: SGTM. My proposal was only focusing
>> to
>> > > avoid
>> > > > >> > >> dangling
>> > > > >> > >> > > > > > > > > transactions if records are added without
>> > > > registered
>> > > > >> > >> > partition.
>> > > > >> > >> > > > --
>> > > > >> > >> > > > > > > Maybe
>> > > > >> > >> > > > > > > > > you can add a few more details to the KIP
>> about
>> > > > this
>> > > > >> > >> scenario
>> > > > >> > >> > > for
>> > > > >> > >> > > > > > > better
>> > > > >> > >> > > > > > > > > documentation purpose?
>> > > > >> > >> > > > > > > > >
>> > > > >> > >> > > > > > > > > 40: I think you hit a fair point about race
>> > > > >> conditions
>> > > > >> > or
>> > > > >> > >> > > client
>> > > > >> > >> > > > > bugs
>> > > > >> > >> > > > > > > > > (incorrectly not bumping the epoch). The
>> > > > >> > >> complexity/confusion
>> > > > >> > >> > > for
>> > > > >> > >> > > > > > using
>> > > > >> > >> > > > > > > > > the bumped epoch I see, is mainly for
>> internal
>> > > > >> > debugging,
>> > > > >> > >> ie,
>> > > > >> > >> > > > > > > inspecting
>> > > > >> > >> > > > > > > > > log segment dumps -- it seems harder to
>> reason
>> > > > about
>> > > > >> the
>> > > > >> > >> > system
>> > > > >> > >> > > > for
>> > > > >> > >> > > > > > us
>> > > > >> > >> > > > > > > > > humans. But if we get better guarantees, it
>> > would
>> > > > be
>> > > > >> > >> worth to
>> > > > >> > >> > > use
>> > > > >> > >> > > > > the
>> > > > >> > >> > > > > > > > > bumped epoch.
>> > > > >> > >> > > > > > > > >
>> > > > >> > >> > > > > > > > > 60: as I mentioned already, I don't know the
>> > > broker
>> > > > >> > >> internals
>> > > > >> > >> > > to
>> > > > >> > >> > > > > > > provide
>> > > > >> > >> > > > > > > > > more input. So if nobody else chimes in, we
>> > > should
>> > > > >> just
>> > > > >> > >> move
>> > > > >> > >> > > > > forward
>> > > > >> > >> > > > > > > > > with your proposal.
>> > > > >> > >> > > > > > > > >
>> > > > >> > >> > > > > > > > >
>> > > > >> > >> > > > > > > > > -Matthias
>> > > > >> > >> > > > > > > > >
>> > > > >> > >> > > > > > > > >
>> > > > >> > >> > > > > > > > > On 12/6/22 4:22 PM, Justine Olshan wrote:
>> > > > >> > >> > > > > > > > > > Hi all,
>> > > > >> > >> > > > > > > > > > After Artem's questions about error
>> behavior,
>> > > > I've
>> > > > >> > >> > > re-evaluated
>> > > > >> > >> > > > > the
>> > > > >> > >> > > > > > > > > > unknown producer ID exception and had some
>> > > > >> discussions
>> > > > >> > >> > > offline.
>> > > > >> > >> > > > > > > > > >
>> > > > >> > >> > > > > > > > > > I think generally it makes sense to
>> simplify
>> > > > error
>> > > > >> > >> handling
>> > > > >> > >> > > in
>> > > > >> > >> > > > > > cases
>> > > > >> > >> > > > > > > > like
>> > > > >> > >> > > > > > > > > > this and the UNKNOWN_PRODUCER_ID error
>> has a
>> > > > pretty
>> > > > >> > long
>> > > > >> > >> > and
>> > > > >> > >> > > > > > > > complicated
>> > > > >> > >> > > > > > > > > > history. Because of this, I propose
>> adding a
>> > > new
>> > > > >> error
>> > > > >> > >> code
>> > > > >> > >> > > > > > > > > ABORTABLE_ERROR
>> > > > >> > >> > > > > > > > > > that when encountered by new clients
>> (gated
>> > by
>> > > > the
>> > > > >> > >> produce
>> > > > >> > >> > > > > request
>> > > > >> > >> > > > > > > > > version)
>> > > > >> > >> > > > > > > > > > will simply abort the transaction. This
>> > allows
>> > > > the
>> > > > >> > >> server
>> > > > >> > >> > to
>> > > > >> > >> > > > have
>> > > > >> > >> > > > > > > some
>> > > > >> > >> > > > > > > > > say
>> > > > >> > >> > > > > > > > > > in whether the client aborts and makes
>> > handling
>> > > > >> much
>> > > > >> > >> > simpler.
>> > > > >> > >> > > > In
>> > > > >> > >> > > > > > the
>> > > > >> > >> > > > > > > > > > future, we can also use this error in
>> other
>> > > > >> situations
>> > > > >> > >> > where
>> > > > >> > >> > > we
>> > > > >> > >> > > > > > want
>> > > > >> > >> > > > > > > to
>> > > > >> > >> > > > > > > > > > abort the transactions. We can even use on
>> > > other
>> > > > >> apis.
>> > > > >> > >> > > > > > > > > >
>> > > > >> > >> > > > > > > > > > I've added this to the KIP. Let me know if
>> > > there
>> > > > >> are
>> > > > >> > any
>> > > > >> > >> > > > > questions
>> > > > >> > >> > > > > > or
>> > > > >> > >> > > > > > > > > > issues.
>> > > > >> > >> > > > > > > > > >
>> > > > >> > >> > > > > > > > > > Justine
>> > > > >> > >> > > > > > > > > >
>> > > > >> > >> > > > > > > > > > On Fri, Dec 2, 2022 at 10:22 AM Justine
>> > Olshan
>> > > <
>> > > > >> > >> > > > > > jolshan@confluent.io
>> > > > >> > >> > > > > > > >
>> > > > >> > >> > > > > > > > > wrote:
>> > > > >> > >> > > > > > > > > >
>> > > > >> > >> > > > > > > > > >> Hey Matthias,
>> > > > >> > >> > > > > > > > > >>
>> > > > >> > >> > > > > > > > > >>
>> > > > >> > >> > > > > > > > > >> 20/30 — Maybe I also didn't express
>> myself
>> > > > >> clearly.
>> > > > >> > For
>> > > > >> > >> > > older
>> > > > >> > >> > > > > > > clients
>> > > > >> > >> > > > > > > > we
>> > > > >> > >> > > > > > > > > >> don't have a way to distinguish between a
>> > > > previous
>> > > > >> > and
>> > > > >> > >> the
>> > > > >> > >> > > > > current
>> > > > >> > >> > > > > > > > > >> transaction since we don't have the epoch
>> > > bump.
>> > > > >> This
>> > > > >> > >> means
>> > > > >> > >> > > > that
>> > > > >> > >> > > > > a
>> > > > >> > >> > > > > > > late
>> > > > >> > >> > > > > > > > > >> message from the previous transaction
>> may be
>> > > > >> added to
>> > > > >> > >> the
>> > > > >> > >> > > new
>> > > > >> > >> > > > > one.
>> > > > >> > >> > > > > > > > With
>> > > > >> > >> > > > > > > > > >> older clients — we can't guarantee this
>> > won't
>> > > > >> happen
>> > > > >> > >> if we
>> > > > >> > >> > > > > already
>> > > > >> > >> > > > > > > > sent
>> > > > >> > >> > > > > > > > > the
>> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call (why we make
>> changes
>> > > for
>> > > > >> the
>> > > > >> > >> newer
>> > > > >> > >> > > > > client)
>> > > > >> > >> > > > > > > but
>> > > > >> > >> > > > > > > > > we
>> > > > >> > >> > > > > > > > > >> can at least gate some by ensuring that
>> the
>> > > > >> partition
>> > > > >> > >> has
>> > > > >> > >> > > been
>> > > > >> > >> > > > > > added
>> > > > >> > >> > > > > > > > to
>> > > > >> > >> > > > > > > > > the
>> > > > >> > >> > > > > > > > > >> transaction. The rationale here is that
>> > there
>> > > > are
>> > > > >> > >> likely
>> > > > >> > >> > > LESS
>> > > > >> > >> > > > > late
>> > > > >> > >> > > > > > > > > arrivals
>> > > > >> > >> > > > > > > > > >> as time goes on, so hopefully most late
>> > > arrivals
>> > > > >> will
>> > > > >> > >> come
>> > > > >> > >> > > in
>> > > > >> > >> > > > > > BEFORE
>> > > > >> > >> > > > > > > > the
>> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call. Those that
>> arrive
>> > > > before
>> > > > >> > will
>> > > > >> > >> be
>> > > > >> > >> > > > > properly
>> > > > >> > >> > > > > > > > gated
>> > > > >> > >> > > > > > > > > >> with the describeTransactions approach.
>> > > > >> > >> > > > > > > > > >>
>> > > > >> > >> > > > > > > > > >> If we take the approach you suggested,
>> ANY
>> > > late
>> > > > >> > arrival
>> > > > >> > >> > > from a
>> > > > >> > >> > > > > > > > previous
>> > > > >> > >> > > > > > > > > >> transaction will be added. And we don't
>> want
>> > > > >> that. I
>> > > > >> > >> also
>> > > > >> > >> > > > don't
>> > > > >> > >> > > > > > see
>> > > > >> > >> > > > > > > > any
>> > > > >> > >> > > > > > > > > >> benefit in sending addPartitionsToTxn
>> over
>> > the
>> > > > >> > >> > describeTxns
>> > > > >> > >> > > > > call.
>> > > > >> > >> > > > > > > They
>> > > > >> > >> > > > > > > > > will
>> > > > >> > >> > > > > > > > > >> both be one extra RPC to the Txn
>> > coordinator.
>> > > > >> > >> > > > > > > > > >>
>> > > > >> > >> > > > > > > > > >>
>> > > > >> > >> > > > > > > > > >> To be clear — newer clients will use
>> > > > >> > addPartitionsToTxn
>> > > > >> > >> > > > instead
>> > > > >> > >> > > > > of
>> > > > >> > >> > > > > > > the
>> > > > >> > >> > > > > > > > > >> DescribeTxns.
>> > > > >> > >> > > > > > > > > >>
>> > > > >> > >> > > > > > > > > >>
>> > > > >> > >> > > > > > > > > >> 40)
>> > > > >> > >> > > > > > > > > >> My concern is that if we have some delay
>> in
>> > > the
>> > > > >> > client
>> > > > >> > >> to
>> > > > >> > >> > > bump
>> > > > >> > >> > > > > the
>> > > > >> > >> > > > > > > > > epoch,
>> > > > >> > >> > > > > > > > > >> it could continue to send epoch 73 and
>> those
>> > > > >> records
>> > > > >> > >> would
>> > > > >> > >> > > not
>> > > > >> > >> > > > > be
>> > > > >> > >> > > > > > > > > fenced.
>> > > > >> > >> > > > > > > > > >> Perhaps this is not an issue if we don't
>> > allow
>> > > > the
>> > > > >> > next
>> > > > >> > >> > > > produce
>> > > > >> > >> > > > > to
>> > > > >> > >> > > > > > > go
>> > > > >> > >> > > > > > > > > >> through before the EndTxn request
>> returns.
>> > I'm
>> > > > >> also
>> > > > >> > >> > thinking
>> > > > >> > >> > > > > about
>> > > > >> > >> > > > > > > > > cases of
>> > > > >> > >> > > > > > > > > >> failure. I will need to think on this a
>> bit.
>> > > > >> > >> > > > > > > > > >>
>> > > > >> > >> > > > > > > > > >> I wasn't sure if it was that confusing.
>> But
>> > if
>> > > > we
>> > > > >> > >> think it
>> > > > >> > >> > > is,
>> > > > >> > >> > > > > we
>> > > > >> > >> > > > > > > can
>> > > > >> > >> > > > > > > > > >> investigate other ways.
>> > > > >> > >> > > > > > > > > >>
>> > > > >> > >> > > > > > > > > >>
>> > > > >> > >> > > > > > > > > >> 60)
>> > > > >> > >> > > > > > > > > >>
>> > > > >> > >> > > > > > > > > >> I'm not sure these are the same
>> purgatories
>> > > > since
>> > > > >> one
>> > > > >> > >> is a
>> > > > >> > >> > > > > produce
>> > > > >> > >> > > > > > > > > >> purgatory (I was planning on using a
>> > callback
>> > > > >> rather
>> > > > >> > >> than
>> > > > >> > >> > > > > > purgatory)
>> > > > >> > >> > > > > > > > and
>> > > > >> > >> > > > > > > > > >> the other is simply a request to append
>> to
>> > the
>> > > > >> log.
>> > > > >> > Not
>> > > > >> > >> > sure
>> > > > >> > >> > > > we
>> > > > >> > >> > > > > > have
>> > > > >> > >> > > > > > > > any
>> > > > >> > >> > > > > > > > > >> structure here for ordering, but my
>> > > > understanding
>> > > > >> is
>> > > > >> > >> that
>> > > > >> > >> > > the
>> > > > >> > >> > > > > > broker
>> > > > >> > >> > > > > > > > > could
>> > > > >> > >> > > > > > > > > >> handle the write request before it hears
>> > back
>> > > > from
>> > > > >> > the
>> > > > >> > >> Txn
>> > > > >> > >> > > > > > > > Coordinator.
>> > > > >> > >> > > > > > > > > >>
>> > > > >> > >> > > > > > > > > >> Let me know if I misunderstood something
>> or
>> > > > >> something
>> > > > >> > >> was
>> > > > >> > >> > > > > unclear.
>> > > > >> > >> > > > > > > > > >>
>> > > > >> > >> > > > > > > > > >> Justine
>> > > > >> > >> > > > > > > > > >>
>> > > > >> > >> > > > > > > > > >> On Thu, Dec 1, 2022 at 12:15 PM Matthias
>> J.
>> > > Sax
>> > > > <
>> > > > >> > >> > > > > mjsax@apache.org
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > > > > > wrote:
>> > > > >> > >> > > > > > > > > >>
>> > > > >> > >> > > > > > > > > >>> Thanks for the details Justine!
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > > >>>> 20)
>> > > > >> > >> > > > > > > > > >>>>
>> > > > >> > >> > > > > > > > > >>>> The client side change for 2 is
>> removing
>> > the
>> > > > >> > >> > addPartitions
>> > > > >> > >> > > > to
>> > > > >> > >> > > > > > > > > >>> transaction
>> > > > >> > >> > > > > > > > > >>>> call. We don't need to make this from
>> the
>> > > > >> producer
>> > > > >> > to
>> > > > >> > >> > the
>> > > > >> > >> > > > txn
>> > > > >> > >> > > > > > > > > >>> coordinator,
>> > > > >> > >> > > > > > > > > >>>> only server side.
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > > >>> I think I did not express myself
>> clearly. I
>> > > > >> > understand
>> > > > >> > >> > that
>> > > > >> > >> > > > we
>> > > > >> > >> > > > > > can
>> > > > >> > >> > > > > > > > (and
>> > > > >> > >> > > > > > > > > >>> should) change the producer to not send
>> the
>> > > > >> > >> > `addPartitions`
>> > > > >> > >> > > > > > request
>> > > > >> > >> > > > > > > > any
>> > > > >> > >> > > > > > > > > >>> longer. But I don't thinks it's
>> requirement
>> > > to
>> > > > >> > change
>> > > > >> > >> the
>> > > > >> > >> > > > > broker?
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > > >>> What I am trying to say is: as a
>> safe-guard
>> > > and
>> > > > >> > >> > improvement
>> > > > >> > >> > > > for
>> > > > >> > >> > > > > > > older
>> > > > >> > >> > > > > > > > > >>> producers, the partition leader can just
>> > send
>> > > > the
>> > > > >> > >> > > > > `addPartitions`
>> > > > >> > >> > > > > > > > > >>> request to the TX-coordinator in any
>> case
>> > --
>> > > if
>> > > > >> the
>> > > > >> > >> old
>> > > > >> > >> > > > > producer
>> > > > >> > >> > > > > > > > > >>> correctly did send the `addPartition`
>> > request
>> > > > to
>> > > > >> the
>> > > > >> > >> > > > > > TX-coordinator
>> > > > >> > >> > > > > > > > > >>> already, the TX-coordinator can just
>> > "ignore"
>> > > > is
>> > > > >> as
>> > > > >> > >> > > > idempotent.
>> > > > >> > >> > > > > > > > > However,
>> > > > >> > >> > > > > > > > > >>> if the old producer has a bug and did
>> > forget
>> > > to
>> > > > >> sent
>> > > > >> > >> the
>> > > > >> > >> > > > > > > > `addPartition`
>> > > > >> > >> > > > > > > > > >>> request, we would now ensure that the
>> > > partition
>> > > > >> is
>> > > > >> > >> indeed
>> > > > >> > >> > > > added
>> > > > >> > >> > > > > > to
>> > > > >> > >> > > > > > > > the
>> > > > >> > >> > > > > > > > > >>> TX and thus fix a potential producer bug
>> > > (even
>> > > > >> if we
>> > > > >> > >> > don't
>> > > > >> > >> > > > get
>> > > > >> > >> > > > > > the
>> > > > >> > >> > > > > > > > > >>> fencing via the bump epoch). -- It
>> seems to
>> > > be
>> > > > a
>> > > > >> > good
>> > > > >> > >> > > > > > improvement?
>> > > > >> > >> > > > > > > Or
>> > > > >> > >> > > > > > > > > is
>> > > > >> > >> > > > > > > > > >>> there a reason to not do this?
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > > >>>> 30)
>> > > > >> > >> > > > > > > > > >>>>
>> > > > >> > >> > > > > > > > > >>>> Transaction is ongoing = partition was
>> > added
>> > > > to
>> > > > >> > >> > > transaction
>> > > > >> > >> > > > > via
>> > > > >> > >> > > > > > > > > >>>> addPartitionsToTxn. We check this with
>> the
>> > > > >> > >> > > > > DescribeTransactions
>> > > > >> > >> > > > > > > > call.
>> > > > >> > >> > > > > > > > > >>> Let
>> > > > >> > >> > > > > > > > > >>>> me know if this wasn't sufficiently
>> > > explained
>> > > > >> here:
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > > >>> If we do what I propose in (20), we
>> don't
>> > > > really
>> > > > >> > need
>> > > > >> > >> to
>> > > > >> > >> > > make
>> > > > >> > >> > > > > > this
>> > > > >> > >> > > > > > > > > >>> `DescribeTransaction` call, as the
>> > partition
>> > > > >> leader
>> > > > >> > >> adds
>> > > > >> > >> > > the
>> > > > >> > >> > > > > > > > partition
>> > > > >> > >> > > > > > > > > >>> for older clients and we get this check
>> for
>> > > > free.
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > > >>>> 40)
>> > > > >> > >> > > > > > > > > >>>>
>> > > > >> > >> > > > > > > > > >>>> The idea here is that if any messages
>> > > somehow
>> > > > >> come
>> > > > >> > in
>> > > > >> > >> > > before
>> > > > >> > >> > > > > we
>> > > > >> > >> > > > > > > get
>> > > > >> > >> > > > > > > > > the
>> > > > >> > >> > > > > > > > > >>> new
>> > > > >> > >> > > > > > > > > >>>> epoch to the producer, they will be
>> > fenced.
>> > > > >> > However,
>> > > > >> > >> if
>> > > > >> > >> > we
>> > > > >> > >> > > > > don't
>> > > > >> > >> > > > > > > > think
>> > > > >> > >> > > > > > > > > >>> this
>> > > > >> > >> > > > > > > > > >>>> is necessary, it can be discussed
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > > >>> I agree that we should have epoch
>> fencing.
>> > My
>> > > > >> > >> question is
>> > > > >> > >> > > > > > > different:
>> > > > >> > >> > > > > > > > > >>> Assume we are at epoch 73, and we have
>> an
>> > > > ongoing
>> > > > >> > >> > > > transaction,
>> > > > >> > >> > > > > > that
>> > > > >> > >> > > > > > > > is
>> > > > >> > >> > > > > > > > > >>> committed. It seems natural to write the
>> > > > "prepare
>> > > > >> > >> commit"
>> > > > >> > >> > > > > marker
>> > > > >> > >> > > > > > > and
>> > > > >> > >> > > > > > > > > the
>> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` both with epoch
>> 73,
>> > > too,
>> > > > >> as
>> > > > >> > it
>> > > > >> > >> > > belongs
>> > > > >> > >> > > > > to
>> > > > >> > >> > > > > > > the
>> > > > >> > >> > > > > > > > > >>> current transaction. Of course, we now
>> also
>> > > > bump
>> > > > >> the
>> > > > >> > >> > epoch
>> > > > >> > >> > > > and
>> > > > >> > >> > > > > > > expect
>> > > > >> > >> > > > > > > > > >>> the next requests to have epoch 74, and
>> > would
>> > > > >> reject
>> > > > >> > >> an
>> > > > >> > >> > > > request
>> > > > >> > >> > > > > > > with
>> > > > >> > >> > > > > > > > > >>> epoch 73, as the corresponding TX for
>> epoch
>> > > 73
>> > > > >> was
>> > > > >> > >> > already
>> > > > >> > >> > > > > > > committed.
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > > >>> It seems you propose to write the
>> "prepare
>> > > > commit
>> > > > >> > >> marker"
>> > > > >> > >> > > and
>> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` with epoch 74
>> > though,
>> > > > what
>> > > > >> > >> would
>> > > > >> > >> > > work,
>> > > > >> > >> > > > > but
>> > > > >> > >> > > > > > > it
>> > > > >> > >> > > > > > > > > >>> seems confusing. Is there a reason why
>> we
>> > > would
>> > > > >> use
>> > > > >> > >> the
>> > > > >> > >> > > > bumped
>> > > > >> > >> > > > > > > epoch
>> > > > >> > >> > > > > > > > 74
>> > > > >> > >> > > > > > > > > >>> instead of the current epoch 73?
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > > >>>> 60)
>> > > > >> > >> > > > > > > > > >>>>
>> > > > >> > >> > > > > > > > > >>>> When we are checking if the
>> transaction is
>> > > > >> ongoing,
>> > > > >> > >> we
>> > > > >> > >> > > need
>> > > > >> > >> > > > to
>> > > > >> > >> > > > > > > make
>> > > > >> > >> > > > > > > > a
>> > > > >> > >> > > > > > > > > >>> round
>> > > > >> > >> > > > > > > > > >>>> trip from the leader partition to the
>> > > > >> transaction
>> > > > >> > >> > > > coordinator.
>> > > > >> > >> > > > > > In
>> > > > >> > >> > > > > > > > the
>> > > > >> > >> > > > > > > > > >>> time
>> > > > >> > >> > > > > > > > > >>>> we are waiting for this message to come
>> > > back,
>> > > > in
>> > > > >> > >> theory
>> > > > >> > >> > we
>> > > > >> > >> > > > > could
>> > > > >> > >> > > > > > > > have
>> > > > >> > >> > > > > > > > > >>> sent
>> > > > >> > >> > > > > > > > > >>>> a commit/abort call that would make the
>> > > > original
>> > > > >> > >> result
>> > > > >> > >> > of
>> > > > >> > >> > > > the
>> > > > >> > >> > > > > > > check
>> > > > >> > >> > > > > > > > > >>> out of
>> > > > >> > >> > > > > > > > > >>>> date. That is why we can check the
>> leader
>> > > > state
>> > > > >> > >> before
>> > > > >> > >> > we
>> > > > >> > >> > > > > write
>> > > > >> > >> > > > > > to
>> > > > >> > >> > > > > > > > the
>> > > > >> > >> > > > > > > > > >>> log.
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > > >>> Thanks. Got it.
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > > >>> However, is this really an issue? We put
>> > the
>> > > > >> produce
>> > > > >> > >> > > request
>> > > > >> > >> > > > in
>> > > > >> > >> > > > > > > > > >>> purgatory, so how could we process the
>> > > > >> > >> > > > `WriteTxnMarkerRequest`
>> > > > >> > >> > > > > > > first?
>> > > > >> > >> > > > > > > > > >>> Don't we need to put the
>> > > > `WriteTxnMarkerRequest`
>> > > > >> > into
>> > > > >> > >> > > > > purgatory,
>> > > > >> > >> > > > > > > too,
>> > > > >> > >> > > > > > > > > >>> for this case, and process both request
>> > > > in-order?
>> > > > >> > >> (Again,
>> > > > >> > >> > > my
>> > > > >> > >> > > > > > broker
>> > > > >> > >> > > > > > > > > >>> knowledge is limited and maybe we don't
>> > > > maintain
>> > > > >> > >> request
>> > > > >> > >> > > > order
>> > > > >> > >> > > > > > for
>> > > > >> > >> > > > > > > > this
>> > > > >> > >> > > > > > > > > >>> case, what seems to be an issue IMHO,
>> and I
>> > > am
>> > > > >> > >> wondering
>> > > > >> > >> > if
>> > > > >> > >> > > > > > > changing
>> > > > >> > >> > > > > > > > > >>> request handling to preserve order for
>> this
>> > > > case
>> > > > >> > >> might be
>> > > > >> > >> > > the
>> > > > >> > >> > > > > > > cleaner
>> > > > >> > >> > > > > > > > > >>> solution?)
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > > >>> -Matthias
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > > >>> On 11/30/22 3:28 PM, Artem Livshits
>> wrote:
>> > > > >> > >> > > > > > > > > >>>> Hi Justine,
>> > > > >> > >> > > > > > > > > >>>>
>> > > > >> > >> > > > > > > > > >>>> I think the interesting part is not in
>> > this
>> > > > >> logic
>> > > > >> > >> > (because
>> > > > >> > >> > > > it
>> > > > >> > >> > > > > > > tries
>> > > > >> > >> > > > > > > > to
>> > > > >> > >> > > > > > > > > >>>> figure out when UNKNOWN_PRODUCER_ID is
>> > > > retriable
>> > > > >> > and
>> > > > >> > >> if
>> > > > >> > >> > > it's
>> > > > >> > >> > > > > > > > > retryable,
>> > > > >> > >> > > > > > > > > >>>> it's definitely not fatal), but what
>> > happens
>> > > > >> when
>> > > > >> > >> this
>> > > > >> > >> > > logic
>> > > > >> > >> > > > > > > doesn't
>> > > > >> > >> > > > > > > > > >>> return
>> > > > >> > >> > > > > > > > > >>>> 'true' and falls through.  In the old
>> > > clients
>> > > > it
>> > > > >> > >> seems
>> > > > >> > >> > to
>> > > > >> > >> > > be
>> > > > >> > >> > > > > > > fatal,
>> > > > >> > >> > > > > > > > if
>> > > > >> > >> > > > > > > > > >>> we
>> > > > >> > >> > > > > > > > > >>>> keep the behavior in the new clients,
>> I'd
>> > > > >> expect it
>> > > > >> > >> > would
>> > > > >> > >> > > be
>> > > > >> > >> > > > > > fatal
>> > > > >> > >> > > > > > > > as
>> > > > >> > >> > > > > > > > > >>> well.
>> > > > >> > >> > > > > > > > > >>>>
>> > > > >> > >> > > > > > > > > >>>> -Artem
>> > > > >> > >> > > > > > > > > >>>>
>> > > > >> > >> > > > > > > > > >>>> On Tue, Nov 29, 2022 at 11:57 AM
>> Justine
>> > > > Olshan
>> > > > >> > >> > > > > > > > > >>>> <jo...@confluent.io.invalid> wrote:
>> > > > >> > >> > > > > > > > > >>>>
>> > > > >> > >> > > > > > > > > >>>>> Hi Artem and Jeff,
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>> Thanks for taking a look and sorry for
>> > the
>> > > > slow
>> > > > >> > >> > response.
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>> You both mentioned the change to
>> handle
>> > > > >> > >> > > UNKNOWN_PRODUCER_ID
>> > > > >> > >> > > > > > > errors.
>> > > > >> > >> > > > > > > > > To
>> > > > >> > >> > > > > > > > > >>> be
>> > > > >> > >> > > > > > > > > >>>>> clear — this error code will only be
>> sent
>> > > > again
>> > > > >> > when
>> > > > >> > >> > the
>> > > > >> > >> > > > > > client's
>> > > > >> > >> > > > > > > > > >>> request
>> > > > >> > >> > > > > > > > > >>>>> version is high enough to ensure we
>> > handle
>> > > it
>> > > > >> > >> > correctly.
>> > > > >> > >> > > > > > > > > >>>>> The current (Java) client handles
>> this by
>> > > the
>> > > > >> > >> following
>> > > > >> > >> > > > > > (somewhat
>> > > > >> > >> > > > > > > > > long)
>> > > > >> > >> > > > > > > > > >>>>> code snippet:
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>> // An UNKNOWN_PRODUCER_ID means that
>> we
>> > > have
>> > > > >> lost
>> > > > >> > >> the
>> > > > >> > >> > > > > producer
>> > > > >> > >> > > > > > > > state
>> > > > >> > >> > > > > > > > > >>> on the
>> > > > >> > >> > > > > > > > > >>>>> broker. Depending on the log start
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>> // offset, we may want to retry
>> these, as
>> > > > >> > described
>> > > > >> > >> for
>> > > > >> > >> > > > each
>> > > > >> > >> > > > > > case
>> > > > >> > >> > > > > > > > > >>> below. If
>> > > > >> > >> > > > > > > > > >>>>> none of those apply, then for the
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>> // idempotent producer, we will
>> locally
>> > > bump
>> > > > >> the
>> > > > >> > >> epoch
>> > > > >> > >> > > and
>> > > > >> > >> > > > > > reset
>> > > > >> > >> > > > > > > > the
>> > > > >> > >> > > > > > > > > >>>>> sequence numbers of in-flight batches
>> > from
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>> // sequence 0, then retry the failed
>> > batch,
>> > > > >> which
>> > > > >> > >> > should
>> > > > >> > >> > > > now
>> > > > >> > >> > > > > > > > succeed.
>> > > > >> > >> > > > > > > > > >>> For
>> > > > >> > >> > > > > > > > > >>>>> the transactional producer, allow the
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>> // batch to fail. When processing the
>> > > failed
>> > > > >> > batch,
>> > > > >> > >> we
>> > > > >> > >> > > will
>> > > > >> > >> > > > > > > > > transition
>> > > > >> > >> > > > > > > > > >>> to
>> > > > >> > >> > > > > > > > > >>>>> an abortable error and set a flag
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>> // indicating that we need to bump the
>> > > epoch
>> > > > >> (if
>> > > > >> > >> > > supported
>> > > > >> > >> > > > by
>> > > > >> > >> > > > > > the
>> > > > >> > >> > > > > > > > > >>> broker).
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>> if (error ==
>> > Errors.*UNKNOWN_PRODUCER_ID*)
>> > > {
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>       if (response.logStartOffset ==
>> -1)
>> > {
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           // We don't know the log
>> start
>> > > > offset
>> > > > >> > with
>> > > > >> > >> > this
>> > > > >> > >> > > > > > > response.
>> > > > >> > >> > > > > > > > > We
>> > > > >> > >> > > > > > > > > >>> should
>> > > > >> > >> > > > > > > > > >>>>> just retry the request until we get
>> it.
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           // The UNKNOWN_PRODUCER_ID
>> > error
>> > > > code
>> > > > >> > was
>> > > > >> > >> > added
>> > > > >> > >> > > > > along
>> > > > >> > >> > > > > > > > with
>> > > > >> > >> > > > > > > > > >>> the new
>> > > > >> > >> > > > > > > > > >>>>> ProduceResponse which includes the
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           // logStartOffset. So the
>> '-1'
>> > > > >> sentinel
>> > > > >> > is
>> > > > >> > >> > not
>> > > > >> > >> > > > for
>> > > > >> > >> > > > > > > > backward
>> > > > >> > >> > > > > > > > > >>>>> compatibility. Instead, it is possible
>> > for
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           // a broker to not know the
>> > > > >> > >> logStartOffset at
>> > > > >> > >> > > > when
>> > > > >> > >> > > > > it
>> > > > >> > >> > > > > > > is
>> > > > >> > >> > > > > > > > > >>> returning
>> > > > >> > >> > > > > > > > > >>>>> the response because the partition
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           // may have moved away from
>> the
>> > > > >> broker
>> > > > >> > >> from
>> > > > >> > >> > the
>> > > > >> > >> > > > > time
>> > > > >> > >> > > > > > > the
>> > > > >> > >> > > > > > > > > >>> error was
>> > > > >> > >> > > > > > > > > >>>>> initially raised to the time the
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           // response was being
>> > > constructed.
>> > > > In
>> > > > >> > >> these
>> > > > >> > >> > > > cases,
>> > > > >> > >> > > > > we
>> > > > >> > >> > > > > > > > > should
>> > > > >> > >> > > > > > > > > >>> just
>> > > > >> > >> > > > > > > > > >>>>> retry the request: we are guaranteed
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           // to eventually get a
>> > > > logStartOffset
>> > > > >> > once
>> > > > >> > >> > > things
>> > > > >> > >> > > > > > > settle
>> > > > >> > >> > > > > > > > > down.
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           return true;
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>       }
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>       if
>> (batch.sequenceHasBeenReset()) {
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           // When the first inflight
>> > batch
>> > > > >> fails
>> > > > >> > >> due to
>> > > > >> > >> > > the
>> > > > >> > >> > > > > > > > > truncation
>> > > > >> > >> > > > > > > > > >>> case,
>> > > > >> > >> > > > > > > > > >>>>> then the sequences of all the other
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           // in flight batches would
>> have
>> > > > been
>> > > > >> > >> > restarted
>> > > > >> > >> > > > from
>> > > > >> > >> > > > > > the
>> > > > >> > >> > > > > > > > > >>> beginning.
>> > > > >> > >> > > > > > > > > >>>>> However, when those responses
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           // come back from the
>> broker,
>> > > they
>> > > > >> would
>> > > > >> > >> also
>> > > > >> > >> > > > come
>> > > > >> > >> > > > > > with
>> > > > >> > >> > > > > > > > an
>> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID error. In this
>> case,
>> > we
>> > > > >> should
>> > > > >> > >> not
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           // reset the sequence
>> numbers
>> > to
>> > > > the
>> > > > >> > >> > beginning.
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           return true;
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>       } else if
>> > > > >> > >> > > > > (lastAckedOffset(batch.topicPartition).orElse(
>> > > > >> > >> > > > > > > > > >>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
>> > > > >> > >> > > > response.logStartOffset) {
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           // The head of the log has
>> been
>> > > > >> removed,
>> > > > >> > >> > > probably
>> > > > >> > >> > > > > due
>> > > > >> > >> > > > > > > to
>> > > > >> > >> > > > > > > > > the
>> > > > >> > >> > > > > > > > > >>>>> retention time elapsing. In this case,
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           // we expect to lose the
>> > producer
>> > > > >> state.
>> > > > >> > >> For
>> > > > >> > >> > > the
>> > > > >> > >> > > > > > > > > transactional
>> > > > >> > >> > > > > > > > > >>>>> producer, reset the sequences of all
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           // inflight batches to be
>> from
>> > > the
>> > > > >> > >> beginning
>> > > > >> > >> > > and
>> > > > >> > >> > > > > > retry
>> > > > >> > >> > > > > > > > > them,
>> > > > >> > >> > > > > > > > > >>> so
>> > > > >> > >> > > > > > > > > >>>>> that the transaction does not need to
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           // be aborted. For the
>> > idempotent
>> > > > >> > >> producer,
>> > > > >> > >> > > bump
>> > > > >> > >> > > > > the
>> > > > >> > >> > > > > > > > epoch
>> > > > >> > >> > > > > > > > > to
>> > > > >> > >> > > > > > > > > >>> avoid
>> > > > >> > >> > > > > > > > > >>>>> reusing (sequence, epoch) pairs
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           if (isTransactional()) {
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > >
>> > > > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
>> > > > >> > >> > > > > > > > > >>>>> this.producerIdAndEpoch);
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           } else {
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > >
>> requestEpochBumpForPartition(batch.topicPartition);
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           }
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           return true;
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>       }
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>       if (!isTransactional()) {
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           // For the idempotent
>> producer,
>> > > > >> always
>> > > > >> > >> retry
>> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
>> > > > >> > >> > > > > > > > > >>>>> errors. If the batch has the current
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           // producer ID and epoch,
>> > > request a
>> > > > >> bump
>> > > > >> > >> of
>> > > > >> > >> > the
>> > > > >> > >> > > > > > epoch.
>> > > > >> > >> > > > > > > > > >>> Otherwise
>> > > > >> > >> > > > > > > > > >>>>> just retry the produce.
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > >  requestEpochBumpForPartition(batch.topicPartition);
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>           return true;
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>       }
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>> }
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>> I was considering keeping this
>> behavior —
>> > > but
>> > > > >> am
>> > > > >> > >> open
>> > > > >> > >> > to
>> > > > >> > >> > > > > > > > simplifying
>> > > > >> > >> > > > > > > > > >>> it.
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>> We are leaving changes to older
>> clients
>> > off
>> > > > the
>> > > > >> > >> table
>> > > > >> > >> > > here
>> > > > >> > >> > > > > > since
>> > > > >> > >> > > > > > > it
>> > > > >> > >> > > > > > > > > >>> caused
>> > > > >> > >> > > > > > > > > >>>>> many issues for clients in the past.
>> > > > Previously
>> > > > >> > this
>> > > > >> > >> > was
>> > > > >> > >> > > a
>> > > > >> > >> > > > > > fatal
>> > > > >> > >> > > > > > > > > error
>> > > > >> > >> > > > > > > > > >>> and
>> > > > >> > >> > > > > > > > > >>>>> we didn't have the mechanisms in
>> place to
>> > > > >> detect
>> > > > >> > >> when
>> > > > >> > >> > > this
>> > > > >> > >> > > > > was
>> > > > >> > >> > > > > > a
>> > > > >> > >> > > > > > > > > >>> legitimate
>> > > > >> > >> > > > > > > > > >>>>> case vs some bug or gap in the
>> protocol.
>> > > > >> Ensuring
>> > > > >> > >> each
>> > > > >> > >> > > > > > > transaction
>> > > > >> > >> > > > > > > > > has
>> > > > >> > >> > > > > > > > > >>> its
>> > > > >> > >> > > > > > > > > >>>>> own epoch should close this gap.
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>> And to address Jeff's second point:
>> > > > >> > >> > > > > > > > > >>>>> *does the typical produce request path
>> > > append
>> > > > >> > >> records
>> > > > >> > >> > to
>> > > > >> > >> > > > > local
>> > > > >> > >> > > > > > > log
>> > > > >> > >> > > > > > > > > >>> along*
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>> *with the currentTxnFirstOffset
>> > > information?
>> > > > I
>> > > > >> > would
>> > > > >> > >> > like
>> > > > >> > >> > > > to
>> > > > >> > >> > > > > > > > > >>> understand*
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>> *when the field is written to disk.*
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>> Yes, the first produce request
>> populates
>> > > this
>> > > > >> > field
>> > > > >> > >> and
>> > > > >> > >> > > > > writes
>> > > > >> > >> > > > > > > the
>> > > > >> > >> > > > > > > > > >>> offset
>> > > > >> > >> > > > > > > > > >>>>> as part of the record batch and also
>> to
>> > the
>> > > > >> > producer
>> > > > >> > >> > > state
>> > > > >> > >> > > > > > > > snapshot.
>> > > > >> > >> > > > > > > > > >>> When
>> > > > >> > >> > > > > > > > > >>>>> we reload the records on restart
>> and/or
>> > > > >> > >> reassignment,
>> > > > >> > >> > we
>> > > > >> > >> > > > > > > repopulate
>> > > > >> > >> > > > > > > > > >>> this
>> > > > >> > >> > > > > > > > > >>>>> field with the snapshot from disk
>> along
>> > > with
>> > > > >> the
>> > > > >> > >> rest
>> > > > >> > >> > of
>> > > > >> > >> > > > the
>> > > > >> > >> > > > > > > > producer
>> > > > >> > >> > > > > > > > > >>>>> state.
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>> Let me know if there are further
>> comments
>> > > > >> and/or
>> > > > >> > >> > > questions.
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>> Thanks,
>> > > > >> > >> > > > > > > > > >>>>> Justine
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>> On Tue, Nov 22, 2022 at 9:00 PM Jeff
>> Kim
>> > > > >> > >> > > > > > > > > <jeff.kim@confluent.io.invalid
>> > > > >> > >> > > > > > > > > >>>>
>> > > > >> > >> > > > > > > > > >>>>> wrote:
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>>> Hi Justine,
>> > > > >> > >> > > > > > > > > >>>>>>
>> > > > >> > >> > > > > > > > > >>>>>> Thanks for the KIP! I have two
>> > questions:
>> > > > >> > >> > > > > > > > > >>>>>>
>> > > > >> > >> > > > > > > > > >>>>>> 1) For new clients, we can once again
>> > > return
>> > > > >> an
>> > > > >> > >> error
>> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
>> > > > >> > >> > > > > > > > > >>>>>> for sequences
>> > > > >> > >> > > > > > > > > >>>>>> that are non-zero when there is no
>> > > producer
>> > > > >> state
>> > > > >> > >> > > present
>> > > > >> > >> > > > on
>> > > > >> > >> > > > > > the
>> > > > >> > >> > > > > > > > > >>> server.
>> > > > >> > >> > > > > > > > > >>>>>> This will indicate we missed the 0
>> > > sequence
>> > > > >> and
>> > > > >> > we
>> > > > >> > >> > don't
>> > > > >> > >> > > > yet
>> > > > >> > >> > > > > > > want
>> > > > >> > >> > > > > > > > to
>> > > > >> > >> > > > > > > > > >>>>> write
>> > > > >> > >> > > > > > > > > >>>>>> to the log.
>> > > > >> > >> > > > > > > > > >>>>>>
>> > > > >> > >> > > > > > > > > >>>>>> I would like to understand the
>> current
>> > > > >> behavior
>> > > > >> > to
>> > > > >> > >> > > handle
>> > > > >> > >> > > > > > older
>> > > > >> > >> > > > > > > > > >>> clients,
>> > > > >> > >> > > > > > > > > >>>>>> and if there are any changes we are
>> > > making.
>> > > > >> Maybe
>> > > > >> > >> I'm
>> > > > >> > >> > > > > missing
>> > > > >> > >> > > > > > > > > >>> something,
>> > > > >> > >> > > > > > > > > >>>>>> but we would want to identify
>> whether we
>> > > > >> missed
>> > > > >> > >> the 0
>> > > > >> > >> > > > > sequence
>> > > > >> > >> > > > > > > for
>> > > > >> > >> > > > > > > > > >>> older
>> > > > >> > >> > > > > > > > > >>>>>> clients, no?
>> > > > >> > >> > > > > > > > > >>>>>>
>> > > > >> > >> > > > > > > > > >>>>>> 2) Upon returning from the
>> transaction
>> > > > >> > >> coordinator, we
>> > > > >> > >> > > can
>> > > > >> > >> > > > > set
>> > > > >> > >> > > > > > > the
>> > > > >> > >> > > > > > > > > >>>>>> transaction
>> > > > >> > >> > > > > > > > > >>>>>> as ongoing on the leader by
>> populating
>> > > > >> > >> > > > currentTxnFirstOffset
>> > > > >> > >> > > > > > > > > >>>>>> through the typical produce request
>> > > > handling.
>> > > > >> > >> > > > > > > > > >>>>>>
>> > > > >> > >> > > > > > > > > >>>>>> does the typical produce request path
>> > > append
>> > > > >> > >> records
>> > > > >> > >> > to
>> > > > >> > >> > > > > local
>> > > > >> > >> > > > > > > log
>> > > > >> > >> > > > > > > > > >>> along
>> > > > >> > >> > > > > > > > > >>>>>> with the currentTxnFirstOffset
>> > > information?
>> > > > I
>> > > > >> > would
>> > > > >> > >> > like
>> > > > >> > >> > > > to
>> > > > >> > >> > > > > > > > > understand
>> > > > >> > >> > > > > > > > > >>>>>> when the field is written to disk.
>> > > > >> > >> > > > > > > > > >>>>>>
>> > > > >> > >> > > > > > > > > >>>>>> Thanks,
>> > > > >> > >> > > > > > > > > >>>>>> Jeff
>> > > > >> > >> > > > > > > > > >>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>
>> > > > >> > >> > > > > > > > > >>>>>> On Tue, Nov 22, 2022 at 4:44 PM Artem
>> > > > Livshits
>> > > > >> > >> > > > > > > > > >>>>>> <al...@confluent.io.invalid>
>> wrote:
>> > > > >> > >> > > > > > > > > >>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>> Hi Justine,
>> > > > >> > >> > > > > > > > > >>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>> Thank you for the KIP.  I have one
>> > > > question.
>> > > > >> > >> > > > > > > > > >>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>> 5) For new clients, we can once
>> again
>> > > > return
>> > > > >> an
>> > > > >> > >> error
>> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
>> > > > >> > >> > > > > > > > > >>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>> I believe we had problems in the
>> past
>> > > with
>> > > > >> > >> returning
>> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
>> > > > >> > >> > > > > > > > > >>>>>>> because it was considered fatal and
>> > > > required
>> > > > >> > >> client
>> > > > >> > >> > > > > restart.
>> > > > >> > >> > > > > > > It
>> > > > >> > >> > > > > > > > > >>> would
>> > > > >> > >> > > > > > > > > >>>>> be
>> > > > >> > >> > > > > > > > > >>>>>>> good to spell out the new client
>> > behavior
>> > > > >> when
>> > > > >> > it
>> > > > >> > >> > > > receives
>> > > > >> > >> > > > > > the
>> > > > >> > >> > > > > > > > > error.
>> > > > >> > >> > > > > > > > > >>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>> -Artem
>> > > > >> > >> > > > > > > > > >>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>> On Tue, Nov 22, 2022 at 10:00 AM
>> > Justine
>> > > > >> Olshan
>> > > > >> > >> > > > > > > > > >>>>>>> <jo...@confluent.io.invalid>
>> wrote:
>> > > > >> > >> > > > > > > > > >>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>> Thanks for taking a look Matthias.
>> > I've
>> > > > >> tried
>> > > > >> > to
>> > > > >> > >> > > answer
>> > > > >> > >> > > > > your
>> > > > >> > >> > > > > > > > > >>>>> questions
>> > > > >> > >> > > > > > > > > >>>>>>>> below:
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>> 10)
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>> Right — so the hanging transaction
>> > only
>> > > > >> occurs
>> > > > >> > >> when
>> > > > >> > >> > we
>> > > > >> > >> > > > > have
>> > > > >> > >> > > > > > a
>> > > > >> > >> > > > > > > > late
>> > > > >> > >> > > > > > > > > >>>>>>> message
>> > > > >> > >> > > > > > > > > >>>>>>>> come in and the partition is never
>> > added
>> > > > to
>> > > > >> a
>> > > > >> > >> > > > transaction
>> > > > >> > >> > > > > > > again.
>> > > > >> > >> > > > > > > > > If
>> > > > >> > >> > > > > > > > > >>>>> we
>> > > > >> > >> > > > > > > > > >>>>>>>> never add the partition to a
>> > > transaction,
>> > > > we
>> > > > >> > will
>> > > > >> > >> > > never
>> > > > >> > >> > > > > > write
>> > > > >> > >> > > > > > > a
>> > > > >> > >> > > > > > > > > >>>>> marker
>> > > > >> > >> > > > > > > > > >>>>>>> and
>> > > > >> > >> > > > > > > > > >>>>>>>> never advance the LSO.
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>> If we do end up adding the
>> partition
>> > to
>> > > > the
>> > > > >> > >> > > transaction
>> > > > >> > >> > > > (I
>> > > > >> > >> > > > > > > > suppose
>> > > > >> > >> > > > > > > > > >>>>> this
>> > > > >> > >> > > > > > > > > >>>>>>> can
>> > > > >> > >> > > > > > > > > >>>>>>>> happen before or after the late
>> > message
>> > > > >> comes
>> > > > >> > in)
>> > > > >> > >> > then
>> > > > >> > >> > > > we
>> > > > >> > >> > > > > > will
>> > > > >> > >> > > > > > > > > >>>>> include
>> > > > >> > >> > > > > > > > > >>>>>>> the
>> > > > >> > >> > > > > > > > > >>>>>>>> late message in the next
>> (incorrect)
>> > > > >> > transaction.
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>> So perhaps it is clearer to make
>> the
>> > > > >> > distinction
>> > > > >> > >> > > between
>> > > > >> > >> > > > > > > > messages
>> > > > >> > >> > > > > > > > > >>>>> that
>> > > > >> > >> > > > > > > > > >>>>>>>> eventually get added to the
>> > transaction
>> > > > (but
>> > > > >> > the
>> > > > >> > >> > wrong
>> > > > >> > >> > > > > one)
>> > > > >> > >> > > > > > or
>> > > > >> > >> > > > > > > > > >>>>> messages
>> > > > >> > >> > > > > > > > > >>>>>>>> that never get added and become
>> > hanging.
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>> 20)
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>> The client side change for 2 is
>> > removing
>> > > > the
>> > > > >> > >> > > > addPartitions
>> > > > >> > >> > > > > > to
>> > > > >> > >> > > > > > > > > >>>>>> transaction
>> > > > >> > >> > > > > > > > > >>>>>>>> call. We don't need to make this
>> from
>> > > the
>> > > > >> > >> producer
>> > > > >> > >> > to
>> > > > >> > >> > > > the
>> > > > >> > >> > > > > > txn
>> > > > >> > >> > > > > > > > > >>>>>>> coordinator,
>> > > > >> > >> > > > > > > > > >>>>>>>> only server side.
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>> In my opinion, the issue with the
>> > > > >> > >> addPartitionsToTxn
>> > > > >> > >> > > > call
>> > > > >> > >> > > > > > for
>> > > > >> > >> > > > > > > > > older
>> > > > >> > >> > > > > > > > > >>>>>>> clients
>> > > > >> > >> > > > > > > > > >>>>>>>> is that we don't have the epoch
>> bump,
>> > so
>> > > > we
>> > > > >> > don't
>> > > > >> > >> > know
>> > > > >> > >> > > > if
>> > > > >> > >> > > > > > the
>> > > > >> > >> > > > > > > > > >>> message
>> > > > >> > >> > > > > > > > > >>>>>>>> belongs to the previous
>> transaction or
>> > > > this
>> > > > >> > one.
>> > > > >> > >> We
>> > > > >> > >> > > need
>> > > > >> > >> > > > > to
>> > > > >> > >> > > > > > > > check
>> > > > >> > >> > > > > > > > > if
>> > > > >> > >> > > > > > > > > >>>>>> the
>> > > > >> > >> > > > > > > > > >>>>>>>> partition has been added to this
>> > > > >> transaction.
>> > > > >> > Of
>> > > > >> > >> > > course,
>> > > > >> > >> > > > > > this
>> > > > >> > >> > > > > > > > > means
>> > > > >> > >> > > > > > > > > >>>>> we
>> > > > >> > >> > > > > > > > > >>>>>>>> won't completely cover the case
>> where
>> > we
>> > > > >> have a
>> > > > >> > >> > really
>> > > > >> > >> > > > > late
>> > > > >> > >> > > > > > > > > message
>> > > > >> > >> > > > > > > > > >>>>> and
>> > > > >> > >> > > > > > > > > >>>>>>> we
>> > > > >> > >> > > > > > > > > >>>>>>>> have added the partition to the new
>> > > > >> > transaction,
>> > > > >> > >> but
>> > > > >> > >> > > > > that's
>> > > > >> > >> > > > > > > > > >>>>>> unfortunately
>> > > > >> > >> > > > > > > > > >>>>>>>> something we will need the new
>> clients
>> > > to
>> > > > >> > cover.
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>> 30)
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>> Transaction is ongoing = partition
>> was
>> > > > >> added to
>> > > > >> > >> > > > > transaction
>> > > > >> > >> > > > > > > via
>> > > > >> > >> > > > > > > > > >>>>>>>> addPartitionsToTxn. We check this
>> with
>> > > the
>> > > > >> > >> > > > > > > DescribeTransactions
>> > > > >> > >> > > > > > > > > >>> call.
>> > > > >> > >> > > > > > > > > >>>>>> Let
>> > > > >> > >> > > > > > > > > >>>>>>>> me know if this wasn't sufficiently
>> > > > >> explained
>> > > > >> > >> here:
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > >
>> > > > >> > >> > > > > > > >
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > >
>> > > > >> > >> > > > >
>> > > > >> > >> > > >
>> > > > >> > >> > >
>> > > > >> > >> >
>> > > > >> > >>
>> > > > >> >
>> > > > >>
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>> 40)
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>> The idea here is that if any
>> messages
>> > > > >> somehow
>> > > > >> > >> come
>> > > > >> > >> > in
>> > > > >> > >> > > > > before
>> > > > >> > >> > > > > > > we
>> > > > >> > >> > > > > > > > > get
>> > > > >> > >> > > > > > > > > >>>>> the
>> > > > >> > >> > > > > > > > > >>>>>>> new
>> > > > >> > >> > > > > > > > > >>>>>>>> epoch to the producer, they will be
>> > > > fenced.
>> > > > >> > >> However,
>> > > > >> > >> > > if
>> > > > >> > >> > > > we
>> > > > >> > >> > > > > > > don't
>> > > > >> > >> > > > > > > > > >>>>> think
>> > > > >> > >> > > > > > > > > >>>>>>> this
>> > > > >> > >> > > > > > > > > >>>>>>>> is necessary, it can be discussed
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>> 50)
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>> It should be synchronous because
>> if we
>> > > > have
>> > > > >> an
>> > > > >> > >> event
>> > > > >> > >> > > > (ie,
>> > > > >> > >> > > > > an
>> > > > >> > >> > > > > > > > > error)
>> > > > >> > >> > > > > > > > > >>>>>> that
>> > > > >> > >> > > > > > > > > >>>>>>>> causes us to need to abort the
>> > > > transaction,
>> > > > >> we
>> > > > >> > >> need
>> > > > >> > >> > to
>> > > > >> > >> > > > > know
>> > > > >> > >> > > > > > > > which
>> > > > >> > >> > > > > > > > > >>>>>>>> partitions to send transaction
>> markers
>> > > to.
>> > > > >> We
>> > > > >> > >> know
>> > > > >> > >> > the
>> > > > >> > >> > > > > > > > partitions
>> > > > >> > >> > > > > > > > > >>>>>> because
>> > > > >> > >> > > > > > > > > >>>>>>>> we added them to the coordinator
>> via
>> > the
>> > > > >> > >> > > > > addPartitionsToTxn
>> > > > >> > >> > > > > > > > call.
>> > > > >> > >> > > > > > > > > >>>>>>>> Previously we have had asynchronous
>> > > calls
>> > > > in
>> > > > >> > the
>> > > > >> > >> > past
>> > > > >> > >> > > > (ie,
>> > > > >> > >> > > > > > > > writing
>> > > > >> > >> > > > > > > > > >>>>> the
>> > > > >> > >> > > > > > > > > >>>>>>>> commit markers when the
>> transaction is
>> > > > >> > completed)
>> > > > >> > >> > but
>> > > > >> > >> > > > > often
>> > > > >> > >> > > > > > > this
>> > > > >> > >> > > > > > > > > >>> just
>> > > > >> > >> > > > > > > > > >>>>>>>> causes confusion as we need to wait
>> > for
>> > > > some
>> > > > >> > >> > > operations
>> > > > >> > >> > > > to
>> > > > >> > >> > > > > > > > > complete.
>> > > > >> > >> > > > > > > > > >>>>> In
>> > > > >> > >> > > > > > > > > >>>>>>> the
>> > > > >> > >> > > > > > > > > >>>>>>>> writing commit markers case,
>> clients
>> > > often
>> > > > >> see
>> > > > >> > >> > > > > > > > > >>>>> CONCURRENT_TRANSACTIONs
>> > > > >> > >> > > > > > > > > >>>>>>>> error messages and that can be
>> > > confusing.
>> > > > >> For
>> > > > >> > >> that
>> > > > >> > >> > > > reason,
>> > > > >> > >> > > > > > it
>> > > > >> > >> > > > > > > > may
>> > > > >> > >> > > > > > > > > be
>> > > > >> > >> > > > > > > > > >>>>>>>> simpler to just have synchronous
>> > calls —
>> > > > >> > >> especially
>> > > > >> > >> > if
>> > > > >> > >> > > > we
>> > > > >> > >> > > > > > need
>> > > > >> > >> > > > > > > > to
>> > > > >> > >> > > > > > > > > >>>>> block
>> > > > >> > >> > > > > > > > > >>>>>>> on
>> > > > >> > >> > > > > > > > > >>>>>>>> some operation's completion anyway
>> > > before
>> > > > we
>> > > > >> > can
>> > > > >> > >> > start
>> > > > >> > >> > > > the
>> > > > >> > >> > > > > > > next
>> > > > >> > >> > > > > > > > > >>>>>>>> transaction. And yes, I meant
>> > > > coordinator. I
>> > > > >> > will
>> > > > >> > >> > fix
>> > > > >> > >> > > > > that.
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>> 60)
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>> When we are checking if the
>> > transaction
>> > > is
>> > > > >> > >> ongoing,
>> > > > >> > >> > we
>> > > > >> > >> > > > > need
>> > > > >> > >> > > > > > to
>> > > > >> > >> > > > > > > > > make
>> > > > >> > >> > > > > > > > > >>> a
>> > > > >> > >> > > > > > > > > >>>>>>> round
>> > > > >> > >> > > > > > > > > >>>>>>>> trip from the leader partition to
>> the
>> > > > >> > transaction
>> > > > >> > >> > > > > > coordinator.
>> > > > >> > >> > > > > > > > In
>> > > > >> > >> > > > > > > > > >>> the
>> > > > >> > >> > > > > > > > > >>>>>>> time
>> > > > >> > >> > > > > > > > > >>>>>>>> we are waiting for this message to
>> > come
>> > > > >> back,
>> > > > >> > in
>> > > > >> > >> > > theory
>> > > > >> > >> > > > we
>> > > > >> > >> > > > > > > could
>> > > > >> > >> > > > > > > > > >>> have
>> > > > >> > >> > > > > > > > > >>>>>>> sent
>> > > > >> > >> > > > > > > > > >>>>>>>> a commit/abort call that would make
>> > the
>> > > > >> > original
>> > > > >> > >> > > result
>> > > > >> > >> > > > of
>> > > > >> > >> > > > > > the
>> > > > >> > >> > > > > > > > > check
>> > > > >> > >> > > > > > > > > >>>>>> out
>> > > > >> > >> > > > > > > > > >>>>>>> of
>> > > > >> > >> > > > > > > > > >>>>>>>> date. That is why we can check the
>> > > leader
>> > > > >> state
>> > > > >> > >> > before
>> > > > >> > >> > > > we
>> > > > >> > >> > > > > > > write
>> > > > >> > >> > > > > > > > to
>> > > > >> > >> > > > > > > > > >>>>> the
>> > > > >> > >> > > > > > > > > >>>>>>> log.
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>> I'm happy to update the KIP if
>> some of
>> > > > these
>> > > > >> > >> things
>> > > > >> > >> > > were
>> > > > >> > >> > > > > not
>> > > > >> > >> > > > > > > > > clear.
>> > > > >> > >> > > > > > > > > >>>>>>>> Thanks,
>> > > > >> > >> > > > > > > > > >>>>>>>> Justine
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>> On Mon, Nov 21, 2022 at 7:11 PM
>> > Matthias
>> > > > J.
>> > > > >> > Sax <
>> > > > >> > >> > > > > > > > mjsax@apache.org
>> > > > >> > >> > > > > > > > > >
>> > > > >> > >> > > > > > > > > >>>>>>> wrote:
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>> Thanks for the KIP.
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>> Couple of clarification questions
>> (I
>> > am
>> > > > >> not a
>> > > > >> > >> > broker
>> > > > >> > >> > > > > expert
>> > > > >> > >> > > > > > > do
>> > > > >> > >> > > > > > > > > >>>>> maybe
>> > > > >> > >> > > > > > > > > >>>>>>>>> some question are obvious for
>> others,
>> > > but
>> > > > >> not
>> > > > >> > >> for
>> > > > >> > >> > me
>> > > > >> > >> > > > with
>> > > > >> > >> > > > > > my
>> > > > >> > >> > > > > > > > lack
>> > > > >> > >> > > > > > > > > >>>>> of
>> > > > >> > >> > > > > > > > > >>>>>>>>> broker knowledge).
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>> (10)
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>> 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.
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>> What happens if the message come
>> in
>> > > > before
>> > > > >> the
>> > > > >> > >> next
>> > > > >> > >> > > > > > > > > >>>>>> addPartitionsToTxn
>> > > > >> > >> > > > > > > > > >>>>>>>>> request? It seems the broker
>> hosting
>> > > the
>> > > > >> data
>> > > > >> > >> > > > partitions
>> > > > >> > >> > > > > > > won't
>> > > > >> > >> > > > > > > > > know
>> > > > >> > >> > > > > > > > > >>>>>>>>> anything about it and append it to
>> > the
>> > > > >> > >> partition,
>> > > > >> > >> > > too?
>> > > > >> > >> > > > > What
>> > > > >> > >> > > > > > > is
>> > > > >> > >> > > > > > > > > the
>> > > > >> > >> > > > > > > > > >>>>>>>>> difference between both cases?
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>> Also, it seems a TX would only
>> hang,
>> > if
>> > > > >> there
>> > > > >> > >> is no
>> > > > >> > >> > > > > > following
>> > > > >> > >> > > > > > > > TX
>> > > > >> > >> > > > > > > > > >>>>> that
>> > > > >> > >> > > > > > > > > >>>>>>> is
>> > > > >> > >> > > > > > > > > >>>>>>>>> either committer or aborted? Thus,
>> > for
>> > > > the
>> > > > >> > case
>> > > > >> > >> > > above,
>> > > > >> > >> > > > > the
>> > > > >> > >> > > > > > TX
>> > > > >> > >> > > > > > > > > might
>> > > > >> > >> > > > > > > > > >>>>>>>>> actually not hang (of course, we
>> > might
>> > > > get
>> > > > >> an
>> > > > >> > >> EOS
>> > > > >> > >> > > > > violation
>> > > > >> > >> > > > > > > if
>> > > > >> > >> > > > > > > > > the
>> > > > >> > >> > > > > > > > > >>>>>>> first
>> > > > >> > >> > > > > > > > > >>>>>>>>> TX was aborted and the second
>> > > committed,
>> > > > or
>> > > > >> > the
>> > > > >> > >> > other
>> > > > >> > >> > > > way
>> > > > >> > >> > > > > > > > > around).
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>> (20)
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>> Of course, 1 and 2 require
>> > client-side
>> > > > >> > >> changes, so
>> > > > >> > >> > > for
>> > > > >> > >> > > > > > older
>> > > > >> > >> > > > > > > > > >>>>>> clients,
>> > > > >> > >> > > > > > > > > >>>>>>>>> those approaches won’t apply.
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>> For (1) I understand why a client
>> > > change
>> > > > is
>> > > > >> > >> > > necessary,
>> > > > >> > >> > > > > but
>> > > > >> > >> > > > > > > not
>> > > > >> > >> > > > > > > > > sure
>> > > > >> > >> > > > > > > > > >>>>>> why
>> > > > >> > >> > > > > > > > > >>>>>>>>> we need a client change for (2).
>> Can
>> > > you
>> > > > >> > >> elaborate?
>> > > > >> > >> > > --
>> > > > >> > >> > > > > > Later
>> > > > >> > >> > > > > > > > you
>> > > > >> > >> > > > > > > > > >>>>>>> explain
>> > > > >> > >> > > > > > > > > >>>>>>>>> that we should send a
>> > > > >> > >> DescribeTransactionRequest,
>> > > > >> > >> > > but I
>> > > > >> > >> > > > > am
>> > > > >> > >> > > > > > > not
>> > > > >> > >> > > > > > > > > sure
>> > > > >> > >> > > > > > > > > >>>>>>> why?
>> > > > >> > >> > > > > > > > > >>>>>>>>> Can't we not just do an implicit
>> > > > >> > >> AddPartiitonToTx,
>> > > > >> > >> > > too?
>> > > > >> > >> > > > > If
>> > > > >> > >> > > > > > > the
>> > > > >> > >> > > > > > > > > old
>> > > > >> > >> > > > > > > > > >>>>>>>>> producer correctly registered the
>> > > > partition
>> > > > >> > >> > already,
>> > > > >> > >> > > > the
>> > > > >> > >> > > > > > > > > >>>>>> TX-coordinator
>> > > > >> > >> > > > > > > > > >>>>>>>>> can just ignore it as it's an
>> > > idempotent
>> > > > >> > >> operation?
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>> (30)
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>> To cover older clients, we will
>> > > ensure a
>> > > > >> > >> > transaction
>> > > > >> > >> > > > is
>> > > > >> > >> > > > > > > > ongoing
>> > > > >> > >> > > > > > > > > >>>>>>> before
>> > > > >> > >> > > > > > > > > >>>>>>>>> we write to a transaction
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>> Not sure what you mean by this?
>> Can
>> > you
>> > > > >> > >> elaborate?
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>> (40)
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>> [the TX-coordinator] will write
>> the
>> > > > >> prepare
>> > > > >> > >> commit
>> > > > >> > >> > > > > message
>> > > > >> > >> > > > > > > > with
>> > > > >> > >> > > > > > > > > a
>> > > > >> > >> > > > > > > > > >>>>>>>> bumped
>> > > > >> > >> > > > > > > > > >>>>>>>>> epoch and send
>> WriteTxnMarkerRequests
>> > > > with
>> > > > >> the
>> > > > >> > >> > bumped
>> > > > >> > >> > > > > > epoch.
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>> Why do we use the bumped epoch for
>> > > both?
>> > > > It
>> > > > >> > >> seems
>> > > > >> > >> > > more
>> > > > >> > >> > > > > > > > intuitive
>> > > > >> > >> > > > > > > > > to
>> > > > >> > >> > > > > > > > > >>>>>> use
>> > > > >> > >> > > > > > > > > >>>>>>>>> the current epoch, and only return
>> > the
>> > > > >> bumped
>> > > > >> > >> epoch
>> > > > >> > >> > > to
>> > > > >> > >> > > > > the
>> > > > >> > >> > > > > > > > > >>>>> producer?
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>> (50) "Implicit
>> > > AddPartitionToTransaction"
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>> Why does the implicitly sent
>> request
>> > > need
>> > > > >> to
>> > > > >> > be
>> > > > >> > >> > > > > > synchronous?
>> > > > >> > >> > > > > > > > The
>> > > > >> > >> > > > > > > > > >>>>> KIP
>> > > > >> > >> > > > > > > > > >>>>>>>>> also says
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>> in case we need to abort and
>> need to
>> > > > know
>> > > > >> > which
>> > > > >> > >> > > > > partitions
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>> What do you mean by this?
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>> we don’t want to write to it
>> before
>> > we
>> > > > >> store
>> > > > >> > in
>> > > > >> > >> > the
>> > > > >> > >> > > > > > > > transaction
>> > > > >> > >> > > > > > > > > >>>>>>> manager
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>> Do you mean TX-coordinator
>> instead of
>> > > > >> > "manager"?
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>> (60)
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>> For older clients and ensuring
>> that
>> > the
>> > > > TX
>> > > > >> is
>> > > > >> > >> > > ongoing,
>> > > > >> > >> > > > > you
>> > > > >> > >> > > > > > > > > >>>>> describe a
>> > > > >> > >> > > > > > > > > >>>>>>>>> race condition. I am not sure if I
>> > can
>> > > > >> follow
>> > > > >> > >> here.
>> > > > >> > >> > > Can
>> > > > >> > >> > > > > you
>> > > > >> > >> > > > > > > > > >>>>>> elaborate?
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>> -Matthias
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>> On 11/18/22 1:21 PM, Justine
>> Olshan
>> > > > wrote:
>> > > > >> > >> > > > > > > > > >>>>>>>>>> Hey all!
>> > > > >> > >> > > > > > > > > >>>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>> I'd like to start a discussion
>> on my
>> > > > >> proposal
>> > > > >> > >> to
>> > > > >> > >> > add
>> > > > >> > >> > > > > some
>> > > > >> > >> > > > > > > > > >>>>>> server-side
>> > > > >> > >> > > > > > > > > >>>>>>>>>> checks on transactions to avoid
>> > > hanging
>> > > > >> > >> > > transactions.
>> > > > >> > >> > > > I
>> > > > >> > >> > > > > > know
>> > > > >> > >> > > > > > > > > this
>> > > > >> > >> > > > > > > > > >>>>>> has
>> > > > >> > >> > > > > > > > > >>>>>>>>> been
>> > > > >> > >> > > > > > > > > >>>>>>>>>> an issue for some time, so I
>> really
>> > > hope
>> > > > >> this
>> > > > >> > >> KIP
>> > > > >> > >> > > will
>> > > > >> > >> > > > > be
>> > > > >> > >> > > > > > > > > helpful
>> > > > >> > >> > > > > > > > > >>>>>> for
>> > > > >> > >> > > > > > > > > >>>>>>>>> many
>> > > > >> > >> > > > > > > > > >>>>>>>>>> users of EOS.
>> > > > >> > >> > > > > > > > > >>>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>> The KIP includes changes that
>> will
>> > be
>> > > > >> > >> compatible
>> > > > >> > >> > > with
>> > > > >> > >> > > > > old
>> > > > >> > >> > > > > > > > > clients
>> > > > >> > >> > > > > > > > > >>>>>> and
>> > > > >> > >> > > > > > > > > >>>>>>>>>> changes to improve performance
>> and
>> > > > >> > correctness
>> > > > >> > >> on
>> > > > >> > >> > > new
>> > > > >> > >> > > > > > > clients.
>> > > > >> > >> > > > > > > > > >>>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>> Please take a look and leave any
>> > > > comments
>> > > > >> you
>> > > > >> > >> may
>> > > > >> > >> > > > have!
>> > > > >> > >> > > > > > > > > >>>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>> KIP:
>> > > > >> > >> > > > > > > > > >>>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > >
>> > > > >> > >> > > > > > > >
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > >
>> > > > >> > >> > > > >
>> > > > >> > >> > > >
>> > > > >> > >> > >
>> > > > >> > >> >
>> > > > >> > >>
>> > > > >> >
>> > > > >>
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
>> > > > >> > >> > > > > > > > > >>>>>>>>>> JIRA:
>> > > > >> > >> > > > https://issues.apache.org/jira/browse/KAFKA-14402
>> > > > >> > >> > > > > > > > > >>>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>> Thanks!
>> > > > >> > >> > > > > > > > > >>>>>>>>>> Justine
>> > > > >> > >> > > > > > > > > >>>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>>
>> > > > >> > >> > > > > > > > > >>>>>>
>> > > > >> > >> > > > > > > > > >>>>>
>> > > > >> > >> > > > > > > > > >>>>
>> > > > >> > >> > > > > > > > > >>>
>> > > > >> > >> > > > > > > > > >>
>> > > > >> > >> > > > > > > > > >
>> > > > >> > >> > > > > > > > >
>> > > > >> > >> > > > > > > >
>> > > > >> > >> > > > > > >
>> > > > >> > >> > > > > >
>> > > > >> > >> > > > >
>> > > > >> > >> > > >
>> > > > >> > >> > >
>> > > > >> > >> >
>> > > > >> > >>
>> > > > >> > >
>> > > > >> >
>> > > > >>
>> > > > >
>> > > >
>> > >
>> >
>>
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
Hey all, just wanted to quickly update and say I've modified the KIP to
explicitly mention that AddOffsetCommitsToTxnRequest will be replaced by a
coordinator-side (inter-broker) AddPartitionsToTxn implicit request. This
mirrors the user partitions and will implicitly add offset partitions to
transactions when we commit offsets on them. We will deprecate
AddOffsetCommitsToTxnRequest
for new clients.

Also to address Artem's comments --
I'm a bit unsure if the changes here will change the previous behavior for
fencing producers. In the case you mention in the first paragraph, are you
saying we bump the epoch before we try to abort the transaction? I think I
need to understand the scenarios you mention a bit better.

As for the second part -- I think it makes sense to have some sort of
"sentinel" epoch to signal epoch is about to overflow (I think we sort of
have this value in place in some ways) so we can codify it in the KIP. I'll
look into that and try to update soon.

Thanks,
Justine.

On Fri, Jan 13, 2023 at 5:01 PM Artem Livshits
<al...@confluent.io.invalid> wrote:

> It's good to know that KIP-588 addressed some of the issues.  Looking at
> the code, it still looks like there are some cases that would result in
> fatal error, e.g. PRODUCER_FENCED is issued by the transaction coordinator
> if epoch doesn't match, and the client treats it as a fatal error (code in
> TransactionManager request handling).  If we consider, for example,
> committing a transaction that returns a timeout, but actually succeeds,
> trying to abort it or re-commit may result in PRODUCER_FENCED error
> (because of epoch bump).
>
> For failed commits, specifically, we need to know the actual outcome,
> because if we return an error the application may think that the
> transaction is aborted and redo the work, leading to duplicates.
>
> Re: overflowing epoch.  We could either do it on the TC and return both
> producer id and epoch (e.g. change the protocol), or signal the client that
> it needs to get a new producer id.  Checking for max epoch could be a
> reasonable signal, the value to check should probably be present in the KIP
> as this is effectively a part of the contract.  Also, the TC should
> probably return an error if the client didn't change producer id after
> hitting max epoch.
>
> -Artem
>
>
> On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
> <jo...@confluent.io.invalid> wrote:
>
> > Thanks for the discussion Artem.
> >
> > With respect to the handling of fenced producers, we have some behavior
> > already in place. As of KIP-588:
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> > ,
> > we handle timeouts more gracefully. The producer can recover.
> >
> > Produce requests can also recover from epoch fencing by aborting the
> > transaction and starting over.
> >
> > What other cases were you considering that would cause us to have a
> fenced
> > epoch but we'd want to recover?
> >
> > The first point about handling epoch overflows is fair. I think there is
> > some logic we'd need to consider. (ie, if we are one away from the max
> > epoch, we need to reset the producer ID.) I'm still wondering if there
> is a
> > way to direct this from the response, or if everything should be done on
> > the client side. Let me know if you have any thoughts here.
> >
> > Thanks,
> > Justine
> >
> > On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
> > <al...@confluent.io.invalid> wrote:
> >
> > > There are some workflows in the client that are implied by protocol
> > > changes, e.g.:
> > >
> > > - for new clients, epoch changes with every transaction and can
> overflow,
> > > in old clients this condition was handled transparently, because epoch
> > was
> > > bumped in InitProducerId and it would return a new producer id if epoch
> > > overflows, the new clients would need to implement some workflow to
> > refresh
> > > producer id
> > > - how to handle fenced producers, for new clients epoch changes with
> > every
> > > transaction, so in presence of failures during commits / aborts, the
> > > producer could get easily fenced, old clients would pretty much would
> get
> > > fenced when a new incarnation of the producer was initialized with
> > > InitProducerId so it's ok to treat as a fatal error, the new clients
> > would
> > > need to implement some workflow to handle that error, otherwise they
> > could
> > > get fenced by themselves
> > > - in particular (as a subset of the previous issue), what would the
> > client
> > > do if it got a timeout during commit?  commit could've succeeded or
> > failed
> > >
> > > Not sure if this has to be defined in the KIP as implementing those
> > > probably wouldn't require protocol changes, but we have multiple
> > > implementations of Kafka clients, so probably would be good to have
> some
> > > client implementation guidance.  Could also be done as a separate doc.
> > >
> > > -Artem
> > >
> > > On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
> > <jolshan@confluent.io.invalid
> > > >
> > > wrote:
> > >
> > > > Hey all, I've updated the KIP to incorporate Jason's suggestions.
> > > >
> > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > >
> > > >
> > > > 1. Use AddPartitionsToTxn + verify flag to check on old clients
> > > > 2. Updated AddPartitionsToTxn API to support transaction batching
> > > > 3. Mention IBP bump
> > > > 4. Mention auth change on new AddPartitionsToTxn version.
> > > >
> > > > I'm planning on opening a vote soon.
> > > > Thanks,
> > > > Justine
> > > >
> > > > On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <jo...@confluent.io>
> > > > wrote:
> > > >
> > > > > Thanks Jason. Those changes make sense to me. I will update the
> KIP.
> > > > >
> > > > >
> > > > >
> > > > > On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
> > > > <ja...@confluent.io.invalid>
> > > > > wrote:
> > > > >
> > > > >> Hey Justine,
> > > > >>
> > > > >> > I was wondering about compatibility here. When we send requests
> > > > >> between brokers, we want to ensure that the receiving broker
> > > understands
> > > > >> the request (specifically the new fields). Typically this is done
> > via
> > > > >> IBP/metadata version.
> > > > >> I'm trying to think if there is a way around it but I'm not sure
> > there
> > > > is.
> > > > >>
> > > > >> Yes. I think we would gate usage of this behind an IBP bump. Does
> > that
> > > > >> seem
> > > > >> reasonable?
> > > > >>
> > > > >> > As for the improvements -- can you clarify how the multiple
> > > > >> transactional
> > > > >> IDs would help here? Were you thinking of a case where we
> wait/batch
> > > > >> multiple produce requests together? My understanding for now was 1
> > > > >> transactional ID and one validation per 1 produce request.
> > > > >>
> > > > >> Each call to `AddPartitionsToTxn` is essentially a write to the
> > > > >> transaction
> > > > >> log and must block on replication. The more we can fit into a
> single
> > > > >> request, the more writes we can do in parallel. The alternative is
> > to
> > > > make
> > > > >> use of more connections, but usually we prefer batching since the
> > > > network
> > > > >> stack is not really optimized for high connection/request loads.
> > > > >>
> > > > >> > Finally with respect to the authorizations, I think it makes
> sense
> > > to
> > > > >> skip
> > > > >> topic authorizations, but I'm a bit confused by the "leader ID"
> > field.
> > > > >> Wouldn't we just want to flag the request as from a broker (does
> it
> > > > matter
> > > > >> which one?).
> > > > >>
> > > > >> We could also make it version-based. For the next version, we
> could
> > > > >> require
> > > > >> CLUSTER auth. So clients would not be able to use the API anymore,
> > > which
> > > > >> is
> > > > >> probably what we want.
> > > > >>
> > > > >> -Jason
> > > > >>
> > > > >> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> > > > >> <jo...@confluent.io.invalid>
> > > > >> wrote:
> > > > >>
> > > > >> > As a follow up, I was just thinking about the batching a bit
> more.
> > > > >> > I suppose if we have one request in flight and we queue up the
> > other
> > > > >> > produce requests in some sort of purgatory, we could send
> > > information
> > > > >> out
> > > > >> > for all of them rather than one by one. So that would be a
> benefit
> > > of
> > > > >> > batching partitions to add per transaction.
> > > > >> >
> > > > >> > I'll need to think a bit more on the design of this part of the
> > KIP,
> > > > and
> > > > >> > will update the KIP in the next few days.
> > > > >> >
> > > > >> > Thanks,
> > > > >> > Justine
> > > > >> >
> > > > >> > On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan <
> > > jolshan@confluent.io>
> > > > >> > wrote:
> > > > >> >
> > > > >> > > Hey Jason -- thanks for the input -- I was just digging a bit
> > > deeper
> > > > >> into
> > > > >> > > the design + implementation of the validation calls here and
> > what
> > > > you
> > > > >> say
> > > > >> > > makes sense.
> > > > >> > >
> > > > >> > > I was wondering about compatibility here. When we send
> requests
> > > > >> > > between brokers, we want to ensure that the receiving broker
> > > > >> understands
> > > > >> > > the request (specifically the new fields). Typically this is
> > done
> > > > via
> > > > >> > > IBP/metadata version.
> > > > >> > > I'm trying to think if there is a way around it but I'm not
> sure
> > > > there
> > > > >> > is.
> > > > >> > >
> > > > >> > > As for the improvements -- can you clarify how the multiple
> > > > >> transactional
> > > > >> > > IDs would help here? Were you thinking of a case where we
> > > wait/batch
> > > > >> > > multiple produce requests together? My understanding for now
> > was 1
> > > > >> > > transactional ID and one validation per 1 produce request.
> > > > >> > >
> > > > >> > > Finally with respect to the authorizations, I think it makes
> > sense
> > > > to
> > > > >> > skip
> > > > >> > > topic authorizations, but I'm a bit confused by the "leader
> ID"
> > > > field.
> > > > >> > > Wouldn't we just want to flag the request as from a broker
> (does
> > > it
> > > > >> > matter
> > > > >> > > which one?).
> > > > >> > >
> > > > >> > > I think I want to adopt these suggestions, just had a few
> > > questions
> > > > on
> > > > >> > the
> > > > >> > > details.
> > > > >> > >
> > > > >> > > Thanks,
> > > > >> > > Justine
> > > > >> > >
> > > > >> > > On Thu, Jan 5, 2023 at 5:05 PM Jason Gustafson
> > > > >> > <ja...@confluent.io.invalid>
> > > > >> > > wrote:
> > > > >> > >
> > > > >> > >> Hi Justine,
> > > > >> > >>
> > > > >> > >> Thanks for the proposal.
> > > > >> > >>
> > > > >> > >> I was thinking about the implementation a little bit. In the
> > > > current
> > > > >> > >> proposal, the behavior depends on whether we have an old or
> new
> > > > >> client.
> > > > >> > >> For
> > > > >> > >> old clients, we send `DescribeTransactions` and verify the
> > result
> > > > and
> > > > >> > for
> > > > >> > >> new clients, we send `AddPartitionsToTxn`. We might be able
> to
> > > > >> simplify
> > > > >> > >> the
> > > > >> > >> implementation if we can use the same request type. For
> > example,
> > > > >> what if
> > > > >> > >> we
> > > > >> > >> bump the protocol version for `AddPartitionsToTxn` and add a
> > > > >> > >> `validateOnly`
> > > > >> > >> flag? For older versions, we can set `validateOnly=true` so
> > that
> > > > the
> > > > >> > >> request only returns successfully if the partition had
> already
> > > been
> > > > >> > added.
> > > > >> > >> For new versions, we can set `validateOnly=false` and the
> > > partition
> > > > >> will
> > > > >> > >> be
> > > > >> > >> added to the transaction. The other slightly annoying thing
> > that
> > > > this
> > > > >> > >> would
> > > > >> > >> get around is the need to collect the transaction state for
> all
> > > > >> > partitions
> > > > >> > >> even when we only care about a subset.
> > > > >> > >>
> > > > >> > >> Some additional improvements to consider:
> > > > >> > >>
> > > > >> > >> - We can give `AddPartitionsToTxn` better batch support for
> > > > >> inter-broker
> > > > >> > >> usage. Currently we only allow one `TransactionalId` to be
> > > > specified,
> > > > >> > but
> > > > >> > >> the broker may get some benefit being able to batch across
> > > multiple
> > > > >> > >> transactions.
> > > > >> > >> - Another small improvement is skipping topic authorization
> > > checks
> > > > >> for
> > > > >> > >> `AddPartitionsToTxn` when the request is from a broker.
> Perhaps
> > > we
> > > > >> can
> > > > >> > add
> > > > >> > >> a field for the `LeaderId` or something like that and require
> > > > CLUSTER
> > > > >> > >> permission when set.
> > > > >> > >>
> > > > >> > >> Best,
> > > > >> > >> Jason
> > > > >> > >>
> > > > >> > >>
> > > > >> > >>
> > > > >> > >> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
> > <jun@confluent.io.invalid
> > > >
> > > > >> > wrote:
> > > > >> > >>
> > > > >> > >> > Hi, Justine,
> > > > >> > >> >
> > > > >> > >> > Thanks for the explanation. It makes sense to me now.
> > > > >> > >> >
> > > > >> > >> > Jun
> > > > >> > >> >
> > > > >> > >> > On Mon, Dec 19, 2022 at 1:42 PM Justine Olshan
> > > > >> > >> > <jo...@confluent.io.invalid>
> > > > >> > >> > wrote:
> > > > >> > >> >
> > > > >> > >> > > Hi Jun,
> > > > >> > >> > >
> > > > >> > >> > > My understanding of the mechanism is that when we get to
> > the
> > > > last
> > > > >> > >> epoch,
> > > > >> > >> > we
> > > > >> > >> > > increment to the fencing/last epoch and if any further
> > > requests
> > > > >> come
> > > > >> > >> in
> > > > >> > >> > for
> > > > >> > >> > > this producer ID they are fenced. Then the producer gets
> a
> > > new
> > > > ID
> > > > >> > and
> > > > >> > >> > > restarts with epoch/sequence 0. The fenced epoch sticks
> > > around
> > > > >> for
> > > > >> > the
> > > > >> > >> > > duration of producer.id.expiration.ms and blocks any
> late
> > > > >> messages
> > > > >> > >> > there.
> > > > >> > >> > > The new ID will get to take advantage of the improved
> > > semantics
> > > > >> > around
> > > > >> > >> > > non-zero start sequences. So I think we are covered.
> > > > >> > >> > >
> > > > >> > >> > > The only potential issue is overloading the cache, but
> > > > hopefully
> > > > >> the
> > > > >> > >> > > improvements (lowered producer.id.expiration.ms) will
> help
> > > > with
> > > > >> > that.
> > > > >> > >> > Let
> > > > >> > >> > > me know if you still have concerns.
> > > > >> > >> > >
> > > > >> > >> > > Thanks,
> > > > >> > >> > > Justine
> > > > >> > >> > >
> > > > >> > >> > > On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
> > > > >> <ju...@confluent.io.invalid>
> > > > >> > >> > wrote:
> > > > >> > >> > >
> > > > >> > >> > > > Hi, Justine,
> > > > >> > >> > > >
> > > > >> > >> > > > Thanks for the explanation.
> > > > >> > >> > > >
> > > > >> > >> > > > 70. The proposed fencing logic doesn't apply when pid
> > > > changes,
> > > > >> is
> > > > >> > >> that
> > > > >> > >> > > > right? If so, I am not sure how complete we are
> > addressing
> > > > this
> > > > >> > >> issue
> > > > >> > >> > if
> > > > >> > >> > > > the pid changes more frequently.
> > > > >> > >> > > >
> > > > >> > >> > > > Thanks,
> > > > >> > >> > > >
> > > > >> > >> > > > Jun
> > > > >> > >> > > >
> > > > >> > >> > > >
> > > > >> > >> > > >
> > > > >> > >> > > > On Fri, Dec 16, 2022 at 9:16 AM Justine Olshan
> > > > >> > >> > > > <jo...@confluent.io.invalid>
> > > > >> > >> > > > wrote:
> > > > >> > >> > > >
> > > > >> > >> > > > > Hi Jun,
> > > > >> > >> > > > >
> > > > >> > >> > > > > Thanks for replying!
> > > > >> > >> > > > >
> > > > >> > >> > > > > 70.We already do the overflow mechanism, so my change
> > > would
> > > > >> just
> > > > >> > >> make
> > > > >> > >> > > it
> > > > >> > >> > > > > happen more often.
> > > > >> > >> > > > > I was also not suggesting a new field in the log, but
> > in
> > > > the
> > > > >> > >> > response,
> > > > >> > >> > > > > which would be gated by the client version. Sorry if
> > > > >> something
> > > > >> > >> there
> > > > >> > >> > is
> > > > >> > >> > > > > unclear. I think we are starting to diverge.
> > > > >> > >> > > > > The goal of this KIP is to not change to the marker
> > > format
> > > > at
> > > > >> > all.
> > > > >> > >> > > > >
> > > > >> > >> > > > > 71. Yes, I guess I was going under the assumption
> that
> > > the
> > > > >> log
> > > > >> > >> would
> > > > >> > >> > > just
> > > > >> > >> > > > > look at its last epoch and treat it as the current
> > > epoch. I
> > > > >> > >> suppose
> > > > >> > >> > we
> > > > >> > >> > > > can
> > > > >> > >> > > > > have some special logic that if the last epoch was
> on a
> > > > >> marker
> > > > >> > we
> > > > >> > >> > > > actually
> > > > >> > >> > > > > expect the next epoch or something like that. We just
> > > need
> > > > to
> > > > >> > >> > > distinguish
> > > > >> > >> > > > > based on whether we had a commit/abort marker.
> > > > >> > >> > > > >
> > > > >> > >> > > > > 72.
> > > > >> > >> > > > > > if the producer epoch hasn't been bumped on the
> > > > >> > >> > > > > broker, it seems that the stucked message will fail
> the
> > > > >> sequence
> > > > >> > >> > > > validation
> > > > >> > >> > > > > and will be ignored. If the producer epoch has been
> > > bumped,
> > > > >> we
> > > > >> > >> ignore
> > > > >> > >> > > the
> > > > >> > >> > > > > sequence check and the stuck message could be
> appended
> > to
> > > > the
> > > > >> > log.
> > > > >> > >> > So,
> > > > >> > >> > > is
> > > > >> > >> > > > > the latter case that we want to guard?
> > > > >> > >> > > > >
> > > > >> > >> > > > > I'm not sure I follow that "the message will fail the
> > > > >> sequence
> > > > >> > >> > > > validation".
> > > > >> > >> > > > > In some of these cases, we had an abort marker (due
> to
> > an
> > > > >> error)
> > > > >> > >> and
> > > > >> > >> > > then
> > > > >> > >> > > > > the late message comes in with the correct sequence
> > > number.
> > > > >> This
> > > > >> > >> is a
> > > > >> > >> > > > case
> > > > >> > >> > > > > covered by the KIP.
> > > > >> > >> > > > > The latter case is actually not something we've
> > > considered
> > > > >> > here. I
> > > > >> > >> > > think
> > > > >> > >> > > > > generally when we bump the epoch, we are accepting
> that
> > > the
> > > > >> > >> sequence
> > > > >> > >> > > does
> > > > >> > >> > > > > not need to be checked anymore. My understanding is
> > also
> > > > >> that we
> > > > >> > >> > don't
> > > > >> > >> > > > > typically bump epoch mid transaction (based on a
> quick
> > > look
> > > > >> at
> > > > >> > the
> > > > >> > >> > > code)
> > > > >> > >> > > > > but let me know if that is the case.
> > > > >> > >> > > > >
> > > > >> > >> > > > > Thanks,
> > > > >> > >> > > > > Justine
> > > > >> > >> > > > >
> > > > >> > >> > > > > On Thu, Dec 15, 2022 at 12:23 PM Jun Rao
> > > > >> > <jun@confluent.io.invalid
> > > > >> > >> >
> > > > >> > >> > > > wrote:
> > > > >> > >> > > > >
> > > > >> > >> > > > > > Hi, Justine,
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > Thanks for the reply.
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > 70. Assigning a new pid on int overflow seems a bit
> > > > hacky.
> > > > >> If
> > > > >> > we
> > > > >> > >> > > need a
> > > > >> > >> > > > > txn
> > > > >> > >> > > > > > level id, it will be better to model this
> explicitly.
> > > > >> Adding a
> > > > >> > >> new
> > > > >> > >> > > > field
> > > > >> > >> > > > > > would require a bit more work since it requires a
> new
> > > txn
> > > > >> > marker
> > > > >> > >> > > format
> > > > >> > >> > > > > in
> > > > >> > >> > > > > > the log. So, we probably need to guard it with an
> IBP
> > > or
> > > > >> > >> metadata
> > > > >> > >> > > > version
> > > > >> > >> > > > > > and document the impact on downgrade once the new
> > > format
> > > > is
> > > > >> > >> written
> > > > >> > >> > > to
> > > > >> > >> > > > > the
> > > > >> > >> > > > > > log.
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > 71. Hmm, once the marker is written, the partition
> > will
> > > > >> expect
> > > > >> > >> the
> > > > >> > >> > > next
> > > > >> > >> > > > > > append to be on the next epoch. Does that cover the
> > > case
> > > > >> you
> > > > >> > >> > > mentioned?
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > 72. Also, just to be clear on the stucked message
> > issue
> > > > >> > >> described
> > > > >> > >> > in
> > > > >> > >> > > > the
> > > > >> > >> > > > > > motivation. With EoS, we also validate the sequence
> > id
> > > > for
> > > > >> > >> > > idempotency.
> > > > >> > >> > > > > So,
> > > > >> > >> > > > > > with the current logic, if the producer epoch
> hasn't
> > > been
> > > > >> > >> bumped on
> > > > >> > >> > > the
> > > > >> > >> > > > > > broker, it seems that the stucked message will fail
> > the
> > > > >> > sequence
> > > > >> > >> > > > > validation
> > > > >> > >> > > > > > and will be ignored. If the producer epoch has been
> > > > >> bumped, we
> > > > >> > >> > ignore
> > > > >> > >> > > > the
> > > > >> > >> > > > > > sequence check and the stuck message could be
> > appended
> > > to
> > > > >> the
> > > > >> > >> log.
> > > > >> > >> > > So,
> > > > >> > >> > > > is
> > > > >> > >> > > > > > the latter case that we want to guard?
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > Thanks,
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > Jun
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > On Wed, Dec 14, 2022 at 10:44 AM Justine Olshan
> > > > >> > >> > > > > > <jo...@confluent.io.invalid> wrote:
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > > Matthias — thanks again for taking time to look a
> > > this.
> > > > >> You
> > > > >> > >> said:
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > > > My proposal was only focusing to avoid dangling
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > > transactions if records are added without
> > registered
> > > > >> > >> partition.
> > > > >> > >> > --
> > > > >> > >> > > > > Maybe
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > > you can add a few more details to the KIP about
> > this
> > > > >> > scenario
> > > > >> > >> for
> > > > >> > >> > > > > better
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > > documentation purpose?
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > > I'm not sure I understand what you mean here. The
> > > > >> motivation
> > > > >> > >> > > section
> > > > >> > >> > > > > > > describes two scenarios about how the record can
> be
> > > > added
> > > > >> > >> > without a
> > > > >> > >> > > > > > > registered partition:
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > > > 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.
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > > > Another way hanging transactions can occur is
> > that
> > > a
> > > > >> > client
> > > > >> > >> is
> > > > >> > >> > > > buggy
> > > > >> > >> > > > > > and
> > > > >> > >> > > > > > > may somehow try to write to a partition before it
> > > adds
> > > > >> the
> > > > >> > >> > > partition
> > > > >> > >> > > > to
> > > > >> > >> > > > > > the
> > > > >> > >> > > > > > > transaction.
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > > For the first example of this would it be helpful
> > to
> > > > say
> > > > >> > that
> > > > >> > >> > this
> > > > >> > >> > > > > > message
> > > > >> > >> > > > > > > comes in after the abort, but before the
> partition
> > is
> > > > >> added
> > > > >> > to
> > > > >> > >> > the
> > > > >> > >> > > > next
> > > > >> > >> > > > > > > transaction so it becomes "hanging." Perhaps the
> > next
> > > > >> > sentence
> > > > >> > >> > > > > describing
> > > > >> > >> > > > > > > the message becoming part of the next transaction
> > (a
> > > > >> > different
> > > > >> > >> > > case)
> > > > >> > >> > > > > was
> > > > >> > >> > > > > > > not properly differentiated.
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > > Jun — thanks for reading the KIP.
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > > 70. The int typing was a concern. Currently we
> > have a
> > > > >> > >> mechanism
> > > > >> > >> > in
> > > > >> > >> > > > > place
> > > > >> > >> > > > > > to
> > > > >> > >> > > > > > > fence the final epoch when the epoch is about to
> > > > overflow
> > > > >> > and
> > > > >> > >> > > assign
> > > > >> > >> > > > a
> > > > >> > >> > > > > > new
> > > > >> > >> > > > > > > producer ID with epoch 0. Of course, this is a
> bit
> > > > tricky
> > > > >> > >> when it
> > > > >> > >> > > > comes
> > > > >> > >> > > > > > to
> > > > >> > >> > > > > > > the response back to the client.
> > > > >> > >> > > > > > > Making this a long could be another option, but I
> > > > wonder
> > > > >> are
> > > > >> > >> > there
> > > > >> > >> > > > any
> > > > >> > >> > > > > > > implications on changing this field if the epoch
> is
> > > > >> > persisted
> > > > >> > >> to
> > > > >> > >> > > > disk?
> > > > >> > >> > > > > > I'd
> > > > >> > >> > > > > > > need to check the usages.
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > > 71.This was something Matthias asked about as
> > well. I
> > > > was
> > > > >> > >> > > > considering a
> > > > >> > >> > > > > > > possible edge case where a produce request from a
> > new
> > > > >> > >> transaction
> > > > >> > >> > > > > somehow
> > > > >> > >> > > > > > > gets sent right after the marker is written, but
> > > before
> > > > >> the
> > > > >> > >> > > producer
> > > > >> > >> > > > is
> > > > >> > >> > > > > > > alerted of the newly bumped epoch. In this case,
> we
> > > may
> > > > >> > >> include
> > > > >> > >> > > this
> > > > >> > >> > > > > > record
> > > > >> > >> > > > > > > when we don't want to. I suppose we could try to
> do
> > > > >> > something
> > > > >> > >> > > client
> > > > >> > >> > > > > side
> > > > >> > >> > > > > > > to bump the epoch after sending an endTxn as well
> > in
> > > > this
> > > > >> > >> > scenario
> > > > >> > >> > > —
> > > > >> > >> > > > > but
> > > > >> > >> > > > > > I
> > > > >> > >> > > > > > > wonder how it would work when the server is
> > aborting
> > > > >> based
> > > > >> > on
> > > > >> > >> a
> > > > >> > >> > > > > > server-side
> > > > >> > >> > > > > > > error. I could also be missing something and this
> > > > >> scenario
> > > > >> > is
> > > > >> > >> > > > actually
> > > > >> > >> > > > > > not
> > > > >> > >> > > > > > > possible.
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > > Thanks again to everyone reading and commenting.
> > Let
> > > me
> > > > >> know
> > > > >> > >> > about
> > > > >> > >> > > > any
> > > > >> > >> > > > > > > further questions or comments.
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > > Justine
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > > On Wed, Dec 14, 2022 at 9:41 AM Jun Rao
> > > > >> > >> <jun@confluent.io.invalid
> > > > >> > >> > >
> > > > >> > >> > > > > > wrote:
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > > > Hi, Justine,
> > > > >> > >> > > > > > > >
> > > > >> > >> > > > > > > > Thanks for the KIP. A couple of comments.
> > > > >> > >> > > > > > > >
> > > > >> > >> > > > > > > > 70. Currently, the producer epoch is an int. I
> am
> > > not
> > > > >> sure
> > > > >> > >> if
> > > > >> > >> > > it's
> > > > >> > >> > > > > > enough
> > > > >> > >> > > > > > > > to accommodate all transactions in the lifetime
> > of
> > > a
> > > > >> > >> producer.
> > > > >> > >> > > > Should
> > > > >> > >> > > > > > we
> > > > >> > >> > > > > > > > change that to a long or add a new long field
> > like
> > > > >> txnId?
> > > > >> > >> > > > > > > >
> > > > >> > >> > > > > > > > 71. "it will write the prepare commit message
> > with
> > > a
> > > > >> > bumped
> > > > >> > >> > epoch
> > > > >> > >> > > > and
> > > > >> > >> > > > > > > send
> > > > >> > >> > > > > > > > WriteTxnMarkerRequests with the bumped epoch."
> > Hmm,
> > > > the
> > > > >> > >> epoch
> > > > >> > >> > is
> > > > >> > >> > > > > > > associated
> > > > >> > >> > > > > > > > with the current txn right? So, it seems weird
> to
> > > > >> write a
> > > > >> > >> > commit
> > > > >> > >> > > > > > message
> > > > >> > >> > > > > > > > with a bumped epoch. Should we only bump up the
> > > epoch
> > > > >> in
> > > > >> > >> > > > > EndTxnResponse
> > > > >> > >> > > > > > > and
> > > > >> > >> > > > > > > > rename the field to sth like nextProducerEpoch?
> > > > >> > >> > > > > > > >
> > > > >> > >> > > > > > > > Thanks,
> > > > >> > >> > > > > > > >
> > > > >> > >> > > > > > > > Jun
> > > > >> > >> > > > > > > >
> > > > >> > >> > > > > > > >
> > > > >> > >> > > > > > > >
> > > > >> > >> > > > > > > > On Mon, Dec 12, 2022 at 8:54 PM Matthias J.
> Sax <
> > > > >> > >> > > mjsax@apache.org>
> > > > >> > >> > > > > > > wrote:
> > > > >> > >> > > > > > > >
> > > > >> > >> > > > > > > > > Thanks for the background.
> > > > >> > >> > > > > > > > >
> > > > >> > >> > > > > > > > > 20/30: SGTM. My proposal was only focusing to
> > > avoid
> > > > >> > >> dangling
> > > > >> > >> > > > > > > > > transactions if records are added without
> > > > registered
> > > > >> > >> > partition.
> > > > >> > >> > > > --
> > > > >> > >> > > > > > > Maybe
> > > > >> > >> > > > > > > > > you can add a few more details to the KIP
> about
> > > > this
> > > > >> > >> scenario
> > > > >> > >> > > for
> > > > >> > >> > > > > > > better
> > > > >> > >> > > > > > > > > documentation purpose?
> > > > >> > >> > > > > > > > >
> > > > >> > >> > > > > > > > > 40: I think you hit a fair point about race
> > > > >> conditions
> > > > >> > or
> > > > >> > >> > > client
> > > > >> > >> > > > > bugs
> > > > >> > >> > > > > > > > > (incorrectly not bumping the epoch). The
> > > > >> > >> complexity/confusion
> > > > >> > >> > > for
> > > > >> > >> > > > > > using
> > > > >> > >> > > > > > > > > the bumped epoch I see, is mainly for
> internal
> > > > >> > debugging,
> > > > >> > >> ie,
> > > > >> > >> > > > > > > inspecting
> > > > >> > >> > > > > > > > > log segment dumps -- it seems harder to
> reason
> > > > about
> > > > >> the
> > > > >> > >> > system
> > > > >> > >> > > > for
> > > > >> > >> > > > > > us
> > > > >> > >> > > > > > > > > humans. But if we get better guarantees, it
> > would
> > > > be
> > > > >> > >> worth to
> > > > >> > >> > > use
> > > > >> > >> > > > > the
> > > > >> > >> > > > > > > > > bumped epoch.
> > > > >> > >> > > > > > > > >
> > > > >> > >> > > > > > > > > 60: as I mentioned already, I don't know the
> > > broker
> > > > >> > >> internals
> > > > >> > >> > > to
> > > > >> > >> > > > > > > provide
> > > > >> > >> > > > > > > > > more input. So if nobody else chimes in, we
> > > should
> > > > >> just
> > > > >> > >> move
> > > > >> > >> > > > > forward
> > > > >> > >> > > > > > > > > with your proposal.
> > > > >> > >> > > > > > > > >
> > > > >> > >> > > > > > > > >
> > > > >> > >> > > > > > > > > -Matthias
> > > > >> > >> > > > > > > > >
> > > > >> > >> > > > > > > > >
> > > > >> > >> > > > > > > > > On 12/6/22 4:22 PM, Justine Olshan wrote:
> > > > >> > >> > > > > > > > > > Hi all,
> > > > >> > >> > > > > > > > > > After Artem's questions about error
> behavior,
> > > > I've
> > > > >> > >> > > re-evaluated
> > > > >> > >> > > > > the
> > > > >> > >> > > > > > > > > > unknown producer ID exception and had some
> > > > >> discussions
> > > > >> > >> > > offline.
> > > > >> > >> > > > > > > > > >
> > > > >> > >> > > > > > > > > > I think generally it makes sense to
> simplify
> > > > error
> > > > >> > >> handling
> > > > >> > >> > > in
> > > > >> > >> > > > > > cases
> > > > >> > >> > > > > > > > like
> > > > >> > >> > > > > > > > > > this and the UNKNOWN_PRODUCER_ID error has
> a
> > > > pretty
> > > > >> > long
> > > > >> > >> > and
> > > > >> > >> > > > > > > > complicated
> > > > >> > >> > > > > > > > > > history. Because of this, I propose adding
> a
> > > new
> > > > >> error
> > > > >> > >> code
> > > > >> > >> > > > > > > > > ABORTABLE_ERROR
> > > > >> > >> > > > > > > > > > that when encountered by new clients (gated
> > by
> > > > the
> > > > >> > >> produce
> > > > >> > >> > > > > request
> > > > >> > >> > > > > > > > > version)
> > > > >> > >> > > > > > > > > > will simply abort the transaction. This
> > allows
> > > > the
> > > > >> > >> server
> > > > >> > >> > to
> > > > >> > >> > > > have
> > > > >> > >> > > > > > > some
> > > > >> > >> > > > > > > > > say
> > > > >> > >> > > > > > > > > > in whether the client aborts and makes
> > handling
> > > > >> much
> > > > >> > >> > simpler.
> > > > >> > >> > > > In
> > > > >> > >> > > > > > the
> > > > >> > >> > > > > > > > > > future, we can also use this error in other
> > > > >> situations
> > > > >> > >> > where
> > > > >> > >> > > we
> > > > >> > >> > > > > > want
> > > > >> > >> > > > > > > to
> > > > >> > >> > > > > > > > > > abort the transactions. We can even use on
> > > other
> > > > >> apis.
> > > > >> > >> > > > > > > > > >
> > > > >> > >> > > > > > > > > > I've added this to the KIP. Let me know if
> > > there
> > > > >> are
> > > > >> > any
> > > > >> > >> > > > > questions
> > > > >> > >> > > > > > or
> > > > >> > >> > > > > > > > > > issues.
> > > > >> > >> > > > > > > > > >
> > > > >> > >> > > > > > > > > > Justine
> > > > >> > >> > > > > > > > > >
> > > > >> > >> > > > > > > > > > On Fri, Dec 2, 2022 at 10:22 AM Justine
> > Olshan
> > > <
> > > > >> > >> > > > > > jolshan@confluent.io
> > > > >> > >> > > > > > > >
> > > > >> > >> > > > > > > > > wrote:
> > > > >> > >> > > > > > > > > >
> > > > >> > >> > > > > > > > > >> Hey Matthias,
> > > > >> > >> > > > > > > > > >>
> > > > >> > >> > > > > > > > > >>
> > > > >> > >> > > > > > > > > >> 20/30 — Maybe I also didn't express myself
> > > > >> clearly.
> > > > >> > For
> > > > >> > >> > > older
> > > > >> > >> > > > > > > clients
> > > > >> > >> > > > > > > > we
> > > > >> > >> > > > > > > > > >> don't have a way to distinguish between a
> > > > previous
> > > > >> > and
> > > > >> > >> the
> > > > >> > >> > > > > current
> > > > >> > >> > > > > > > > > >> transaction since we don't have the epoch
> > > bump.
> > > > >> This
> > > > >> > >> means
> > > > >> > >> > > > that
> > > > >> > >> > > > > a
> > > > >> > >> > > > > > > late
> > > > >> > >> > > > > > > > > >> message from the previous transaction may
> be
> > > > >> added to
> > > > >> > >> the
> > > > >> > >> > > new
> > > > >> > >> > > > > one.
> > > > >> > >> > > > > > > > With
> > > > >> > >> > > > > > > > > >> older clients — we can't guarantee this
> > won't
> > > > >> happen
> > > > >> > >> if we
> > > > >> > >> > > > > already
> > > > >> > >> > > > > > > > sent
> > > > >> > >> > > > > > > > > the
> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call (why we make
> changes
> > > for
> > > > >> the
> > > > >> > >> newer
> > > > >> > >> > > > > client)
> > > > >> > >> > > > > > > but
> > > > >> > >> > > > > > > > > we
> > > > >> > >> > > > > > > > > >> can at least gate some by ensuring that
> the
> > > > >> partition
> > > > >> > >> has
> > > > >> > >> > > been
> > > > >> > >> > > > > > added
> > > > >> > >> > > > > > > > to
> > > > >> > >> > > > > > > > > the
> > > > >> > >> > > > > > > > > >> transaction. The rationale here is that
> > there
> > > > are
> > > > >> > >> likely
> > > > >> > >> > > LESS
> > > > >> > >> > > > > late
> > > > >> > >> > > > > > > > > arrivals
> > > > >> > >> > > > > > > > > >> as time goes on, so hopefully most late
> > > arrivals
> > > > >> will
> > > > >> > >> come
> > > > >> > >> > > in
> > > > >> > >> > > > > > BEFORE
> > > > >> > >> > > > > > > > the
> > > > >> > >> > > > > > > > > >> addPartitionsToTxn call. Those that arrive
> > > > before
> > > > >> > will
> > > > >> > >> be
> > > > >> > >> > > > > properly
> > > > >> > >> > > > > > > > gated
> > > > >> > >> > > > > > > > > >> with the describeTransactions approach.
> > > > >> > >> > > > > > > > > >>
> > > > >> > >> > > > > > > > > >> If we take the approach you suggested, ANY
> > > late
> > > > >> > arrival
> > > > >> > >> > > from a
> > > > >> > >> > > > > > > > previous
> > > > >> > >> > > > > > > > > >> transaction will be added. And we don't
> want
> > > > >> that. I
> > > > >> > >> also
> > > > >> > >> > > > don't
> > > > >> > >> > > > > > see
> > > > >> > >> > > > > > > > any
> > > > >> > >> > > > > > > > > >> benefit in sending addPartitionsToTxn over
> > the
> > > > >> > >> > describeTxns
> > > > >> > >> > > > > call.
> > > > >> > >> > > > > > > They
> > > > >> > >> > > > > > > > > will
> > > > >> > >> > > > > > > > > >> both be one extra RPC to the Txn
> > coordinator.
> > > > >> > >> > > > > > > > > >>
> > > > >> > >> > > > > > > > > >>
> > > > >> > >> > > > > > > > > >> To be clear — newer clients will use
> > > > >> > addPartitionsToTxn
> > > > >> > >> > > > instead
> > > > >> > >> > > > > of
> > > > >> > >> > > > > > > the
> > > > >> > >> > > > > > > > > >> DescribeTxns.
> > > > >> > >> > > > > > > > > >>
> > > > >> > >> > > > > > > > > >>
> > > > >> > >> > > > > > > > > >> 40)
> > > > >> > >> > > > > > > > > >> My concern is that if we have some delay
> in
> > > the
> > > > >> > client
> > > > >> > >> to
> > > > >> > >> > > bump
> > > > >> > >> > > > > the
> > > > >> > >> > > > > > > > > epoch,
> > > > >> > >> > > > > > > > > >> it could continue to send epoch 73 and
> those
> > > > >> records
> > > > >> > >> would
> > > > >> > >> > > not
> > > > >> > >> > > > > be
> > > > >> > >> > > > > > > > > fenced.
> > > > >> > >> > > > > > > > > >> Perhaps this is not an issue if we don't
> > allow
> > > > the
> > > > >> > next
> > > > >> > >> > > > produce
> > > > >> > >> > > > > to
> > > > >> > >> > > > > > > go
> > > > >> > >> > > > > > > > > >> through before the EndTxn request returns.
> > I'm
> > > > >> also
> > > > >> > >> > thinking
> > > > >> > >> > > > > about
> > > > >> > >> > > > > > > > > cases of
> > > > >> > >> > > > > > > > > >> failure. I will need to think on this a
> bit.
> > > > >> > >> > > > > > > > > >>
> > > > >> > >> > > > > > > > > >> I wasn't sure if it was that confusing.
> But
> > if
> > > > we
> > > > >> > >> think it
> > > > >> > >> > > is,
> > > > >> > >> > > > > we
> > > > >> > >> > > > > > > can
> > > > >> > >> > > > > > > > > >> investigate other ways.
> > > > >> > >> > > > > > > > > >>
> > > > >> > >> > > > > > > > > >>
> > > > >> > >> > > > > > > > > >> 60)
> > > > >> > >> > > > > > > > > >>
> > > > >> > >> > > > > > > > > >> I'm not sure these are the same
> purgatories
> > > > since
> > > > >> one
> > > > >> > >> is a
> > > > >> > >> > > > > produce
> > > > >> > >> > > > > > > > > >> purgatory (I was planning on using a
> > callback
> > > > >> rather
> > > > >> > >> than
> > > > >> > >> > > > > > purgatory)
> > > > >> > >> > > > > > > > and
> > > > >> > >> > > > > > > > > >> the other is simply a request to append to
> > the
> > > > >> log.
> > > > >> > Not
> > > > >> > >> > sure
> > > > >> > >> > > > we
> > > > >> > >> > > > > > have
> > > > >> > >> > > > > > > > any
> > > > >> > >> > > > > > > > > >> structure here for ordering, but my
> > > > understanding
> > > > >> is
> > > > >> > >> that
> > > > >> > >> > > the
> > > > >> > >> > > > > > broker
> > > > >> > >> > > > > > > > > could
> > > > >> > >> > > > > > > > > >> handle the write request before it hears
> > back
> > > > from
> > > > >> > the
> > > > >> > >> Txn
> > > > >> > >> > > > > > > > Coordinator.
> > > > >> > >> > > > > > > > > >>
> > > > >> > >> > > > > > > > > >> Let me know if I misunderstood something
> or
> > > > >> something
> > > > >> > >> was
> > > > >> > >> > > > > unclear.
> > > > >> > >> > > > > > > > > >>
> > > > >> > >> > > > > > > > > >> Justine
> > > > >> > >> > > > > > > > > >>
> > > > >> > >> > > > > > > > > >> On Thu, Dec 1, 2022 at 12:15 PM Matthias
> J.
> > > Sax
> > > > <
> > > > >> > >> > > > > mjsax@apache.org
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > > > > wrote:
> > > > >> > >> > > > > > > > > >>
> > > > >> > >> > > > > > > > > >>> Thanks for the details Justine!
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > > >>>> 20)
> > > > >> > >> > > > > > > > > >>>>
> > > > >> > >> > > > > > > > > >>>> The client side change for 2 is removing
> > the
> > > > >> > >> > addPartitions
> > > > >> > >> > > > to
> > > > >> > >> > > > > > > > > >>> transaction
> > > > >> > >> > > > > > > > > >>>> call. We don't need to make this from
> the
> > > > >> producer
> > > > >> > to
> > > > >> > >> > the
> > > > >> > >> > > > txn
> > > > >> > >> > > > > > > > > >>> coordinator,
> > > > >> > >> > > > > > > > > >>>> only server side.
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > > >>> I think I did not express myself
> clearly. I
> > > > >> > understand
> > > > >> > >> > that
> > > > >> > >> > > > we
> > > > >> > >> > > > > > can
> > > > >> > >> > > > > > > > (and
> > > > >> > >> > > > > > > > > >>> should) change the producer to not send
> the
> > > > >> > >> > `addPartitions`
> > > > >> > >> > > > > > request
> > > > >> > >> > > > > > > > any
> > > > >> > >> > > > > > > > > >>> longer. But I don't thinks it's
> requirement
> > > to
> > > > >> > change
> > > > >> > >> the
> > > > >> > >> > > > > broker?
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > > >>> What I am trying to say is: as a
> safe-guard
> > > and
> > > > >> > >> > improvement
> > > > >> > >> > > > for
> > > > >> > >> > > > > > > older
> > > > >> > >> > > > > > > > > >>> producers, the partition leader can just
> > send
> > > > the
> > > > >> > >> > > > > `addPartitions`
> > > > >> > >> > > > > > > > > >>> request to the TX-coordinator in any case
> > --
> > > if
> > > > >> the
> > > > >> > >> old
> > > > >> > >> > > > > producer
> > > > >> > >> > > > > > > > > >>> correctly did send the `addPartition`
> > request
> > > > to
> > > > >> the
> > > > >> > >> > > > > > TX-coordinator
> > > > >> > >> > > > > > > > > >>> already, the TX-coordinator can just
> > "ignore"
> > > > is
> > > > >> as
> > > > >> > >> > > > idempotent.
> > > > >> > >> > > > > > > > > However,
> > > > >> > >> > > > > > > > > >>> if the old producer has a bug and did
> > forget
> > > to
> > > > >> sent
> > > > >> > >> the
> > > > >> > >> > > > > > > > `addPartition`
> > > > >> > >> > > > > > > > > >>> request, we would now ensure that the
> > > partition
> > > > >> is
> > > > >> > >> indeed
> > > > >> > >> > > > added
> > > > >> > >> > > > > > to
> > > > >> > >> > > > > > > > the
> > > > >> > >> > > > > > > > > >>> TX and thus fix a potential producer bug
> > > (even
> > > > >> if we
> > > > >> > >> > don't
> > > > >> > >> > > > get
> > > > >> > >> > > > > > the
> > > > >> > >> > > > > > > > > >>> fencing via the bump epoch). -- It seems
> to
> > > be
> > > > a
> > > > >> > good
> > > > >> > >> > > > > > improvement?
> > > > >> > >> > > > > > > Or
> > > > >> > >> > > > > > > > > is
> > > > >> > >> > > > > > > > > >>> there a reason to not do this?
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > > >>>> 30)
> > > > >> > >> > > > > > > > > >>>>
> > > > >> > >> > > > > > > > > >>>> Transaction is ongoing = partition was
> > added
> > > > to
> > > > >> > >> > > transaction
> > > > >> > >> > > > > via
> > > > >> > >> > > > > > > > > >>>> addPartitionsToTxn. We check this with
> the
> > > > >> > >> > > > > DescribeTransactions
> > > > >> > >> > > > > > > > call.
> > > > >> > >> > > > > > > > > >>> Let
> > > > >> > >> > > > > > > > > >>>> me know if this wasn't sufficiently
> > > explained
> > > > >> here:
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > > >>> If we do what I propose in (20), we don't
> > > > really
> > > > >> > need
> > > > >> > >> to
> > > > >> > >> > > make
> > > > >> > >> > > > > > this
> > > > >> > >> > > > > > > > > >>> `DescribeTransaction` call, as the
> > partition
> > > > >> leader
> > > > >> > >> adds
> > > > >> > >> > > the
> > > > >> > >> > > > > > > > partition
> > > > >> > >> > > > > > > > > >>> for older clients and we get this check
> for
> > > > free.
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > > >>>> 40)
> > > > >> > >> > > > > > > > > >>>>
> > > > >> > >> > > > > > > > > >>>> The idea here is that if any messages
> > > somehow
> > > > >> come
> > > > >> > in
> > > > >> > >> > > before
> > > > >> > >> > > > > we
> > > > >> > >> > > > > > > get
> > > > >> > >> > > > > > > > > the
> > > > >> > >> > > > > > > > > >>> new
> > > > >> > >> > > > > > > > > >>>> epoch to the producer, they will be
> > fenced.
> > > > >> > However,
> > > > >> > >> if
> > > > >> > >> > we
> > > > >> > >> > > > > don't
> > > > >> > >> > > > > > > > think
> > > > >> > >> > > > > > > > > >>> this
> > > > >> > >> > > > > > > > > >>>> is necessary, it can be discussed
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > > >>> I agree that we should have epoch
> fencing.
> > My
> > > > >> > >> question is
> > > > >> > >> > > > > > > different:
> > > > >> > >> > > > > > > > > >>> Assume we are at epoch 73, and we have an
> > > > ongoing
> > > > >> > >> > > > transaction,
> > > > >> > >> > > > > > that
> > > > >> > >> > > > > > > > is
> > > > >> > >> > > > > > > > > >>> committed. It seems natural to write the
> > > > "prepare
> > > > >> > >> commit"
> > > > >> > >> > > > > marker
> > > > >> > >> > > > > > > and
> > > > >> > >> > > > > > > > > the
> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` both with epoch
> 73,
> > > too,
> > > > >> as
> > > > >> > it
> > > > >> > >> > > belongs
> > > > >> > >> > > > > to
> > > > >> > >> > > > > > > the
> > > > >> > >> > > > > > > > > >>> current transaction. Of course, we now
> also
> > > > bump
> > > > >> the
> > > > >> > >> > epoch
> > > > >> > >> > > > and
> > > > >> > >> > > > > > > expect
> > > > >> > >> > > > > > > > > >>> the next requests to have epoch 74, and
> > would
> > > > >> reject
> > > > >> > >> an
> > > > >> > >> > > > request
> > > > >> > >> > > > > > > with
> > > > >> > >> > > > > > > > > >>> epoch 73, as the corresponding TX for
> epoch
> > > 73
> > > > >> was
> > > > >> > >> > already
> > > > >> > >> > > > > > > committed.
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > > >>> It seems you propose to write the
> "prepare
> > > > commit
> > > > >> > >> marker"
> > > > >> > >> > > and
> > > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` with epoch 74
> > though,
> > > > what
> > > > >> > >> would
> > > > >> > >> > > work,
> > > > >> > >> > > > > but
> > > > >> > >> > > > > > > it
> > > > >> > >> > > > > > > > > >>> seems confusing. Is there a reason why we
> > > would
> > > > >> use
> > > > >> > >> the
> > > > >> > >> > > > bumped
> > > > >> > >> > > > > > > epoch
> > > > >> > >> > > > > > > > 74
> > > > >> > >> > > > > > > > > >>> instead of the current epoch 73?
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > > >>>> 60)
> > > > >> > >> > > > > > > > > >>>>
> > > > >> > >> > > > > > > > > >>>> When we are checking if the transaction
> is
> > > > >> ongoing,
> > > > >> > >> we
> > > > >> > >> > > need
> > > > >> > >> > > > to
> > > > >> > >> > > > > > > make
> > > > >> > >> > > > > > > > a
> > > > >> > >> > > > > > > > > >>> round
> > > > >> > >> > > > > > > > > >>>> trip from the leader partition to the
> > > > >> transaction
> > > > >> > >> > > > coordinator.
> > > > >> > >> > > > > > In
> > > > >> > >> > > > > > > > the
> > > > >> > >> > > > > > > > > >>> time
> > > > >> > >> > > > > > > > > >>>> we are waiting for this message to come
> > > back,
> > > > in
> > > > >> > >> theory
> > > > >> > >> > we
> > > > >> > >> > > > > could
> > > > >> > >> > > > > > > > have
> > > > >> > >> > > > > > > > > >>> sent
> > > > >> > >> > > > > > > > > >>>> a commit/abort call that would make the
> > > > original
> > > > >> > >> result
> > > > >> > >> > of
> > > > >> > >> > > > the
> > > > >> > >> > > > > > > check
> > > > >> > >> > > > > > > > > >>> out of
> > > > >> > >> > > > > > > > > >>>> date. That is why we can check the
> leader
> > > > state
> > > > >> > >> before
> > > > >> > >> > we
> > > > >> > >> > > > > write
> > > > >> > >> > > > > > to
> > > > >> > >> > > > > > > > the
> > > > >> > >> > > > > > > > > >>> log.
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > > >>> Thanks. Got it.
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > > >>> However, is this really an issue? We put
> > the
> > > > >> produce
> > > > >> > >> > > request
> > > > >> > >> > > > in
> > > > >> > >> > > > > > > > > >>> purgatory, so how could we process the
> > > > >> > >> > > > `WriteTxnMarkerRequest`
> > > > >> > >> > > > > > > first?
> > > > >> > >> > > > > > > > > >>> Don't we need to put the
> > > > `WriteTxnMarkerRequest`
> > > > >> > into
> > > > >> > >> > > > > purgatory,
> > > > >> > >> > > > > > > too,
> > > > >> > >> > > > > > > > > >>> for this case, and process both request
> > > > in-order?
> > > > >> > >> (Again,
> > > > >> > >> > > my
> > > > >> > >> > > > > > broker
> > > > >> > >> > > > > > > > > >>> knowledge is limited and maybe we don't
> > > > maintain
> > > > >> > >> request
> > > > >> > >> > > > order
> > > > >> > >> > > > > > for
> > > > >> > >> > > > > > > > this
> > > > >> > >> > > > > > > > > >>> case, what seems to be an issue IMHO,
> and I
> > > am
> > > > >> > >> wondering
> > > > >> > >> > if
> > > > >> > >> > > > > > > changing
> > > > >> > >> > > > > > > > > >>> request handling to preserve order for
> this
> > > > case
> > > > >> > >> might be
> > > > >> > >> > > the
> > > > >> > >> > > > > > > cleaner
> > > > >> > >> > > > > > > > > >>> solution?)
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > > >>> -Matthias
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > > >>> On 11/30/22 3:28 PM, Artem Livshits
> wrote:
> > > > >> > >> > > > > > > > > >>>> Hi Justine,
> > > > >> > >> > > > > > > > > >>>>
> > > > >> > >> > > > > > > > > >>>> I think the interesting part is not in
> > this
> > > > >> logic
> > > > >> > >> > (because
> > > > >> > >> > > > it
> > > > >> > >> > > > > > > tries
> > > > >> > >> > > > > > > > to
> > > > >> > >> > > > > > > > > >>>> figure out when UNKNOWN_PRODUCER_ID is
> > > > retriable
> > > > >> > and
> > > > >> > >> if
> > > > >> > >> > > it's
> > > > >> > >> > > > > > > > > retryable,
> > > > >> > >> > > > > > > > > >>>> it's definitely not fatal), but what
> > happens
> > > > >> when
> > > > >> > >> this
> > > > >> > >> > > logic
> > > > >> > >> > > > > > > doesn't
> > > > >> > >> > > > > > > > > >>> return
> > > > >> > >> > > > > > > > > >>>> 'true' and falls through.  In the old
> > > clients
> > > > it
> > > > >> > >> seems
> > > > >> > >> > to
> > > > >> > >> > > be
> > > > >> > >> > > > > > > fatal,
> > > > >> > >> > > > > > > > if
> > > > >> > >> > > > > > > > > >>> we
> > > > >> > >> > > > > > > > > >>>> keep the behavior in the new clients,
> I'd
> > > > >> expect it
> > > > >> > >> > would
> > > > >> > >> > > be
> > > > >> > >> > > > > > fatal
> > > > >> > >> > > > > > > > as
> > > > >> > >> > > > > > > > > >>> well.
> > > > >> > >> > > > > > > > > >>>>
> > > > >> > >> > > > > > > > > >>>> -Artem
> > > > >> > >> > > > > > > > > >>>>
> > > > >> > >> > > > > > > > > >>>> On Tue, Nov 29, 2022 at 11:57 AM Justine
> > > > Olshan
> > > > >> > >> > > > > > > > > >>>> <jo...@confluent.io.invalid> wrote:
> > > > >> > >> > > > > > > > > >>>>
> > > > >> > >> > > > > > > > > >>>>> Hi Artem and Jeff,
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>> Thanks for taking a look and sorry for
> > the
> > > > slow
> > > > >> > >> > response.
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>> You both mentioned the change to handle
> > > > >> > >> > > UNKNOWN_PRODUCER_ID
> > > > >> > >> > > > > > > errors.
> > > > >> > >> > > > > > > > > To
> > > > >> > >> > > > > > > > > >>> be
> > > > >> > >> > > > > > > > > >>>>> clear — this error code will only be
> sent
> > > > again
> > > > >> > when
> > > > >> > >> > the
> > > > >> > >> > > > > > client's
> > > > >> > >> > > > > > > > > >>> request
> > > > >> > >> > > > > > > > > >>>>> version is high enough to ensure we
> > handle
> > > it
> > > > >> > >> > correctly.
> > > > >> > >> > > > > > > > > >>>>> The current (Java) client handles this
> by
> > > the
> > > > >> > >> following
> > > > >> > >> > > > > > (somewhat
> > > > >> > >> > > > > > > > > long)
> > > > >> > >> > > > > > > > > >>>>> code snippet:
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>> // An UNKNOWN_PRODUCER_ID means that we
> > > have
> > > > >> lost
> > > > >> > >> the
> > > > >> > >> > > > > producer
> > > > >> > >> > > > > > > > state
> > > > >> > >> > > > > > > > > >>> on the
> > > > >> > >> > > > > > > > > >>>>> broker. Depending on the log start
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>> // offset, we may want to retry these,
> as
> > > > >> > described
> > > > >> > >> for
> > > > >> > >> > > > each
> > > > >> > >> > > > > > case
> > > > >> > >> > > > > > > > > >>> below. If
> > > > >> > >> > > > > > > > > >>>>> none of those apply, then for the
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>> // idempotent producer, we will locally
> > > bump
> > > > >> the
> > > > >> > >> epoch
> > > > >> > >> > > and
> > > > >> > >> > > > > > reset
> > > > >> > >> > > > > > > > the
> > > > >> > >> > > > > > > > > >>>>> sequence numbers of in-flight batches
> > from
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>> // sequence 0, then retry the failed
> > batch,
> > > > >> which
> > > > >> > >> > should
> > > > >> > >> > > > now
> > > > >> > >> > > > > > > > succeed.
> > > > >> > >> > > > > > > > > >>> For
> > > > >> > >> > > > > > > > > >>>>> the transactional producer, allow the
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>> // batch to fail. When processing the
> > > failed
> > > > >> > batch,
> > > > >> > >> we
> > > > >> > >> > > will
> > > > >> > >> > > > > > > > > transition
> > > > >> > >> > > > > > > > > >>> to
> > > > >> > >> > > > > > > > > >>>>> an abortable error and set a flag
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>> // indicating that we need to bump the
> > > epoch
> > > > >> (if
> > > > >> > >> > > supported
> > > > >> > >> > > > by
> > > > >> > >> > > > > > the
> > > > >> > >> > > > > > > > > >>> broker).
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>> if (error ==
> > Errors.*UNKNOWN_PRODUCER_ID*)
> > > {
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>       if (response.logStartOffset ==
> -1)
> > {
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           // We don't know the log
> start
> > > > offset
> > > > >> > with
> > > > >> > >> > this
> > > > >> > >> > > > > > > response.
> > > > >> > >> > > > > > > > > We
> > > > >> > >> > > > > > > > > >>> should
> > > > >> > >> > > > > > > > > >>>>> just retry the request until we get it.
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           // The UNKNOWN_PRODUCER_ID
> > error
> > > > code
> > > > >> > was
> > > > >> > >> > added
> > > > >> > >> > > > > along
> > > > >> > >> > > > > > > > with
> > > > >> > >> > > > > > > > > >>> the new
> > > > >> > >> > > > > > > > > >>>>> ProduceResponse which includes the
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           // logStartOffset. So the
> '-1'
> > > > >> sentinel
> > > > >> > is
> > > > >> > >> > not
> > > > >> > >> > > > for
> > > > >> > >> > > > > > > > backward
> > > > >> > >> > > > > > > > > >>>>> compatibility. Instead, it is possible
> > for
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           // a broker to not know the
> > > > >> > >> logStartOffset at
> > > > >> > >> > > > when
> > > > >> > >> > > > > it
> > > > >> > >> > > > > > > is
> > > > >> > >> > > > > > > > > >>> returning
> > > > >> > >> > > > > > > > > >>>>> the response because the partition
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           // may have moved away from
> the
> > > > >> broker
> > > > >> > >> from
> > > > >> > >> > the
> > > > >> > >> > > > > time
> > > > >> > >> > > > > > > the
> > > > >> > >> > > > > > > > > >>> error was
> > > > >> > >> > > > > > > > > >>>>> initially raised to the time the
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           // response was being
> > > constructed.
> > > > In
> > > > >> > >> these
> > > > >> > >> > > > cases,
> > > > >> > >> > > > > we
> > > > >> > >> > > > > > > > > should
> > > > >> > >> > > > > > > > > >>> just
> > > > >> > >> > > > > > > > > >>>>> retry the request: we are guaranteed
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           // to eventually get a
> > > > logStartOffset
> > > > >> > once
> > > > >> > >> > > things
> > > > >> > >> > > > > > > settle
> > > > >> > >> > > > > > > > > down.
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>       }
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>       if
> (batch.sequenceHasBeenReset()) {
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           // When the first inflight
> > batch
> > > > >> fails
> > > > >> > >> due to
> > > > >> > >> > > the
> > > > >> > >> > > > > > > > > truncation
> > > > >> > >> > > > > > > > > >>> case,
> > > > >> > >> > > > > > > > > >>>>> then the sequences of all the other
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           // in flight batches would
> have
> > > > been
> > > > >> > >> > restarted
> > > > >> > >> > > > from
> > > > >> > >> > > > > > the
> > > > >> > >> > > > > > > > > >>> beginning.
> > > > >> > >> > > > > > > > > >>>>> However, when those responses
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           // come back from the broker,
> > > they
> > > > >> would
> > > > >> > >> also
> > > > >> > >> > > > come
> > > > >> > >> > > > > > with
> > > > >> > >> > > > > > > > an
> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID error. In this
> case,
> > we
> > > > >> should
> > > > >> > >> not
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           // reset the sequence numbers
> > to
> > > > the
> > > > >> > >> > beginning.
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>       } else if
> > > > >> > >> > > > > (lastAckedOffset(batch.topicPartition).orElse(
> > > > >> > >> > > > > > > > > >>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > > > >> > >> > > > response.logStartOffset) {
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           // The head of the log has
> been
> > > > >> removed,
> > > > >> > >> > > probably
> > > > >> > >> > > > > due
> > > > >> > >> > > > > > > to
> > > > >> > >> > > > > > > > > the
> > > > >> > >> > > > > > > > > >>>>> retention time elapsing. In this case,
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           // we expect to lose the
> > producer
> > > > >> state.
> > > > >> > >> For
> > > > >> > >> > > the
> > > > >> > >> > > > > > > > > transactional
> > > > >> > >> > > > > > > > > >>>>> producer, reset the sequences of all
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           // inflight batches to be
> from
> > > the
> > > > >> > >> beginning
> > > > >> > >> > > and
> > > > >> > >> > > > > > retry
> > > > >> > >> > > > > > > > > them,
> > > > >> > >> > > > > > > > > >>> so
> > > > >> > >> > > > > > > > > >>>>> that the transaction does not need to
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           // be aborted. For the
> > idempotent
> > > > >> > >> producer,
> > > > >> > >> > > bump
> > > > >> > >> > > > > the
> > > > >> > >> > > > > > > > epoch
> > > > >> > >> > > > > > > > > to
> > > > >> > >> > > > > > > > > >>> avoid
> > > > >> > >> > > > > > > > > >>>>> reusing (sequence, epoch) pairs
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           if (isTransactional()) {
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > >
> > > > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > > > >> > >> > > > > > > > > >>>>> this.producerIdAndEpoch);
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           } else {
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > >
> requestEpochBumpForPartition(batch.topicPartition);
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           }
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>       }
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>       if (!isTransactional()) {
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           // For the idempotent
> producer,
> > > > >> always
> > > > >> > >> retry
> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > > >> > >> > > > > > > > > >>>>> errors. If the batch has the current
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           // producer ID and epoch,
> > > request a
> > > > >> bump
> > > > >> > >> of
> > > > >> > >> > the
> > > > >> > >> > > > > > epoch.
> > > > >> > >> > > > > > > > > >>> Otherwise
> > > > >> > >> > > > > > > > > >>>>> just retry the produce.
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > >  requestEpochBumpForPartition(batch.topicPartition);
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>           return true;
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>       }
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>> }
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>> I was considering keeping this
> behavior —
> > > but
> > > > >> am
> > > > >> > >> open
> > > > >> > >> > to
> > > > >> > >> > > > > > > > simplifying
> > > > >> > >> > > > > > > > > >>> it.
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>> We are leaving changes to older clients
> > off
> > > > the
> > > > >> > >> table
> > > > >> > >> > > here
> > > > >> > >> > > > > > since
> > > > >> > >> > > > > > > it
> > > > >> > >> > > > > > > > > >>> caused
> > > > >> > >> > > > > > > > > >>>>> many issues for clients in the past.
> > > > Previously
> > > > >> > this
> > > > >> > >> > was
> > > > >> > >> > > a
> > > > >> > >> > > > > > fatal
> > > > >> > >> > > > > > > > > error
> > > > >> > >> > > > > > > > > >>> and
> > > > >> > >> > > > > > > > > >>>>> we didn't have the mechanisms in place
> to
> > > > >> detect
> > > > >> > >> when
> > > > >> > >> > > this
> > > > >> > >> > > > > was
> > > > >> > >> > > > > > a
> > > > >> > >> > > > > > > > > >>> legitimate
> > > > >> > >> > > > > > > > > >>>>> case vs some bug or gap in the
> protocol.
> > > > >> Ensuring
> > > > >> > >> each
> > > > >> > >> > > > > > > transaction
> > > > >> > >> > > > > > > > > has
> > > > >> > >> > > > > > > > > >>> its
> > > > >> > >> > > > > > > > > >>>>> own epoch should close this gap.
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>> And to address Jeff's second point:
> > > > >> > >> > > > > > > > > >>>>> *does the typical produce request path
> > > append
> > > > >> > >> records
> > > > >> > >> > to
> > > > >> > >> > > > > local
> > > > >> > >> > > > > > > log
> > > > >> > >> > > > > > > > > >>> along*
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>> *with the currentTxnFirstOffset
> > > information?
> > > > I
> > > > >> > would
> > > > >> > >> > like
> > > > >> > >> > > > to
> > > > >> > >> > > > > > > > > >>> understand*
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>> *when the field is written to disk.*
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>> Yes, the first produce request
> populates
> > > this
> > > > >> > field
> > > > >> > >> and
> > > > >> > >> > > > > writes
> > > > >> > >> > > > > > > the
> > > > >> > >> > > > > > > > > >>> offset
> > > > >> > >> > > > > > > > > >>>>> as part of the record batch and also to
> > the
> > > > >> > producer
> > > > >> > >> > > state
> > > > >> > >> > > > > > > > snapshot.
> > > > >> > >> > > > > > > > > >>> When
> > > > >> > >> > > > > > > > > >>>>> we reload the records on restart and/or
> > > > >> > >> reassignment,
> > > > >> > >> > we
> > > > >> > >> > > > > > > repopulate
> > > > >> > >> > > > > > > > > >>> this
> > > > >> > >> > > > > > > > > >>>>> field with the snapshot from disk along
> > > with
> > > > >> the
> > > > >> > >> rest
> > > > >> > >> > of
> > > > >> > >> > > > the
> > > > >> > >> > > > > > > > producer
> > > > >> > >> > > > > > > > > >>>>> state.
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>> Let me know if there are further
> comments
> > > > >> and/or
> > > > >> > >> > > questions.
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>> Thanks,
> > > > >> > >> > > > > > > > > >>>>> Justine
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>> On Tue, Nov 22, 2022 at 9:00 PM Jeff
> Kim
> > > > >> > >> > > > > > > > > <jeff.kim@confluent.io.invalid
> > > > >> > >> > > > > > > > > >>>>
> > > > >> > >> > > > > > > > > >>>>> wrote:
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>>> Hi Justine,
> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > >> > > > > > > > > >>>>>> Thanks for the KIP! I have two
> > questions:
> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > >> > > > > > > > > >>>>>> 1) For new clients, we can once again
> > > return
> > > > >> an
> > > > >> > >> error
> > > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > > >> > >> > > > > > > > > >>>>>> for sequences
> > > > >> > >> > > > > > > > > >>>>>> that are non-zero when there is no
> > > producer
> > > > >> state
> > > > >> > >> > > present
> > > > >> > >> > > > on
> > > > >> > >> > > > > > the
> > > > >> > >> > > > > > > > > >>> server.
> > > > >> > >> > > > > > > > > >>>>>> This will indicate we missed the 0
> > > sequence
> > > > >> and
> > > > >> > we
> > > > >> > >> > don't
> > > > >> > >> > > > yet
> > > > >> > >> > > > > > > want
> > > > >> > >> > > > > > > > to
> > > > >> > >> > > > > > > > > >>>>> write
> > > > >> > >> > > > > > > > > >>>>>> to the log.
> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > >> > > > > > > > > >>>>>> I would like to understand the current
> > > > >> behavior
> > > > >> > to
> > > > >> > >> > > handle
> > > > >> > >> > > > > > older
> > > > >> > >> > > > > > > > > >>> clients,
> > > > >> > >> > > > > > > > > >>>>>> and if there are any changes we are
> > > making.
> > > > >> Maybe
> > > > >> > >> I'm
> > > > >> > >> > > > > missing
> > > > >> > >> > > > > > > > > >>> something,
> > > > >> > >> > > > > > > > > >>>>>> but we would want to identify whether
> we
> > > > >> missed
> > > > >> > >> the 0
> > > > >> > >> > > > > sequence
> > > > >> > >> > > > > > > for
> > > > >> > >> > > > > > > > > >>> older
> > > > >> > >> > > > > > > > > >>>>>> clients, no?
> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > >> > > > > > > > > >>>>>> 2) Upon returning from the transaction
> > > > >> > >> coordinator, we
> > > > >> > >> > > can
> > > > >> > >> > > > > set
> > > > >> > >> > > > > > > the
> > > > >> > >> > > > > > > > > >>>>>> transaction
> > > > >> > >> > > > > > > > > >>>>>> as ongoing on the leader by populating
> > > > >> > >> > > > currentTxnFirstOffset
> > > > >> > >> > > > > > > > > >>>>>> through the typical produce request
> > > > handling.
> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > >> > > > > > > > > >>>>>> does the typical produce request path
> > > append
> > > > >> > >> records
> > > > >> > >> > to
> > > > >> > >> > > > > local
> > > > >> > >> > > > > > > log
> > > > >> > >> > > > > > > > > >>> along
> > > > >> > >> > > > > > > > > >>>>>> with the currentTxnFirstOffset
> > > information?
> > > > I
> > > > >> > would
> > > > >> > >> > like
> > > > >> > >> > > > to
> > > > >> > >> > > > > > > > > understand
> > > > >> > >> > > > > > > > > >>>>>> when the field is written to disk.
> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > >> > > > > > > > > >>>>>> Thanks,
> > > > >> > >> > > > > > > > > >>>>>> Jeff
> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > >> > > > > > > > > >>>>>> On Tue, Nov 22, 2022 at 4:44 PM Artem
> > > > Livshits
> > > > >> > >> > > > > > > > > >>>>>> <al...@confluent.io.invalid>
> wrote:
> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > >> > > > > > > > > >>>>>>> Hi Justine,
> > > > >> > >> > > > > > > > > >>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>> Thank you for the KIP.  I have one
> > > > question.
> > > > >> > >> > > > > > > > > >>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>> 5) For new clients, we can once again
> > > > return
> > > > >> an
> > > > >> > >> error
> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > > >> > >> > > > > > > > > >>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>> I believe we had problems in the past
> > > with
> > > > >> > >> returning
> > > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > > >> > >> > > > > > > > > >>>>>>> because it was considered fatal and
> > > > required
> > > > >> > >> client
> > > > >> > >> > > > > restart.
> > > > >> > >> > > > > > > It
> > > > >> > >> > > > > > > > > >>> would
> > > > >> > >> > > > > > > > > >>>>> be
> > > > >> > >> > > > > > > > > >>>>>>> good to spell out the new client
> > behavior
> > > > >> when
> > > > >> > it
> > > > >> > >> > > > receives
> > > > >> > >> > > > > > the
> > > > >> > >> > > > > > > > > error.
> > > > >> > >> > > > > > > > > >>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>> -Artem
> > > > >> > >> > > > > > > > > >>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>> On Tue, Nov 22, 2022 at 10:00 AM
> > Justine
> > > > >> Olshan
> > > > >> > >> > > > > > > > > >>>>>>> <jo...@confluent.io.invalid>
> wrote:
> > > > >> > >> > > > > > > > > >>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>> Thanks for taking a look Matthias.
> > I've
> > > > >> tried
> > > > >> > to
> > > > >> > >> > > answer
> > > > >> > >> > > > > your
> > > > >> > >> > > > > > > > > >>>>> questions
> > > > >> > >> > > > > > > > > >>>>>>>> below:
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>> 10)
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>> Right — so the hanging transaction
> > only
> > > > >> occurs
> > > > >> > >> when
> > > > >> > >> > we
> > > > >> > >> > > > > have
> > > > >> > >> > > > > > a
> > > > >> > >> > > > > > > > late
> > > > >> > >> > > > > > > > > >>>>>>> message
> > > > >> > >> > > > > > > > > >>>>>>>> come in and the partition is never
> > added
> > > > to
> > > > >> a
> > > > >> > >> > > > transaction
> > > > >> > >> > > > > > > again.
> > > > >> > >> > > > > > > > > If
> > > > >> > >> > > > > > > > > >>>>> we
> > > > >> > >> > > > > > > > > >>>>>>>> never add the partition to a
> > > transaction,
> > > > we
> > > > >> > will
> > > > >> > >> > > never
> > > > >> > >> > > > > > write
> > > > >> > >> > > > > > > a
> > > > >> > >> > > > > > > > > >>>>> marker
> > > > >> > >> > > > > > > > > >>>>>>> and
> > > > >> > >> > > > > > > > > >>>>>>>> never advance the LSO.
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>> If we do end up adding the partition
> > to
> > > > the
> > > > >> > >> > > transaction
> > > > >> > >> > > > (I
> > > > >> > >> > > > > > > > suppose
> > > > >> > >> > > > > > > > > >>>>> this
> > > > >> > >> > > > > > > > > >>>>>>> can
> > > > >> > >> > > > > > > > > >>>>>>>> happen before or after the late
> > message
> > > > >> comes
> > > > >> > in)
> > > > >> > >> > then
> > > > >> > >> > > > we
> > > > >> > >> > > > > > will
> > > > >> > >> > > > > > > > > >>>>> include
> > > > >> > >> > > > > > > > > >>>>>>> the
> > > > >> > >> > > > > > > > > >>>>>>>> late message in the next (incorrect)
> > > > >> > transaction.
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>> So perhaps it is clearer to make the
> > > > >> > distinction
> > > > >> > >> > > between
> > > > >> > >> > > > > > > > messages
> > > > >> > >> > > > > > > > > >>>>> that
> > > > >> > >> > > > > > > > > >>>>>>>> eventually get added to the
> > transaction
> > > > (but
> > > > >> > the
> > > > >> > >> > wrong
> > > > >> > >> > > > > one)
> > > > >> > >> > > > > > or
> > > > >> > >> > > > > > > > > >>>>> messages
> > > > >> > >> > > > > > > > > >>>>>>>> that never get added and become
> > hanging.
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>> 20)
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>> The client side change for 2 is
> > removing
> > > > the
> > > > >> > >> > > > addPartitions
> > > > >> > >> > > > > > to
> > > > >> > >> > > > > > > > > >>>>>> transaction
> > > > >> > >> > > > > > > > > >>>>>>>> call. We don't need to make this
> from
> > > the
> > > > >> > >> producer
> > > > >> > >> > to
> > > > >> > >> > > > the
> > > > >> > >> > > > > > txn
> > > > >> > >> > > > > > > > > >>>>>>> coordinator,
> > > > >> > >> > > > > > > > > >>>>>>>> only server side.
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>> In my opinion, the issue with the
> > > > >> > >> addPartitionsToTxn
> > > > >> > >> > > > call
> > > > >> > >> > > > > > for
> > > > >> > >> > > > > > > > > older
> > > > >> > >> > > > > > > > > >>>>>>> clients
> > > > >> > >> > > > > > > > > >>>>>>>> is that we don't have the epoch
> bump,
> > so
> > > > we
> > > > >> > don't
> > > > >> > >> > know
> > > > >> > >> > > > if
> > > > >> > >> > > > > > the
> > > > >> > >> > > > > > > > > >>> message
> > > > >> > >> > > > > > > > > >>>>>>>> belongs to the previous transaction
> or
> > > > this
> > > > >> > one.
> > > > >> > >> We
> > > > >> > >> > > need
> > > > >> > >> > > > > to
> > > > >> > >> > > > > > > > check
> > > > >> > >> > > > > > > > > if
> > > > >> > >> > > > > > > > > >>>>>> the
> > > > >> > >> > > > > > > > > >>>>>>>> partition has been added to this
> > > > >> transaction.
> > > > >> > Of
> > > > >> > >> > > course,
> > > > >> > >> > > > > > this
> > > > >> > >> > > > > > > > > means
> > > > >> > >> > > > > > > > > >>>>> we
> > > > >> > >> > > > > > > > > >>>>>>>> won't completely cover the case
> where
> > we
> > > > >> have a
> > > > >> > >> > really
> > > > >> > >> > > > > late
> > > > >> > >> > > > > > > > > message
> > > > >> > >> > > > > > > > > >>>>> and
> > > > >> > >> > > > > > > > > >>>>>>> we
> > > > >> > >> > > > > > > > > >>>>>>>> have added the partition to the new
> > > > >> > transaction,
> > > > >> > >> but
> > > > >> > >> > > > > that's
> > > > >> > >> > > > > > > > > >>>>>> unfortunately
> > > > >> > >> > > > > > > > > >>>>>>>> something we will need the new
> clients
> > > to
> > > > >> > cover.
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>> 30)
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>> Transaction is ongoing = partition
> was
> > > > >> added to
> > > > >> > >> > > > > transaction
> > > > >> > >> > > > > > > via
> > > > >> > >> > > > > > > > > >>>>>>>> addPartitionsToTxn. We check this
> with
> > > the
> > > > >> > >> > > > > > > DescribeTransactions
> > > > >> > >> > > > > > > > > >>> call.
> > > > >> > >> > > > > > > > > >>>>>> Let
> > > > >> > >> > > > > > > > > >>>>>>>> me know if this wasn't sufficiently
> > > > >> explained
> > > > >> > >> here:
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > >
> > > > >> > >> > > > > > > >
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > >
> > > > >> > >> > > > >
> > > > >> > >> > > >
> > > > >> > >> > >
> > > > >> > >> >
> > > > >> > >>
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>> 40)
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>> The idea here is that if any
> messages
> > > > >> somehow
> > > > >> > >> come
> > > > >> > >> > in
> > > > >> > >> > > > > before
> > > > >> > >> > > > > > > we
> > > > >> > >> > > > > > > > > get
> > > > >> > >> > > > > > > > > >>>>> the
> > > > >> > >> > > > > > > > > >>>>>>> new
> > > > >> > >> > > > > > > > > >>>>>>>> epoch to the producer, they will be
> > > > fenced.
> > > > >> > >> However,
> > > > >> > >> > > if
> > > > >> > >> > > > we
> > > > >> > >> > > > > > > don't
> > > > >> > >> > > > > > > > > >>>>> think
> > > > >> > >> > > > > > > > > >>>>>>> this
> > > > >> > >> > > > > > > > > >>>>>>>> is necessary, it can be discussed
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>> 50)
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>> It should be synchronous because if
> we
> > > > have
> > > > >> an
> > > > >> > >> event
> > > > >> > >> > > > (ie,
> > > > >> > >> > > > > an
> > > > >> > >> > > > > > > > > error)
> > > > >> > >> > > > > > > > > >>>>>> that
> > > > >> > >> > > > > > > > > >>>>>>>> causes us to need to abort the
> > > > transaction,
> > > > >> we
> > > > >> > >> need
> > > > >> > >> > to
> > > > >> > >> > > > > know
> > > > >> > >> > > > > > > > which
> > > > >> > >> > > > > > > > > >>>>>>>> partitions to send transaction
> markers
> > > to.
> > > > >> We
> > > > >> > >> know
> > > > >> > >> > the
> > > > >> > >> > > > > > > > partitions
> > > > >> > >> > > > > > > > > >>>>>> because
> > > > >> > >> > > > > > > > > >>>>>>>> we added them to the coordinator via
> > the
> > > > >> > >> > > > > addPartitionsToTxn
> > > > >> > >> > > > > > > > call.
> > > > >> > >> > > > > > > > > >>>>>>>> Previously we have had asynchronous
> > > calls
> > > > in
> > > > >> > the
> > > > >> > >> > past
> > > > >> > >> > > > (ie,
> > > > >> > >> > > > > > > > writing
> > > > >> > >> > > > > > > > > >>>>> the
> > > > >> > >> > > > > > > > > >>>>>>>> commit markers when the transaction
> is
> > > > >> > completed)
> > > > >> > >> > but
> > > > >> > >> > > > > often
> > > > >> > >> > > > > > > this
> > > > >> > >> > > > > > > > > >>> just
> > > > >> > >> > > > > > > > > >>>>>>>> causes confusion as we need to wait
> > for
> > > > some
> > > > >> > >> > > operations
> > > > >> > >> > > > to
> > > > >> > >> > > > > > > > > complete.
> > > > >> > >> > > > > > > > > >>>>> In
> > > > >> > >> > > > > > > > > >>>>>>> the
> > > > >> > >> > > > > > > > > >>>>>>>> writing commit markers case, clients
> > > often
> > > > >> see
> > > > >> > >> > > > > > > > > >>>>> CONCURRENT_TRANSACTIONs
> > > > >> > >> > > > > > > > > >>>>>>>> error messages and that can be
> > > confusing.
> > > > >> For
> > > > >> > >> that
> > > > >> > >> > > > reason,
> > > > >> > >> > > > > > it
> > > > >> > >> > > > > > > > may
> > > > >> > >> > > > > > > > > be
> > > > >> > >> > > > > > > > > >>>>>>>> simpler to just have synchronous
> > calls —
> > > > >> > >> especially
> > > > >> > >> > if
> > > > >> > >> > > > we
> > > > >> > >> > > > > > need
> > > > >> > >> > > > > > > > to
> > > > >> > >> > > > > > > > > >>>>> block
> > > > >> > >> > > > > > > > > >>>>>>> on
> > > > >> > >> > > > > > > > > >>>>>>>> some operation's completion anyway
> > > before
> > > > we
> > > > >> > can
> > > > >> > >> > start
> > > > >> > >> > > > the
> > > > >> > >> > > > > > > next
> > > > >> > >> > > > > > > > > >>>>>>>> transaction. And yes, I meant
> > > > coordinator. I
> > > > >> > will
> > > > >> > >> > fix
> > > > >> > >> > > > > that.
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>> 60)
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>> When we are checking if the
> > transaction
> > > is
> > > > >> > >> ongoing,
> > > > >> > >> > we
> > > > >> > >> > > > > need
> > > > >> > >> > > > > > to
> > > > >> > >> > > > > > > > > make
> > > > >> > >> > > > > > > > > >>> a
> > > > >> > >> > > > > > > > > >>>>>>> round
> > > > >> > >> > > > > > > > > >>>>>>>> trip from the leader partition to
> the
> > > > >> > transaction
> > > > >> > >> > > > > > coordinator.
> > > > >> > >> > > > > > > > In
> > > > >> > >> > > > > > > > > >>> the
> > > > >> > >> > > > > > > > > >>>>>>> time
> > > > >> > >> > > > > > > > > >>>>>>>> we are waiting for this message to
> > come
> > > > >> back,
> > > > >> > in
> > > > >> > >> > > theory
> > > > >> > >> > > > we
> > > > >> > >> > > > > > > could
> > > > >> > >> > > > > > > > > >>> have
> > > > >> > >> > > > > > > > > >>>>>>> sent
> > > > >> > >> > > > > > > > > >>>>>>>> a commit/abort call that would make
> > the
> > > > >> > original
> > > > >> > >> > > result
> > > > >> > >> > > > of
> > > > >> > >> > > > > > the
> > > > >> > >> > > > > > > > > check
> > > > >> > >> > > > > > > > > >>>>>> out
> > > > >> > >> > > > > > > > > >>>>>>> of
> > > > >> > >> > > > > > > > > >>>>>>>> date. That is why we can check the
> > > leader
> > > > >> state
> > > > >> > >> > before
> > > > >> > >> > > > we
> > > > >> > >> > > > > > > write
> > > > >> > >> > > > > > > > to
> > > > >> > >> > > > > > > > > >>>>> the
> > > > >> > >> > > > > > > > > >>>>>>> log.
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>> I'm happy to update the KIP if some
> of
> > > > these
> > > > >> > >> things
> > > > >> > >> > > were
> > > > >> > >> > > > > not
> > > > >> > >> > > > > > > > > clear.
> > > > >> > >> > > > > > > > > >>>>>>>> Thanks,
> > > > >> > >> > > > > > > > > >>>>>>>> Justine
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>> On Mon, Nov 21, 2022 at 7:11 PM
> > Matthias
> > > > J.
> > > > >> > Sax <
> > > > >> > >> > > > > > > > mjsax@apache.org
> > > > >> > >> > > > > > > > > >
> > > > >> > >> > > > > > > > > >>>>>>> wrote:
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>> Thanks for the KIP.
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>> Couple of clarification questions
> (I
> > am
> > > > >> not a
> > > > >> > >> > broker
> > > > >> > >> > > > > expert
> > > > >> > >> > > > > > > do
> > > > >> > >> > > > > > > > > >>>>> maybe
> > > > >> > >> > > > > > > > > >>>>>>>>> some question are obvious for
> others,
> > > but
> > > > >> not
> > > > >> > >> for
> > > > >> > >> > me
> > > > >> > >> > > > with
> > > > >> > >> > > > > > my
> > > > >> > >> > > > > > > > lack
> > > > >> > >> > > > > > > > > >>>>> of
> > > > >> > >> > > > > > > > > >>>>>>>>> broker knowledge).
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>> (10)
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>> 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.
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>> What happens if the message come in
> > > > before
> > > > >> the
> > > > >> > >> next
> > > > >> > >> > > > > > > > > >>>>>> addPartitionsToTxn
> > > > >> > >> > > > > > > > > >>>>>>>>> request? It seems the broker
> hosting
> > > the
> > > > >> data
> > > > >> > >> > > > partitions
> > > > >> > >> > > > > > > won't
> > > > >> > >> > > > > > > > > know
> > > > >> > >> > > > > > > > > >>>>>>>>> anything about it and append it to
> > the
> > > > >> > >> partition,
> > > > >> > >> > > too?
> > > > >> > >> > > > > What
> > > > >> > >> > > > > > > is
> > > > >> > >> > > > > > > > > the
> > > > >> > >> > > > > > > > > >>>>>>>>> difference between both cases?
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>> Also, it seems a TX would only
> hang,
> > if
> > > > >> there
> > > > >> > >> is no
> > > > >> > >> > > > > > following
> > > > >> > >> > > > > > > > TX
> > > > >> > >> > > > > > > > > >>>>> that
> > > > >> > >> > > > > > > > > >>>>>>> is
> > > > >> > >> > > > > > > > > >>>>>>>>> either committer or aborted? Thus,
> > for
> > > > the
> > > > >> > case
> > > > >> > >> > > above,
> > > > >> > >> > > > > the
> > > > >> > >> > > > > > TX
> > > > >> > >> > > > > > > > > might
> > > > >> > >> > > > > > > > > >>>>>>>>> actually not hang (of course, we
> > might
> > > > get
> > > > >> an
> > > > >> > >> EOS
> > > > >> > >> > > > > violation
> > > > >> > >> > > > > > > if
> > > > >> > >> > > > > > > > > the
> > > > >> > >> > > > > > > > > >>>>>>> first
> > > > >> > >> > > > > > > > > >>>>>>>>> TX was aborted and the second
> > > committed,
> > > > or
> > > > >> > the
> > > > >> > >> > other
> > > > >> > >> > > > way
> > > > >> > >> > > > > > > > > around).
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>> (20)
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>> Of course, 1 and 2 require
> > client-side
> > > > >> > >> changes, so
> > > > >> > >> > > for
> > > > >> > >> > > > > > older
> > > > >> > >> > > > > > > > > >>>>>> clients,
> > > > >> > >> > > > > > > > > >>>>>>>>> those approaches won’t apply.
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>> For (1) I understand why a client
> > > change
> > > > is
> > > > >> > >> > > necessary,
> > > > >> > >> > > > > but
> > > > >> > >> > > > > > > not
> > > > >> > >> > > > > > > > > sure
> > > > >> > >> > > > > > > > > >>>>>> why
> > > > >> > >> > > > > > > > > >>>>>>>>> we need a client change for (2).
> Can
> > > you
> > > > >> > >> elaborate?
> > > > >> > >> > > --
> > > > >> > >> > > > > > Later
> > > > >> > >> > > > > > > > you
> > > > >> > >> > > > > > > > > >>>>>>> explain
> > > > >> > >> > > > > > > > > >>>>>>>>> that we should send a
> > > > >> > >> DescribeTransactionRequest,
> > > > >> > >> > > but I
> > > > >> > >> > > > > am
> > > > >> > >> > > > > > > not
> > > > >> > >> > > > > > > > > sure
> > > > >> > >> > > > > > > > > >>>>>>> why?
> > > > >> > >> > > > > > > > > >>>>>>>>> Can't we not just do an implicit
> > > > >> > >> AddPartiitonToTx,
> > > > >> > >> > > too?
> > > > >> > >> > > > > If
> > > > >> > >> > > > > > > the
> > > > >> > >> > > > > > > > > old
> > > > >> > >> > > > > > > > > >>>>>>>>> producer correctly registered the
> > > > partition
> > > > >> > >> > already,
> > > > >> > >> > > > the
> > > > >> > >> > > > > > > > > >>>>>> TX-coordinator
> > > > >> > >> > > > > > > > > >>>>>>>>> can just ignore it as it's an
> > > idempotent
> > > > >> > >> operation?
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>> (30)
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>> To cover older clients, we will
> > > ensure a
> > > > >> > >> > transaction
> > > > >> > >> > > > is
> > > > >> > >> > > > > > > > ongoing
> > > > >> > >> > > > > > > > > >>>>>>> before
> > > > >> > >> > > > > > > > > >>>>>>>>> we write to a transaction
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>> Not sure what you mean by this? Can
> > you
> > > > >> > >> elaborate?
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>> (40)
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>> [the TX-coordinator] will write
> the
> > > > >> prepare
> > > > >> > >> commit
> > > > >> > >> > > > > message
> > > > >> > >> > > > > > > > with
> > > > >> > >> > > > > > > > > a
> > > > >> > >> > > > > > > > > >>>>>>>> bumped
> > > > >> > >> > > > > > > > > >>>>>>>>> epoch and send
> WriteTxnMarkerRequests
> > > > with
> > > > >> the
> > > > >> > >> > bumped
> > > > >> > >> > > > > > epoch.
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>> Why do we use the bumped epoch for
> > > both?
> > > > It
> > > > >> > >> seems
> > > > >> > >> > > more
> > > > >> > >> > > > > > > > intuitive
> > > > >> > >> > > > > > > > > to
> > > > >> > >> > > > > > > > > >>>>>> use
> > > > >> > >> > > > > > > > > >>>>>>>>> the current epoch, and only return
> > the
> > > > >> bumped
> > > > >> > >> epoch
> > > > >> > >> > > to
> > > > >> > >> > > > > the
> > > > >> > >> > > > > > > > > >>>>> producer?
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>> (50) "Implicit
> > > AddPartitionToTransaction"
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>> Why does the implicitly sent
> request
> > > need
> > > > >> to
> > > > >> > be
> > > > >> > >> > > > > > synchronous?
> > > > >> > >> > > > > > > > The
> > > > >> > >> > > > > > > > > >>>>> KIP
> > > > >> > >> > > > > > > > > >>>>>>>>> also says
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>> in case we need to abort and need
> to
> > > > know
> > > > >> > which
> > > > >> > >> > > > > partitions
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>> What do you mean by this?
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>> we don’t want to write to it
> before
> > we
> > > > >> store
> > > > >> > in
> > > > >> > >> > the
> > > > >> > >> > > > > > > > transaction
> > > > >> > >> > > > > > > > > >>>>>>> manager
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>> Do you mean TX-coordinator instead
> of
> > > > >> > "manager"?
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>> (60)
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>> For older clients and ensuring that
> > the
> > > > TX
> > > > >> is
> > > > >> > >> > > ongoing,
> > > > >> > >> > > > > you
> > > > >> > >> > > > > > > > > >>>>> describe a
> > > > >> > >> > > > > > > > > >>>>>>>>> race condition. I am not sure if I
> > can
> > > > >> follow
> > > > >> > >> here.
> > > > >> > >> > > Can
> > > > >> > >> > > > > you
> > > > >> > >> > > > > > > > > >>>>>> elaborate?
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>> -Matthias
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>> On 11/18/22 1:21 PM, Justine Olshan
> > > > wrote:
> > > > >> > >> > > > > > > > > >>>>>>>>>> Hey all!
> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>> I'd like to start a discussion on
> my
> > > > >> proposal
> > > > >> > >> to
> > > > >> > >> > add
> > > > >> > >> > > > > some
> > > > >> > >> > > > > > > > > >>>>>> server-side
> > > > >> > >> > > > > > > > > >>>>>>>>>> checks on transactions to avoid
> > > hanging
> > > > >> > >> > > transactions.
> > > > >> > >> > > > I
> > > > >> > >> > > > > > know
> > > > >> > >> > > > > > > > > this
> > > > >> > >> > > > > > > > > >>>>>> has
> > > > >> > >> > > > > > > > > >>>>>>>>> been
> > > > >> > >> > > > > > > > > >>>>>>>>>> an issue for some time, so I
> really
> > > hope
> > > > >> this
> > > > >> > >> KIP
> > > > >> > >> > > will
> > > > >> > >> > > > > be
> > > > >> > >> > > > > > > > > helpful
> > > > >> > >> > > > > > > > > >>>>>> for
> > > > >> > >> > > > > > > > > >>>>>>>>> many
> > > > >> > >> > > > > > > > > >>>>>>>>>> users of EOS.
> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>> The KIP includes changes that will
> > be
> > > > >> > >> compatible
> > > > >> > >> > > with
> > > > >> > >> > > > > old
> > > > >> > >> > > > > > > > > clients
> > > > >> > >> > > > > > > > > >>>>>> and
> > > > >> > >> > > > > > > > > >>>>>>>>>> changes to improve performance and
> > > > >> > correctness
> > > > >> > >> on
> > > > >> > >> > > new
> > > > >> > >> > > > > > > clients.
> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>> Please take a look and leave any
> > > > comments
> > > > >> you
> > > > >> > >> may
> > > > >> > >> > > > have!
> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>> KIP:
> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > >
> > > > >> > >> > > > > > > >
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > >
> > > > >> > >> > > > >
> > > > >> > >> > > >
> > > > >> > >> > >
> > > > >> > >> >
> > > > >> > >>
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > >> > >> > > > > > > > > >>>>>>>>>> JIRA:
> > > > >> > >> > > > https://issues.apache.org/jira/browse/KAFKA-14402
> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>> Thanks!
> > > > >> > >> > > > > > > > > >>>>>>>>>> Justine
> > > > >> > >> > > > > > > > > >>>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>>
> > > > >> > >> > > > > > > > > >>>>>>
> > > > >> > >> > > > > > > > > >>>>>
> > > > >> > >> > > > > > > > > >>>>
> > > > >> > >> > > > > > > > > >>>
> > > > >> > >> > > > > > > > > >>
> > > > >> > >> > > > > > > > > >
> > > > >> > >> > > > > > > > >
> > > > >> > >> > > > > > > >
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > >
> > > > >> > >> > > > >
> > > > >> > >> > > >
> > > > >> > >> > >
> > > > >> > >> >
> > > > >> > >>
> > > > >> > >
> > > > >> >
> > > > >>
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Artem Livshits <al...@confluent.io.INVALID>.
It's good to know that KIP-588 addressed some of the issues.  Looking at
the code, it still looks like there are some cases that would result in
fatal error, e.g. PRODUCER_FENCED is issued by the transaction coordinator
if epoch doesn't match, and the client treats it as a fatal error (code in
TransactionManager request handling).  If we consider, for example,
committing a transaction that returns a timeout, but actually succeeds,
trying to abort it or re-commit may result in PRODUCER_FENCED error
(because of epoch bump).

For failed commits, specifically, we need to know the actual outcome,
because if we return an error the application may think that the
transaction is aborted and redo the work, leading to duplicates.

Re: overflowing epoch.  We could either do it on the TC and return both
producer id and epoch (e.g. change the protocol), or signal the client that
it needs to get a new producer id.  Checking for max epoch could be a
reasonable signal, the value to check should probably be present in the KIP
as this is effectively a part of the contract.  Also, the TC should
probably return an error if the client didn't change producer id after
hitting max epoch.

-Artem


On Thu, Jan 12, 2023 at 10:31 AM Justine Olshan
<jo...@confluent.io.invalid> wrote:

> Thanks for the discussion Artem.
>
> With respect to the handling of fenced producers, we have some behavior
> already in place. As of KIP-588:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> ,
> we handle timeouts more gracefully. The producer can recover.
>
> Produce requests can also recover from epoch fencing by aborting the
> transaction and starting over.
>
> What other cases were you considering that would cause us to have a fenced
> epoch but we'd want to recover?
>
> The first point about handling epoch overflows is fair. I think there is
> some logic we'd need to consider. (ie, if we are one away from the max
> epoch, we need to reset the producer ID.) I'm still wondering if there is a
> way to direct this from the response, or if everything should be done on
> the client side. Let me know if you have any thoughts here.
>
> Thanks,
> Justine
>
> On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
> <al...@confluent.io.invalid> wrote:
>
> > There are some workflows in the client that are implied by protocol
> > changes, e.g.:
> >
> > - for new clients, epoch changes with every transaction and can overflow,
> > in old clients this condition was handled transparently, because epoch
> was
> > bumped in InitProducerId and it would return a new producer id if epoch
> > overflows, the new clients would need to implement some workflow to
> refresh
> > producer id
> > - how to handle fenced producers, for new clients epoch changes with
> every
> > transaction, so in presence of failures during commits / aborts, the
> > producer could get easily fenced, old clients would pretty much would get
> > fenced when a new incarnation of the producer was initialized with
> > InitProducerId so it's ok to treat as a fatal error, the new clients
> would
> > need to implement some workflow to handle that error, otherwise they
> could
> > get fenced by themselves
> > - in particular (as a subset of the previous issue), what would the
> client
> > do if it got a timeout during commit?  commit could've succeeded or
> failed
> >
> > Not sure if this has to be defined in the KIP as implementing those
> > probably wouldn't require protocol changes, but we have multiple
> > implementations of Kafka clients, so probably would be good to have some
> > client implementation guidance.  Could also be done as a separate doc.
> >
> > -Artem
> >
> > On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan
> <jolshan@confluent.io.invalid
> > >
> > wrote:
> >
> > > Hey all, I've updated the KIP to incorporate Jason's suggestions.
> > >
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > >
> > >
> > > 1. Use AddPartitionsToTxn + verify flag to check on old clients
> > > 2. Updated AddPartitionsToTxn API to support transaction batching
> > > 3. Mention IBP bump
> > > 4. Mention auth change on new AddPartitionsToTxn version.
> > >
> > > I'm planning on opening a vote soon.
> > > Thanks,
> > > Justine
> > >
> > > On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <jo...@confluent.io>
> > > wrote:
> > >
> > > > Thanks Jason. Those changes make sense to me. I will update the KIP.
> > > >
> > > >
> > > >
> > > > On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
> > > <ja...@confluent.io.invalid>
> > > > wrote:
> > > >
> > > >> Hey Justine,
> > > >>
> > > >> > I was wondering about compatibility here. When we send requests
> > > >> between brokers, we want to ensure that the receiving broker
> > understands
> > > >> the request (specifically the new fields). Typically this is done
> via
> > > >> IBP/metadata version.
> > > >> I'm trying to think if there is a way around it but I'm not sure
> there
> > > is.
> > > >>
> > > >> Yes. I think we would gate usage of this behind an IBP bump. Does
> that
> > > >> seem
> > > >> reasonable?
> > > >>
> > > >> > As for the improvements -- can you clarify how the multiple
> > > >> transactional
> > > >> IDs would help here? Were you thinking of a case where we wait/batch
> > > >> multiple produce requests together? My understanding for now was 1
> > > >> transactional ID and one validation per 1 produce request.
> > > >>
> > > >> Each call to `AddPartitionsToTxn` is essentially a write to the
> > > >> transaction
> > > >> log and must block on replication. The more we can fit into a single
> > > >> request, the more writes we can do in parallel. The alternative is
> to
> > > make
> > > >> use of more connections, but usually we prefer batching since the
> > > network
> > > >> stack is not really optimized for high connection/request loads.
> > > >>
> > > >> > Finally with respect to the authorizations, I think it makes sense
> > to
> > > >> skip
> > > >> topic authorizations, but I'm a bit confused by the "leader ID"
> field.
> > > >> Wouldn't we just want to flag the request as from a broker (does it
> > > matter
> > > >> which one?).
> > > >>
> > > >> We could also make it version-based. For the next version, we could
> > > >> require
> > > >> CLUSTER auth. So clients would not be able to use the API anymore,
> > which
> > > >> is
> > > >> probably what we want.
> > > >>
> > > >> -Jason
> > > >>
> > > >> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> > > >> <jo...@confluent.io.invalid>
> > > >> wrote:
> > > >>
> > > >> > As a follow up, I was just thinking about the batching a bit more.
> > > >> > I suppose if we have one request in flight and we queue up the
> other
> > > >> > produce requests in some sort of purgatory, we could send
> > information
> > > >> out
> > > >> > for all of them rather than one by one. So that would be a benefit
> > of
> > > >> > batching partitions to add per transaction.
> > > >> >
> > > >> > I'll need to think a bit more on the design of this part of the
> KIP,
> > > and
> > > >> > will update the KIP in the next few days.
> > > >> >
> > > >> > Thanks,
> > > >> > Justine
> > > >> >
> > > >> > On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan <
> > jolshan@confluent.io>
> > > >> > wrote:
> > > >> >
> > > >> > > Hey Jason -- thanks for the input -- I was just digging a bit
> > deeper
> > > >> into
> > > >> > > the design + implementation of the validation calls here and
> what
> > > you
> > > >> say
> > > >> > > makes sense.
> > > >> > >
> > > >> > > I was wondering about compatibility here. When we send requests
> > > >> > > between brokers, we want to ensure that the receiving broker
> > > >> understands
> > > >> > > the request (specifically the new fields). Typically this is
> done
> > > via
> > > >> > > IBP/metadata version.
> > > >> > > I'm trying to think if there is a way around it but I'm not sure
> > > there
> > > >> > is.
> > > >> > >
> > > >> > > As for the improvements -- can you clarify how the multiple
> > > >> transactional
> > > >> > > IDs would help here? Were you thinking of a case where we
> > wait/batch
> > > >> > > multiple produce requests together? My understanding for now
> was 1
> > > >> > > transactional ID and one validation per 1 produce request.
> > > >> > >
> > > >> > > Finally with respect to the authorizations, I think it makes
> sense
> > > to
> > > >> > skip
> > > >> > > topic authorizations, but I'm a bit confused by the "leader ID"
> > > field.
> > > >> > > Wouldn't we just want to flag the request as from a broker (does
> > it
> > > >> > matter
> > > >> > > which one?).
> > > >> > >
> > > >> > > I think I want to adopt these suggestions, just had a few
> > questions
> > > on
> > > >> > the
> > > >> > > details.
> > > >> > >
> > > >> > > Thanks,
> > > >> > > Justine
> > > >> > >
> > > >> > > On Thu, Jan 5, 2023 at 5:05 PM Jason Gustafson
> > > >> > <ja...@confluent.io.invalid>
> > > >> > > wrote:
> > > >> > >
> > > >> > >> Hi Justine,
> > > >> > >>
> > > >> > >> Thanks for the proposal.
> > > >> > >>
> > > >> > >> I was thinking about the implementation a little bit. In the
> > > current
> > > >> > >> proposal, the behavior depends on whether we have an old or new
> > > >> client.
> > > >> > >> For
> > > >> > >> old clients, we send `DescribeTransactions` and verify the
> result
> > > and
> > > >> > for
> > > >> > >> new clients, we send `AddPartitionsToTxn`. We might be able to
> > > >> simplify
> > > >> > >> the
> > > >> > >> implementation if we can use the same request type. For
> example,
> > > >> what if
> > > >> > >> we
> > > >> > >> bump the protocol version for `AddPartitionsToTxn` and add a
> > > >> > >> `validateOnly`
> > > >> > >> flag? For older versions, we can set `validateOnly=true` so
> that
> > > the
> > > >> > >> request only returns successfully if the partition had already
> > been
> > > >> > added.
> > > >> > >> For new versions, we can set `validateOnly=false` and the
> > partition
> > > >> will
> > > >> > >> be
> > > >> > >> added to the transaction. The other slightly annoying thing
> that
> > > this
> > > >> > >> would
> > > >> > >> get around is the need to collect the transaction state for all
> > > >> > partitions
> > > >> > >> even when we only care about a subset.
> > > >> > >>
> > > >> > >> Some additional improvements to consider:
> > > >> > >>
> > > >> > >> - We can give `AddPartitionsToTxn` better batch support for
> > > >> inter-broker
> > > >> > >> usage. Currently we only allow one `TransactionalId` to be
> > > specified,
> > > >> > but
> > > >> > >> the broker may get some benefit being able to batch across
> > multiple
> > > >> > >> transactions.
> > > >> > >> - Another small improvement is skipping topic authorization
> > checks
> > > >> for
> > > >> > >> `AddPartitionsToTxn` when the request is from a broker. Perhaps
> > we
> > > >> can
> > > >> > add
> > > >> > >> a field for the `LeaderId` or something like that and require
> > > CLUSTER
> > > >> > >> permission when set.
> > > >> > >>
> > > >> > >> Best,
> > > >> > >> Jason
> > > >> > >>
> > > >> > >>
> > > >> > >>
> > > >> > >> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao
> <jun@confluent.io.invalid
> > >
> > > >> > wrote:
> > > >> > >>
> > > >> > >> > Hi, Justine,
> > > >> > >> >
> > > >> > >> > Thanks for the explanation. It makes sense to me now.
> > > >> > >> >
> > > >> > >> > Jun
> > > >> > >> >
> > > >> > >> > On Mon, Dec 19, 2022 at 1:42 PM Justine Olshan
> > > >> > >> > <jo...@confluent.io.invalid>
> > > >> > >> > wrote:
> > > >> > >> >
> > > >> > >> > > Hi Jun,
> > > >> > >> > >
> > > >> > >> > > My understanding of the mechanism is that when we get to
> the
> > > last
> > > >> > >> epoch,
> > > >> > >> > we
> > > >> > >> > > increment to the fencing/last epoch and if any further
> > requests
> > > >> come
> > > >> > >> in
> > > >> > >> > for
> > > >> > >> > > this producer ID they are fenced. Then the producer gets a
> > new
> > > ID
> > > >> > and
> > > >> > >> > > restarts with epoch/sequence 0. The fenced epoch sticks
> > around
> > > >> for
> > > >> > the
> > > >> > >> > > duration of producer.id.expiration.ms and blocks any late
> > > >> messages
> > > >> > >> > there.
> > > >> > >> > > The new ID will get to take advantage of the improved
> > semantics
> > > >> > around
> > > >> > >> > > non-zero start sequences. So I think we are covered.
> > > >> > >> > >
> > > >> > >> > > The only potential issue is overloading the cache, but
> > > hopefully
> > > >> the
> > > >> > >> > > improvements (lowered producer.id.expiration.ms) will help
> > > with
> > > >> > that.
> > > >> > >> > Let
> > > >> > >> > > me know if you still have concerns.
> > > >> > >> > >
> > > >> > >> > > Thanks,
> > > >> > >> > > Justine
> > > >> > >> > >
> > > >> > >> > > On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
> > > >> <ju...@confluent.io.invalid>
> > > >> > >> > wrote:
> > > >> > >> > >
> > > >> > >> > > > Hi, Justine,
> > > >> > >> > > >
> > > >> > >> > > > Thanks for the explanation.
> > > >> > >> > > >
> > > >> > >> > > > 70. The proposed fencing logic doesn't apply when pid
> > > changes,
> > > >> is
> > > >> > >> that
> > > >> > >> > > > right? If so, I am not sure how complete we are
> addressing
> > > this
> > > >> > >> issue
> > > >> > >> > if
> > > >> > >> > > > the pid changes more frequently.
> > > >> > >> > > >
> > > >> > >> > > > Thanks,
> > > >> > >> > > >
> > > >> > >> > > > Jun
> > > >> > >> > > >
> > > >> > >> > > >
> > > >> > >> > > >
> > > >> > >> > > > On Fri, Dec 16, 2022 at 9:16 AM Justine Olshan
> > > >> > >> > > > <jo...@confluent.io.invalid>
> > > >> > >> > > > wrote:
> > > >> > >> > > >
> > > >> > >> > > > > Hi Jun,
> > > >> > >> > > > >
> > > >> > >> > > > > Thanks for replying!
> > > >> > >> > > > >
> > > >> > >> > > > > 70.We already do the overflow mechanism, so my change
> > would
> > > >> just
> > > >> > >> make
> > > >> > >> > > it
> > > >> > >> > > > > happen more often.
> > > >> > >> > > > > I was also not suggesting a new field in the log, but
> in
> > > the
> > > >> > >> > response,
> > > >> > >> > > > > which would be gated by the client version. Sorry if
> > > >> something
> > > >> > >> there
> > > >> > >> > is
> > > >> > >> > > > > unclear. I think we are starting to diverge.
> > > >> > >> > > > > The goal of this KIP is to not change to the marker
> > format
> > > at
> > > >> > all.
> > > >> > >> > > > >
> > > >> > >> > > > > 71. Yes, I guess I was going under the assumption that
> > the
> > > >> log
> > > >> > >> would
> > > >> > >> > > just
> > > >> > >> > > > > look at its last epoch and treat it as the current
> > epoch. I
> > > >> > >> suppose
> > > >> > >> > we
> > > >> > >> > > > can
> > > >> > >> > > > > have some special logic that if the last epoch was on a
> > > >> marker
> > > >> > we
> > > >> > >> > > > actually
> > > >> > >> > > > > expect the next epoch or something like that. We just
> > need
> > > to
> > > >> > >> > > distinguish
> > > >> > >> > > > > based on whether we had a commit/abort marker.
> > > >> > >> > > > >
> > > >> > >> > > > > 72.
> > > >> > >> > > > > > if the producer epoch hasn't been bumped on the
> > > >> > >> > > > > broker, it seems that the stucked message will fail the
> > > >> sequence
> > > >> > >> > > > validation
> > > >> > >> > > > > and will be ignored. If the producer epoch has been
> > bumped,
> > > >> we
> > > >> > >> ignore
> > > >> > >> > > the
> > > >> > >> > > > > sequence check and the stuck message could be appended
> to
> > > the
> > > >> > log.
> > > >> > >> > So,
> > > >> > >> > > is
> > > >> > >> > > > > the latter case that we want to guard?
> > > >> > >> > > > >
> > > >> > >> > > > > I'm not sure I follow that "the message will fail the
> > > >> sequence
> > > >> > >> > > > validation".
> > > >> > >> > > > > In some of these cases, we had an abort marker (due to
> an
> > > >> error)
> > > >> > >> and
> > > >> > >> > > then
> > > >> > >> > > > > the late message comes in with the correct sequence
> > number.
> > > >> This
> > > >> > >> is a
> > > >> > >> > > > case
> > > >> > >> > > > > covered by the KIP.
> > > >> > >> > > > > The latter case is actually not something we've
> > considered
> > > >> > here. I
> > > >> > >> > > think
> > > >> > >> > > > > generally when we bump the epoch, we are accepting that
> > the
> > > >> > >> sequence
> > > >> > >> > > does
> > > >> > >> > > > > not need to be checked anymore. My understanding is
> also
> > > >> that we
> > > >> > >> > don't
> > > >> > >> > > > > typically bump epoch mid transaction (based on a quick
> > look
> > > >> at
> > > >> > the
> > > >> > >> > > code)
> > > >> > >> > > > > but let me know if that is the case.
> > > >> > >> > > > >
> > > >> > >> > > > > Thanks,
> > > >> > >> > > > > Justine
> > > >> > >> > > > >
> > > >> > >> > > > > On Thu, Dec 15, 2022 at 12:23 PM Jun Rao
> > > >> > <jun@confluent.io.invalid
> > > >> > >> >
> > > >> > >> > > > wrote:
> > > >> > >> > > > >
> > > >> > >> > > > > > Hi, Justine,
> > > >> > >> > > > > >
> > > >> > >> > > > > > Thanks for the reply.
> > > >> > >> > > > > >
> > > >> > >> > > > > > 70. Assigning a new pid on int overflow seems a bit
> > > hacky.
> > > >> If
> > > >> > we
> > > >> > >> > > need a
> > > >> > >> > > > > txn
> > > >> > >> > > > > > level id, it will be better to model this explicitly.
> > > >> Adding a
> > > >> > >> new
> > > >> > >> > > > field
> > > >> > >> > > > > > would require a bit more work since it requires a new
> > txn
> > > >> > marker
> > > >> > >> > > format
> > > >> > >> > > > > in
> > > >> > >> > > > > > the log. So, we probably need to guard it with an IBP
> > or
> > > >> > >> metadata
> > > >> > >> > > > version
> > > >> > >> > > > > > and document the impact on downgrade once the new
> > format
> > > is
> > > >> > >> written
> > > >> > >> > > to
> > > >> > >> > > > > the
> > > >> > >> > > > > > log.
> > > >> > >> > > > > >
> > > >> > >> > > > > > 71. Hmm, once the marker is written, the partition
> will
> > > >> expect
> > > >> > >> the
> > > >> > >> > > next
> > > >> > >> > > > > > append to be on the next epoch. Does that cover the
> > case
> > > >> you
> > > >> > >> > > mentioned?
> > > >> > >> > > > > >
> > > >> > >> > > > > > 72. Also, just to be clear on the stucked message
> issue
> > > >> > >> described
> > > >> > >> > in
> > > >> > >> > > > the
> > > >> > >> > > > > > motivation. With EoS, we also validate the sequence
> id
> > > for
> > > >> > >> > > idempotency.
> > > >> > >> > > > > So,
> > > >> > >> > > > > > with the current logic, if the producer epoch hasn't
> > been
> > > >> > >> bumped on
> > > >> > >> > > the
> > > >> > >> > > > > > broker, it seems that the stucked message will fail
> the
> > > >> > sequence
> > > >> > >> > > > > validation
> > > >> > >> > > > > > and will be ignored. If the producer epoch has been
> > > >> bumped, we
> > > >> > >> > ignore
> > > >> > >> > > > the
> > > >> > >> > > > > > sequence check and the stuck message could be
> appended
> > to
> > > >> the
> > > >> > >> log.
> > > >> > >> > > So,
> > > >> > >> > > > is
> > > >> > >> > > > > > the latter case that we want to guard?
> > > >> > >> > > > > >
> > > >> > >> > > > > > Thanks,
> > > >> > >> > > > > >
> > > >> > >> > > > > > Jun
> > > >> > >> > > > > >
> > > >> > >> > > > > > On Wed, Dec 14, 2022 at 10:44 AM Justine Olshan
> > > >> > >> > > > > > <jo...@confluent.io.invalid> wrote:
> > > >> > >> > > > > >
> > > >> > >> > > > > > > Matthias — thanks again for taking time to look a
> > this.
> > > >> You
> > > >> > >> said:
> > > >> > >> > > > > > >
> > > >> > >> > > > > > > > My proposal was only focusing to avoid dangling
> > > >> > >> > > > > > >
> > > >> > >> > > > > > > transactions if records are added without
> registered
> > > >> > >> partition.
> > > >> > >> > --
> > > >> > >> > > > > Maybe
> > > >> > >> > > > > > >
> > > >> > >> > > > > > > you can add a few more details to the KIP about
> this
> > > >> > scenario
> > > >> > >> for
> > > >> > >> > > > > better
> > > >> > >> > > > > > >
> > > >> > >> > > > > > > documentation purpose?
> > > >> > >> > > > > > >
> > > >> > >> > > > > > >
> > > >> > >> > > > > > > I'm not sure I understand what you mean here. The
> > > >> motivation
> > > >> > >> > > section
> > > >> > >> > > > > > > describes two scenarios about how the record can be
> > > added
> > > >> > >> > without a
> > > >> > >> > > > > > > registered partition:
> > > >> > >> > > > > > >
> > > >> > >> > > > > > >
> > > >> > >> > > > > > > > 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.
> > > >> > >> > > > > > >
> > > >> > >> > > > > > >
> > > >> > >> > > > > > > > Another way hanging transactions can occur is
> that
> > a
> > > >> > client
> > > >> > >> is
> > > >> > >> > > > buggy
> > > >> > >> > > > > > and
> > > >> > >> > > > > > > may somehow try to write to a partition before it
> > adds
> > > >> the
> > > >> > >> > > partition
> > > >> > >> > > > to
> > > >> > >> > > > > > the
> > > >> > >> > > > > > > transaction.
> > > >> > >> > > > > > >
> > > >> > >> > > > > > >
> > > >> > >> > > > > > >
> > > >> > >> > > > > > > For the first example of this would it be helpful
> to
> > > say
> > > >> > that
> > > >> > >> > this
> > > >> > >> > > > > > message
> > > >> > >> > > > > > > comes in after the abort, but before the partition
> is
> > > >> added
> > > >> > to
> > > >> > >> > the
> > > >> > >> > > > next
> > > >> > >> > > > > > > transaction so it becomes "hanging." Perhaps the
> next
> > > >> > sentence
> > > >> > >> > > > > describing
> > > >> > >> > > > > > > the message becoming part of the next transaction
> (a
> > > >> > different
> > > >> > >> > > case)
> > > >> > >> > > > > was
> > > >> > >> > > > > > > not properly differentiated.
> > > >> > >> > > > > > >
> > > >> > >> > > > > > >
> > > >> > >> > > > > > >
> > > >> > >> > > > > > > Jun — thanks for reading the KIP.
> > > >> > >> > > > > > >
> > > >> > >> > > > > > > 70. The int typing was a concern. Currently we
> have a
> > > >> > >> mechanism
> > > >> > >> > in
> > > >> > >> > > > > place
> > > >> > >> > > > > > to
> > > >> > >> > > > > > > fence the final epoch when the epoch is about to
> > > overflow
> > > >> > and
> > > >> > >> > > assign
> > > >> > >> > > > a
> > > >> > >> > > > > > new
> > > >> > >> > > > > > > producer ID with epoch 0. Of course, this is a bit
> > > tricky
> > > >> > >> when it
> > > >> > >> > > > comes
> > > >> > >> > > > > > to
> > > >> > >> > > > > > > the response back to the client.
> > > >> > >> > > > > > > Making this a long could be another option, but I
> > > wonder
> > > >> are
> > > >> > >> > there
> > > >> > >> > > > any
> > > >> > >> > > > > > > implications on changing this field if the epoch is
> > > >> > persisted
> > > >> > >> to
> > > >> > >> > > > disk?
> > > >> > >> > > > > > I'd
> > > >> > >> > > > > > > need to check the usages.
> > > >> > >> > > > > > >
> > > >> > >> > > > > > > 71.This was something Matthias asked about as
> well. I
> > > was
> > > >> > >> > > > considering a
> > > >> > >> > > > > > > possible edge case where a produce request from a
> new
> > > >> > >> transaction
> > > >> > >> > > > > somehow
> > > >> > >> > > > > > > gets sent right after the marker is written, but
> > before
> > > >> the
> > > >> > >> > > producer
> > > >> > >> > > > is
> > > >> > >> > > > > > > alerted of the newly bumped epoch. In this case, we
> > may
> > > >> > >> include
> > > >> > >> > > this
> > > >> > >> > > > > > record
> > > >> > >> > > > > > > when we don't want to. I suppose we could try to do
> > > >> > something
> > > >> > >> > > client
> > > >> > >> > > > > side
> > > >> > >> > > > > > > to bump the epoch after sending an endTxn as well
> in
> > > this
> > > >> > >> > scenario
> > > >> > >> > > —
> > > >> > >> > > > > but
> > > >> > >> > > > > > I
> > > >> > >> > > > > > > wonder how it would work when the server is
> aborting
> > > >> based
> > > >> > on
> > > >> > >> a
> > > >> > >> > > > > > server-side
> > > >> > >> > > > > > > error. I could also be missing something and this
> > > >> scenario
> > > >> > is
> > > >> > >> > > > actually
> > > >> > >> > > > > > not
> > > >> > >> > > > > > > possible.
> > > >> > >> > > > > > >
> > > >> > >> > > > > > > Thanks again to everyone reading and commenting.
> Let
> > me
> > > >> know
> > > >> > >> > about
> > > >> > >> > > > any
> > > >> > >> > > > > > > further questions or comments.
> > > >> > >> > > > > > >
> > > >> > >> > > > > > > Justine
> > > >> > >> > > > > > >
> > > >> > >> > > > > > > On Wed, Dec 14, 2022 at 9:41 AM Jun Rao
> > > >> > >> <jun@confluent.io.invalid
> > > >> > >> > >
> > > >> > >> > > > > > wrote:
> > > >> > >> > > > > > >
> > > >> > >> > > > > > > > Hi, Justine,
> > > >> > >> > > > > > > >
> > > >> > >> > > > > > > > Thanks for the KIP. A couple of comments.
> > > >> > >> > > > > > > >
> > > >> > >> > > > > > > > 70. Currently, the producer epoch is an int. I am
> > not
> > > >> sure
> > > >> > >> if
> > > >> > >> > > it's
> > > >> > >> > > > > > enough
> > > >> > >> > > > > > > > to accommodate all transactions in the lifetime
> of
> > a
> > > >> > >> producer.
> > > >> > >> > > > Should
> > > >> > >> > > > > > we
> > > >> > >> > > > > > > > change that to a long or add a new long field
> like
> > > >> txnId?
> > > >> > >> > > > > > > >
> > > >> > >> > > > > > > > 71. "it will write the prepare commit message
> with
> > a
> > > >> > bumped
> > > >> > >> > epoch
> > > >> > >> > > > and
> > > >> > >> > > > > > > send
> > > >> > >> > > > > > > > WriteTxnMarkerRequests with the bumped epoch."
> Hmm,
> > > the
> > > >> > >> epoch
> > > >> > >> > is
> > > >> > >> > > > > > > associated
> > > >> > >> > > > > > > > with the current txn right? So, it seems weird to
> > > >> write a
> > > >> > >> > commit
> > > >> > >> > > > > > message
> > > >> > >> > > > > > > > with a bumped epoch. Should we only bump up the
> > epoch
> > > >> in
> > > >> > >> > > > > EndTxnResponse
> > > >> > >> > > > > > > and
> > > >> > >> > > > > > > > rename the field to sth like nextProducerEpoch?
> > > >> > >> > > > > > > >
> > > >> > >> > > > > > > > Thanks,
> > > >> > >> > > > > > > >
> > > >> > >> > > > > > > > Jun
> > > >> > >> > > > > > > >
> > > >> > >> > > > > > > >
> > > >> > >> > > > > > > >
> > > >> > >> > > > > > > > On Mon, Dec 12, 2022 at 8:54 PM Matthias J. Sax <
> > > >> > >> > > mjsax@apache.org>
> > > >> > >> > > > > > > wrote:
> > > >> > >> > > > > > > >
> > > >> > >> > > > > > > > > Thanks for the background.
> > > >> > >> > > > > > > > >
> > > >> > >> > > > > > > > > 20/30: SGTM. My proposal was only focusing to
> > avoid
> > > >> > >> dangling
> > > >> > >> > > > > > > > > transactions if records are added without
> > > registered
> > > >> > >> > partition.
> > > >> > >> > > > --
> > > >> > >> > > > > > > Maybe
> > > >> > >> > > > > > > > > you can add a few more details to the KIP about
> > > this
> > > >> > >> scenario
> > > >> > >> > > for
> > > >> > >> > > > > > > better
> > > >> > >> > > > > > > > > documentation purpose?
> > > >> > >> > > > > > > > >
> > > >> > >> > > > > > > > > 40: I think you hit a fair point about race
> > > >> conditions
> > > >> > or
> > > >> > >> > > client
> > > >> > >> > > > > bugs
> > > >> > >> > > > > > > > > (incorrectly not bumping the epoch). The
> > > >> > >> complexity/confusion
> > > >> > >> > > for
> > > >> > >> > > > > > using
> > > >> > >> > > > > > > > > the bumped epoch I see, is mainly for internal
> > > >> > debugging,
> > > >> > >> ie,
> > > >> > >> > > > > > > inspecting
> > > >> > >> > > > > > > > > log segment dumps -- it seems harder to reason
> > > about
> > > >> the
> > > >> > >> > system
> > > >> > >> > > > for
> > > >> > >> > > > > > us
> > > >> > >> > > > > > > > > humans. But if we get better guarantees, it
> would
> > > be
> > > >> > >> worth to
> > > >> > >> > > use
> > > >> > >> > > > > the
> > > >> > >> > > > > > > > > bumped epoch.
> > > >> > >> > > > > > > > >
> > > >> > >> > > > > > > > > 60: as I mentioned already, I don't know the
> > broker
> > > >> > >> internals
> > > >> > >> > > to
> > > >> > >> > > > > > > provide
> > > >> > >> > > > > > > > > more input. So if nobody else chimes in, we
> > should
> > > >> just
> > > >> > >> move
> > > >> > >> > > > > forward
> > > >> > >> > > > > > > > > with your proposal.
> > > >> > >> > > > > > > > >
> > > >> > >> > > > > > > > >
> > > >> > >> > > > > > > > > -Matthias
> > > >> > >> > > > > > > > >
> > > >> > >> > > > > > > > >
> > > >> > >> > > > > > > > > On 12/6/22 4:22 PM, Justine Olshan wrote:
> > > >> > >> > > > > > > > > > Hi all,
> > > >> > >> > > > > > > > > > After Artem's questions about error behavior,
> > > I've
> > > >> > >> > > re-evaluated
> > > >> > >> > > > > the
> > > >> > >> > > > > > > > > > unknown producer ID exception and had some
> > > >> discussions
> > > >> > >> > > offline.
> > > >> > >> > > > > > > > > >
> > > >> > >> > > > > > > > > > I think generally it makes sense to simplify
> > > error
> > > >> > >> handling
> > > >> > >> > > in
> > > >> > >> > > > > > cases
> > > >> > >> > > > > > > > like
> > > >> > >> > > > > > > > > > this and the UNKNOWN_PRODUCER_ID error has a
> > > pretty
> > > >> > long
> > > >> > >> > and
> > > >> > >> > > > > > > > complicated
> > > >> > >> > > > > > > > > > history. Because of this, I propose adding a
> > new
> > > >> error
> > > >> > >> code
> > > >> > >> > > > > > > > > ABORTABLE_ERROR
> > > >> > >> > > > > > > > > > that when encountered by new clients (gated
> by
> > > the
> > > >> > >> produce
> > > >> > >> > > > > request
> > > >> > >> > > > > > > > > version)
> > > >> > >> > > > > > > > > > will simply abort the transaction. This
> allows
> > > the
> > > >> > >> server
> > > >> > >> > to
> > > >> > >> > > > have
> > > >> > >> > > > > > > some
> > > >> > >> > > > > > > > > say
> > > >> > >> > > > > > > > > > in whether the client aborts and makes
> handling
> > > >> much
> > > >> > >> > simpler.
> > > >> > >> > > > In
> > > >> > >> > > > > > the
> > > >> > >> > > > > > > > > > future, we can also use this error in other
> > > >> situations
> > > >> > >> > where
> > > >> > >> > > we
> > > >> > >> > > > > > want
> > > >> > >> > > > > > > to
> > > >> > >> > > > > > > > > > abort the transactions. We can even use on
> > other
> > > >> apis.
> > > >> > >> > > > > > > > > >
> > > >> > >> > > > > > > > > > I've added this to the KIP. Let me know if
> > there
> > > >> are
> > > >> > any
> > > >> > >> > > > > questions
> > > >> > >> > > > > > or
> > > >> > >> > > > > > > > > > issues.
> > > >> > >> > > > > > > > > >
> > > >> > >> > > > > > > > > > Justine
> > > >> > >> > > > > > > > > >
> > > >> > >> > > > > > > > > > On Fri, Dec 2, 2022 at 10:22 AM Justine
> Olshan
> > <
> > > >> > >> > > > > > jolshan@confluent.io
> > > >> > >> > > > > > > >
> > > >> > >> > > > > > > > > wrote:
> > > >> > >> > > > > > > > > >
> > > >> > >> > > > > > > > > >> Hey Matthias,
> > > >> > >> > > > > > > > > >>
> > > >> > >> > > > > > > > > >>
> > > >> > >> > > > > > > > > >> 20/30 — Maybe I also didn't express myself
> > > >> clearly.
> > > >> > For
> > > >> > >> > > older
> > > >> > >> > > > > > > clients
> > > >> > >> > > > > > > > we
> > > >> > >> > > > > > > > > >> don't have a way to distinguish between a
> > > previous
> > > >> > and
> > > >> > >> the
> > > >> > >> > > > > current
> > > >> > >> > > > > > > > > >> transaction since we don't have the epoch
> > bump.
> > > >> This
> > > >> > >> means
> > > >> > >> > > > that
> > > >> > >> > > > > a
> > > >> > >> > > > > > > late
> > > >> > >> > > > > > > > > >> message from the previous transaction may be
> > > >> added to
> > > >> > >> the
> > > >> > >> > > new
> > > >> > >> > > > > one.
> > > >> > >> > > > > > > > With
> > > >> > >> > > > > > > > > >> older clients — we can't guarantee this
> won't
> > > >> happen
> > > >> > >> if we
> > > >> > >> > > > > already
> > > >> > >> > > > > > > > sent
> > > >> > >> > > > > > > > > the
> > > >> > >> > > > > > > > > >> addPartitionsToTxn call (why we make changes
> > for
> > > >> the
> > > >> > >> newer
> > > >> > >> > > > > client)
> > > >> > >> > > > > > > but
> > > >> > >> > > > > > > > > we
> > > >> > >> > > > > > > > > >> can at least gate some by ensuring that the
> > > >> partition
> > > >> > >> has
> > > >> > >> > > been
> > > >> > >> > > > > > added
> > > >> > >> > > > > > > > to
> > > >> > >> > > > > > > > > the
> > > >> > >> > > > > > > > > >> transaction. The rationale here is that
> there
> > > are
> > > >> > >> likely
> > > >> > >> > > LESS
> > > >> > >> > > > > late
> > > >> > >> > > > > > > > > arrivals
> > > >> > >> > > > > > > > > >> as time goes on, so hopefully most late
> > arrivals
> > > >> will
> > > >> > >> come
> > > >> > >> > > in
> > > >> > >> > > > > > BEFORE
> > > >> > >> > > > > > > > the
> > > >> > >> > > > > > > > > >> addPartitionsToTxn call. Those that arrive
> > > before
> > > >> > will
> > > >> > >> be
> > > >> > >> > > > > properly
> > > >> > >> > > > > > > > gated
> > > >> > >> > > > > > > > > >> with the describeTransactions approach.
> > > >> > >> > > > > > > > > >>
> > > >> > >> > > > > > > > > >> If we take the approach you suggested, ANY
> > late
> > > >> > arrival
> > > >> > >> > > from a
> > > >> > >> > > > > > > > previous
> > > >> > >> > > > > > > > > >> transaction will be added. And we don't want
> > > >> that. I
> > > >> > >> also
> > > >> > >> > > > don't
> > > >> > >> > > > > > see
> > > >> > >> > > > > > > > any
> > > >> > >> > > > > > > > > >> benefit in sending addPartitionsToTxn over
> the
> > > >> > >> > describeTxns
> > > >> > >> > > > > call.
> > > >> > >> > > > > > > They
> > > >> > >> > > > > > > > > will
> > > >> > >> > > > > > > > > >> both be one extra RPC to the Txn
> coordinator.
> > > >> > >> > > > > > > > > >>
> > > >> > >> > > > > > > > > >>
> > > >> > >> > > > > > > > > >> To be clear — newer clients will use
> > > >> > addPartitionsToTxn
> > > >> > >> > > > instead
> > > >> > >> > > > > of
> > > >> > >> > > > > > > the
> > > >> > >> > > > > > > > > >> DescribeTxns.
> > > >> > >> > > > > > > > > >>
> > > >> > >> > > > > > > > > >>
> > > >> > >> > > > > > > > > >> 40)
> > > >> > >> > > > > > > > > >> My concern is that if we have some delay in
> > the
> > > >> > client
> > > >> > >> to
> > > >> > >> > > bump
> > > >> > >> > > > > the
> > > >> > >> > > > > > > > > epoch,
> > > >> > >> > > > > > > > > >> it could continue to send epoch 73 and those
> > > >> records
> > > >> > >> would
> > > >> > >> > > not
> > > >> > >> > > > > be
> > > >> > >> > > > > > > > > fenced.
> > > >> > >> > > > > > > > > >> Perhaps this is not an issue if we don't
> allow
> > > the
> > > >> > next
> > > >> > >> > > > produce
> > > >> > >> > > > > to
> > > >> > >> > > > > > > go
> > > >> > >> > > > > > > > > >> through before the EndTxn request returns.
> I'm
> > > >> also
> > > >> > >> > thinking
> > > >> > >> > > > > about
> > > >> > >> > > > > > > > > cases of
> > > >> > >> > > > > > > > > >> failure. I will need to think on this a bit.
> > > >> > >> > > > > > > > > >>
> > > >> > >> > > > > > > > > >> I wasn't sure if it was that confusing. But
> if
> > > we
> > > >> > >> think it
> > > >> > >> > > is,
> > > >> > >> > > > > we
> > > >> > >> > > > > > > can
> > > >> > >> > > > > > > > > >> investigate other ways.
> > > >> > >> > > > > > > > > >>
> > > >> > >> > > > > > > > > >>
> > > >> > >> > > > > > > > > >> 60)
> > > >> > >> > > > > > > > > >>
> > > >> > >> > > > > > > > > >> I'm not sure these are the same purgatories
> > > since
> > > >> one
> > > >> > >> is a
> > > >> > >> > > > > produce
> > > >> > >> > > > > > > > > >> purgatory (I was planning on using a
> callback
> > > >> rather
> > > >> > >> than
> > > >> > >> > > > > > purgatory)
> > > >> > >> > > > > > > > and
> > > >> > >> > > > > > > > > >> the other is simply a request to append to
> the
> > > >> log.
> > > >> > Not
> > > >> > >> > sure
> > > >> > >> > > > we
> > > >> > >> > > > > > have
> > > >> > >> > > > > > > > any
> > > >> > >> > > > > > > > > >> structure here for ordering, but my
> > > understanding
> > > >> is
> > > >> > >> that
> > > >> > >> > > the
> > > >> > >> > > > > > broker
> > > >> > >> > > > > > > > > could
> > > >> > >> > > > > > > > > >> handle the write request before it hears
> back
> > > from
> > > >> > the
> > > >> > >> Txn
> > > >> > >> > > > > > > > Coordinator.
> > > >> > >> > > > > > > > > >>
> > > >> > >> > > > > > > > > >> Let me know if I misunderstood something or
> > > >> something
> > > >> > >> was
> > > >> > >> > > > > unclear.
> > > >> > >> > > > > > > > > >>
> > > >> > >> > > > > > > > > >> Justine
> > > >> > >> > > > > > > > > >>
> > > >> > >> > > > > > > > > >> On Thu, Dec 1, 2022 at 12:15 PM Matthias J.
> > Sax
> > > <
> > > >> > >> > > > > mjsax@apache.org
> > > >> > >> > > > > > >
> > > >> > >> > > > > > > > > wrote:
> > > >> > >> > > > > > > > > >>
> > > >> > >> > > > > > > > > >>> Thanks for the details Justine!
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > > >>>> 20)
> > > >> > >> > > > > > > > > >>>>
> > > >> > >> > > > > > > > > >>>> The client side change for 2 is removing
> the
> > > >> > >> > addPartitions
> > > >> > >> > > > to
> > > >> > >> > > > > > > > > >>> transaction
> > > >> > >> > > > > > > > > >>>> call. We don't need to make this from the
> > > >> producer
> > > >> > to
> > > >> > >> > the
> > > >> > >> > > > txn
> > > >> > >> > > > > > > > > >>> coordinator,
> > > >> > >> > > > > > > > > >>>> only server side.
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > > >>> I think I did not express myself clearly. I
> > > >> > understand
> > > >> > >> > that
> > > >> > >> > > > we
> > > >> > >> > > > > > can
> > > >> > >> > > > > > > > (and
> > > >> > >> > > > > > > > > >>> should) change the producer to not send the
> > > >> > >> > `addPartitions`
> > > >> > >> > > > > > request
> > > >> > >> > > > > > > > any
> > > >> > >> > > > > > > > > >>> longer. But I don't thinks it's requirement
> > to
> > > >> > change
> > > >> > >> the
> > > >> > >> > > > > broker?
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > > >>> What I am trying to say is: as a safe-guard
> > and
> > > >> > >> > improvement
> > > >> > >> > > > for
> > > >> > >> > > > > > > older
> > > >> > >> > > > > > > > > >>> producers, the partition leader can just
> send
> > > the
> > > >> > >> > > > > `addPartitions`
> > > >> > >> > > > > > > > > >>> request to the TX-coordinator in any case
> --
> > if
> > > >> the
> > > >> > >> old
> > > >> > >> > > > > producer
> > > >> > >> > > > > > > > > >>> correctly did send the `addPartition`
> request
> > > to
> > > >> the
> > > >> > >> > > > > > TX-coordinator
> > > >> > >> > > > > > > > > >>> already, the TX-coordinator can just
> "ignore"
> > > is
> > > >> as
> > > >> > >> > > > idempotent.
> > > >> > >> > > > > > > > > However,
> > > >> > >> > > > > > > > > >>> if the old producer has a bug and did
> forget
> > to
> > > >> sent
> > > >> > >> the
> > > >> > >> > > > > > > > `addPartition`
> > > >> > >> > > > > > > > > >>> request, we would now ensure that the
> > partition
> > > >> is
> > > >> > >> indeed
> > > >> > >> > > > added
> > > >> > >> > > > > > to
> > > >> > >> > > > > > > > the
> > > >> > >> > > > > > > > > >>> TX and thus fix a potential producer bug
> > (even
> > > >> if we
> > > >> > >> > don't
> > > >> > >> > > > get
> > > >> > >> > > > > > the
> > > >> > >> > > > > > > > > >>> fencing via the bump epoch). -- It seems to
> > be
> > > a
> > > >> > good
> > > >> > >> > > > > > improvement?
> > > >> > >> > > > > > > Or
> > > >> > >> > > > > > > > > is
> > > >> > >> > > > > > > > > >>> there a reason to not do this?
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > > >>>> 30)
> > > >> > >> > > > > > > > > >>>>
> > > >> > >> > > > > > > > > >>>> Transaction is ongoing = partition was
> added
> > > to
> > > >> > >> > > transaction
> > > >> > >> > > > > via
> > > >> > >> > > > > > > > > >>>> addPartitionsToTxn. We check this with the
> > > >> > >> > > > > DescribeTransactions
> > > >> > >> > > > > > > > call.
> > > >> > >> > > > > > > > > >>> Let
> > > >> > >> > > > > > > > > >>>> me know if this wasn't sufficiently
> > explained
> > > >> here:
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > > >>> If we do what I propose in (20), we don't
> > > really
> > > >> > need
> > > >> > >> to
> > > >> > >> > > make
> > > >> > >> > > > > > this
> > > >> > >> > > > > > > > > >>> `DescribeTransaction` call, as the
> partition
> > > >> leader
> > > >> > >> adds
> > > >> > >> > > the
> > > >> > >> > > > > > > > partition
> > > >> > >> > > > > > > > > >>> for older clients and we get this check for
> > > free.
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > > >>>> 40)
> > > >> > >> > > > > > > > > >>>>
> > > >> > >> > > > > > > > > >>>> The idea here is that if any messages
> > somehow
> > > >> come
> > > >> > in
> > > >> > >> > > before
> > > >> > >> > > > > we
> > > >> > >> > > > > > > get
> > > >> > >> > > > > > > > > the
> > > >> > >> > > > > > > > > >>> new
> > > >> > >> > > > > > > > > >>>> epoch to the producer, they will be
> fenced.
> > > >> > However,
> > > >> > >> if
> > > >> > >> > we
> > > >> > >> > > > > don't
> > > >> > >> > > > > > > > think
> > > >> > >> > > > > > > > > >>> this
> > > >> > >> > > > > > > > > >>>> is necessary, it can be discussed
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > > >>> I agree that we should have epoch fencing.
> My
> > > >> > >> question is
> > > >> > >> > > > > > > different:
> > > >> > >> > > > > > > > > >>> Assume we are at epoch 73, and we have an
> > > ongoing
> > > >> > >> > > > transaction,
> > > >> > >> > > > > > that
> > > >> > >> > > > > > > > is
> > > >> > >> > > > > > > > > >>> committed. It seems natural to write the
> > > "prepare
> > > >> > >> commit"
> > > >> > >> > > > > marker
> > > >> > >> > > > > > > and
> > > >> > >> > > > > > > > > the
> > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` both with epoch 73,
> > too,
> > > >> as
> > > >> > it
> > > >> > >> > > belongs
> > > >> > >> > > > > to
> > > >> > >> > > > > > > the
> > > >> > >> > > > > > > > > >>> current transaction. Of course, we now also
> > > bump
> > > >> the
> > > >> > >> > epoch
> > > >> > >> > > > and
> > > >> > >> > > > > > > expect
> > > >> > >> > > > > > > > > >>> the next requests to have epoch 74, and
> would
> > > >> reject
> > > >> > >> an
> > > >> > >> > > > request
> > > >> > >> > > > > > > with
> > > >> > >> > > > > > > > > >>> epoch 73, as the corresponding TX for epoch
> > 73
> > > >> was
> > > >> > >> > already
> > > >> > >> > > > > > > committed.
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > > >>> It seems you propose to write the "prepare
> > > commit
> > > >> > >> marker"
> > > >> > >> > > and
> > > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` with epoch 74
> though,
> > > what
> > > >> > >> would
> > > >> > >> > > work,
> > > >> > >> > > > > but
> > > >> > >> > > > > > > it
> > > >> > >> > > > > > > > > >>> seems confusing. Is there a reason why we
> > would
> > > >> use
> > > >> > >> the
> > > >> > >> > > > bumped
> > > >> > >> > > > > > > epoch
> > > >> > >> > > > > > > > 74
> > > >> > >> > > > > > > > > >>> instead of the current epoch 73?
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > > >>>> 60)
> > > >> > >> > > > > > > > > >>>>
> > > >> > >> > > > > > > > > >>>> When we are checking if the transaction is
> > > >> ongoing,
> > > >> > >> we
> > > >> > >> > > need
> > > >> > >> > > > to
> > > >> > >> > > > > > > make
> > > >> > >> > > > > > > > a
> > > >> > >> > > > > > > > > >>> round
> > > >> > >> > > > > > > > > >>>> trip from the leader partition to the
> > > >> transaction
> > > >> > >> > > > coordinator.
> > > >> > >> > > > > > In
> > > >> > >> > > > > > > > the
> > > >> > >> > > > > > > > > >>> time
> > > >> > >> > > > > > > > > >>>> we are waiting for this message to come
> > back,
> > > in
> > > >> > >> theory
> > > >> > >> > we
> > > >> > >> > > > > could
> > > >> > >> > > > > > > > have
> > > >> > >> > > > > > > > > >>> sent
> > > >> > >> > > > > > > > > >>>> a commit/abort call that would make the
> > > original
> > > >> > >> result
> > > >> > >> > of
> > > >> > >> > > > the
> > > >> > >> > > > > > > check
> > > >> > >> > > > > > > > > >>> out of
> > > >> > >> > > > > > > > > >>>> date. That is why we can check the leader
> > > state
> > > >> > >> before
> > > >> > >> > we
> > > >> > >> > > > > write
> > > >> > >> > > > > > to
> > > >> > >> > > > > > > > the
> > > >> > >> > > > > > > > > >>> log.
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > > >>> Thanks. Got it.
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > > >>> However, is this really an issue? We put
> the
> > > >> produce
> > > >> > >> > > request
> > > >> > >> > > > in
> > > >> > >> > > > > > > > > >>> purgatory, so how could we process the
> > > >> > >> > > > `WriteTxnMarkerRequest`
> > > >> > >> > > > > > > first?
> > > >> > >> > > > > > > > > >>> Don't we need to put the
> > > `WriteTxnMarkerRequest`
> > > >> > into
> > > >> > >> > > > > purgatory,
> > > >> > >> > > > > > > too,
> > > >> > >> > > > > > > > > >>> for this case, and process both request
> > > in-order?
> > > >> > >> (Again,
> > > >> > >> > > my
> > > >> > >> > > > > > broker
> > > >> > >> > > > > > > > > >>> knowledge is limited and maybe we don't
> > > maintain
> > > >> > >> request
> > > >> > >> > > > order
> > > >> > >> > > > > > for
> > > >> > >> > > > > > > > this
> > > >> > >> > > > > > > > > >>> case, what seems to be an issue IMHO, and I
> > am
> > > >> > >> wondering
> > > >> > >> > if
> > > >> > >> > > > > > > changing
> > > >> > >> > > > > > > > > >>> request handling to preserve order for this
> > > case
> > > >> > >> might be
> > > >> > >> > > the
> > > >> > >> > > > > > > cleaner
> > > >> > >> > > > > > > > > >>> solution?)
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > > >>> -Matthias
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > > >>> On 11/30/22 3:28 PM, Artem Livshits wrote:
> > > >> > >> > > > > > > > > >>>> Hi Justine,
> > > >> > >> > > > > > > > > >>>>
> > > >> > >> > > > > > > > > >>>> I think the interesting part is not in
> this
> > > >> logic
> > > >> > >> > (because
> > > >> > >> > > > it
> > > >> > >> > > > > > > tries
> > > >> > >> > > > > > > > to
> > > >> > >> > > > > > > > > >>>> figure out when UNKNOWN_PRODUCER_ID is
> > > retriable
> > > >> > and
> > > >> > >> if
> > > >> > >> > > it's
> > > >> > >> > > > > > > > > retryable,
> > > >> > >> > > > > > > > > >>>> it's definitely not fatal), but what
> happens
> > > >> when
> > > >> > >> this
> > > >> > >> > > logic
> > > >> > >> > > > > > > doesn't
> > > >> > >> > > > > > > > > >>> return
> > > >> > >> > > > > > > > > >>>> 'true' and falls through.  In the old
> > clients
> > > it
> > > >> > >> seems
> > > >> > >> > to
> > > >> > >> > > be
> > > >> > >> > > > > > > fatal,
> > > >> > >> > > > > > > > if
> > > >> > >> > > > > > > > > >>> we
> > > >> > >> > > > > > > > > >>>> keep the behavior in the new clients, I'd
> > > >> expect it
> > > >> > >> > would
> > > >> > >> > > be
> > > >> > >> > > > > > fatal
> > > >> > >> > > > > > > > as
> > > >> > >> > > > > > > > > >>> well.
> > > >> > >> > > > > > > > > >>>>
> > > >> > >> > > > > > > > > >>>> -Artem
> > > >> > >> > > > > > > > > >>>>
> > > >> > >> > > > > > > > > >>>> On Tue, Nov 29, 2022 at 11:57 AM Justine
> > > Olshan
> > > >> > >> > > > > > > > > >>>> <jo...@confluent.io.invalid> wrote:
> > > >> > >> > > > > > > > > >>>>
> > > >> > >> > > > > > > > > >>>>> Hi Artem and Jeff,
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>> Thanks for taking a look and sorry for
> the
> > > slow
> > > >> > >> > response.
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>> You both mentioned the change to handle
> > > >> > >> > > UNKNOWN_PRODUCER_ID
> > > >> > >> > > > > > > errors.
> > > >> > >> > > > > > > > > To
> > > >> > >> > > > > > > > > >>> be
> > > >> > >> > > > > > > > > >>>>> clear — this error code will only be sent
> > > again
> > > >> > when
> > > >> > >> > the
> > > >> > >> > > > > > client's
> > > >> > >> > > > > > > > > >>> request
> > > >> > >> > > > > > > > > >>>>> version is high enough to ensure we
> handle
> > it
> > > >> > >> > correctly.
> > > >> > >> > > > > > > > > >>>>> The current (Java) client handles this by
> > the
> > > >> > >> following
> > > >> > >> > > > > > (somewhat
> > > >> > >> > > > > > > > > long)
> > > >> > >> > > > > > > > > >>>>> code snippet:
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>> // An UNKNOWN_PRODUCER_ID means that we
> > have
> > > >> lost
> > > >> > >> the
> > > >> > >> > > > > producer
> > > >> > >> > > > > > > > state
> > > >> > >> > > > > > > > > >>> on the
> > > >> > >> > > > > > > > > >>>>> broker. Depending on the log start
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>> // offset, we may want to retry these, as
> > > >> > described
> > > >> > >> for
> > > >> > >> > > > each
> > > >> > >> > > > > > case
> > > >> > >> > > > > > > > > >>> below. If
> > > >> > >> > > > > > > > > >>>>> none of those apply, then for the
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>> // idempotent producer, we will locally
> > bump
> > > >> the
> > > >> > >> epoch
> > > >> > >> > > and
> > > >> > >> > > > > > reset
> > > >> > >> > > > > > > > the
> > > >> > >> > > > > > > > > >>>>> sequence numbers of in-flight batches
> from
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>> // sequence 0, then retry the failed
> batch,
> > > >> which
> > > >> > >> > should
> > > >> > >> > > > now
> > > >> > >> > > > > > > > succeed.
> > > >> > >> > > > > > > > > >>> For
> > > >> > >> > > > > > > > > >>>>> the transactional producer, allow the
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>> // batch to fail. When processing the
> > failed
> > > >> > batch,
> > > >> > >> we
> > > >> > >> > > will
> > > >> > >> > > > > > > > > transition
> > > >> > >> > > > > > > > > >>> to
> > > >> > >> > > > > > > > > >>>>> an abortable error and set a flag
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>> // indicating that we need to bump the
> > epoch
> > > >> (if
> > > >> > >> > > supported
> > > >> > >> > > > by
> > > >> > >> > > > > > the
> > > >> > >> > > > > > > > > >>> broker).
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>> if (error ==
> Errors.*UNKNOWN_PRODUCER_ID*)
> > {
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>       if (response.logStartOffset == -1)
> {
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           // We don't know the log start
> > > offset
> > > >> > with
> > > >> > >> > this
> > > >> > >> > > > > > > response.
> > > >> > >> > > > > > > > > We
> > > >> > >> > > > > > > > > >>> should
> > > >> > >> > > > > > > > > >>>>> just retry the request until we get it.
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           // The UNKNOWN_PRODUCER_ID
> error
> > > code
> > > >> > was
> > > >> > >> > added
> > > >> > >> > > > > along
> > > >> > >> > > > > > > > with
> > > >> > >> > > > > > > > > >>> the new
> > > >> > >> > > > > > > > > >>>>> ProduceResponse which includes the
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           // logStartOffset. So the '-1'
> > > >> sentinel
> > > >> > is
> > > >> > >> > not
> > > >> > >> > > > for
> > > >> > >> > > > > > > > backward
> > > >> > >> > > > > > > > > >>>>> compatibility. Instead, it is possible
> for
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           // a broker to not know the
> > > >> > >> logStartOffset at
> > > >> > >> > > > when
> > > >> > >> > > > > it
> > > >> > >> > > > > > > is
> > > >> > >> > > > > > > > > >>> returning
> > > >> > >> > > > > > > > > >>>>> the response because the partition
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           // may have moved away from the
> > > >> broker
> > > >> > >> from
> > > >> > >> > the
> > > >> > >> > > > > time
> > > >> > >> > > > > > > the
> > > >> > >> > > > > > > > > >>> error was
> > > >> > >> > > > > > > > > >>>>> initially raised to the time the
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           // response was being
> > constructed.
> > > In
> > > >> > >> these
> > > >> > >> > > > cases,
> > > >> > >> > > > > we
> > > >> > >> > > > > > > > > should
> > > >> > >> > > > > > > > > >>> just
> > > >> > >> > > > > > > > > >>>>> retry the request: we are guaranteed
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           // to eventually get a
> > > logStartOffset
> > > >> > once
> > > >> > >> > > things
> > > >> > >> > > > > > > settle
> > > >> > >> > > > > > > > > down.
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           return true;
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>       }
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>       if (batch.sequenceHasBeenReset()) {
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           // When the first inflight
> batch
> > > >> fails
> > > >> > >> due to
> > > >> > >> > > the
> > > >> > >> > > > > > > > > truncation
> > > >> > >> > > > > > > > > >>> case,
> > > >> > >> > > > > > > > > >>>>> then the sequences of all the other
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           // in flight batches would have
> > > been
> > > >> > >> > restarted
> > > >> > >> > > > from
> > > >> > >> > > > > > the
> > > >> > >> > > > > > > > > >>> beginning.
> > > >> > >> > > > > > > > > >>>>> However, when those responses
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           // come back from the broker,
> > they
> > > >> would
> > > >> > >> also
> > > >> > >> > > > come
> > > >> > >> > > > > > with
> > > >> > >> > > > > > > > an
> > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID error. In this case,
> we
> > > >> should
> > > >> > >> not
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           // reset the sequence numbers
> to
> > > the
> > > >> > >> > beginning.
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           return true;
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>       } else if
> > > >> > >> > > > > (lastAckedOffset(batch.topicPartition).orElse(
> > > >> > >> > > > > > > > > >>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > > >> > >> > > > response.logStartOffset) {
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           // The head of the log has been
> > > >> removed,
> > > >> > >> > > probably
> > > >> > >> > > > > due
> > > >> > >> > > > > > > to
> > > >> > >> > > > > > > > > the
> > > >> > >> > > > > > > > > >>>>> retention time elapsing. In this case,
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           // we expect to lose the
> producer
> > > >> state.
> > > >> > >> For
> > > >> > >> > > the
> > > >> > >> > > > > > > > > transactional
> > > >> > >> > > > > > > > > >>>>> producer, reset the sequences of all
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           // inflight batches to be from
> > the
> > > >> > >> beginning
> > > >> > >> > > and
> > > >> > >> > > > > > retry
> > > >> > >> > > > > > > > > them,
> > > >> > >> > > > > > > > > >>> so
> > > >> > >> > > > > > > > > >>>>> that the transaction does not need to
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           // be aborted. For the
> idempotent
> > > >> > >> producer,
> > > >> > >> > > bump
> > > >> > >> > > > > the
> > > >> > >> > > > > > > > epoch
> > > >> > >> > > > > > > > > to
> > > >> > >> > > > > > > > > >>> avoid
> > > >> > >> > > > > > > > > >>>>> reusing (sequence, epoch) pairs
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           if (isTransactional()) {
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > >
> > > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > > >> > >> > > > > > > > > >>>>> this.producerIdAndEpoch);
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           } else {
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > >  requestEpochBumpForPartition(batch.topicPartition);
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           }
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           return true;
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>       }
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>       if (!isTransactional()) {
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           // For the idempotent producer,
> > > >> always
> > > >> > >> retry
> > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > >> > >> > > > > > > > > >>>>> errors. If the batch has the current
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           // producer ID and epoch,
> > request a
> > > >> bump
> > > >> > >> of
> > > >> > >> > the
> > > >> > >> > > > > > epoch.
> > > >> > >> > > > > > > > > >>> Otherwise
> > > >> > >> > > > > > > > > >>>>> just retry the produce.
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > >  requestEpochBumpForPartition(batch.topicPartition);
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>           return true;
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>       }
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>> }
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>> I was considering keeping this behavior —
> > but
> > > >> am
> > > >> > >> open
> > > >> > >> > to
> > > >> > >> > > > > > > > simplifying
> > > >> > >> > > > > > > > > >>> it.
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>> We are leaving changes to older clients
> off
> > > the
> > > >> > >> table
> > > >> > >> > > here
> > > >> > >> > > > > > since
> > > >> > >> > > > > > > it
> > > >> > >> > > > > > > > > >>> caused
> > > >> > >> > > > > > > > > >>>>> many issues for clients in the past.
> > > Previously
> > > >> > this
> > > >> > >> > was
> > > >> > >> > > a
> > > >> > >> > > > > > fatal
> > > >> > >> > > > > > > > > error
> > > >> > >> > > > > > > > > >>> and
> > > >> > >> > > > > > > > > >>>>> we didn't have the mechanisms in place to
> > > >> detect
> > > >> > >> when
> > > >> > >> > > this
> > > >> > >> > > > > was
> > > >> > >> > > > > > a
> > > >> > >> > > > > > > > > >>> legitimate
> > > >> > >> > > > > > > > > >>>>> case vs some bug or gap in the protocol.
> > > >> Ensuring
> > > >> > >> each
> > > >> > >> > > > > > > transaction
> > > >> > >> > > > > > > > > has
> > > >> > >> > > > > > > > > >>> its
> > > >> > >> > > > > > > > > >>>>> own epoch should close this gap.
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>> And to address Jeff's second point:
> > > >> > >> > > > > > > > > >>>>> *does the typical produce request path
> > append
> > > >> > >> records
> > > >> > >> > to
> > > >> > >> > > > > local
> > > >> > >> > > > > > > log
> > > >> > >> > > > > > > > > >>> along*
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>> *with the currentTxnFirstOffset
> > information?
> > > I
> > > >> > would
> > > >> > >> > like
> > > >> > >> > > > to
> > > >> > >> > > > > > > > > >>> understand*
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>> *when the field is written to disk.*
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>> Yes, the first produce request populates
> > this
> > > >> > field
> > > >> > >> and
> > > >> > >> > > > > writes
> > > >> > >> > > > > > > the
> > > >> > >> > > > > > > > > >>> offset
> > > >> > >> > > > > > > > > >>>>> as part of the record batch and also to
> the
> > > >> > producer
> > > >> > >> > > state
> > > >> > >> > > > > > > > snapshot.
> > > >> > >> > > > > > > > > >>> When
> > > >> > >> > > > > > > > > >>>>> we reload the records on restart and/or
> > > >> > >> reassignment,
> > > >> > >> > we
> > > >> > >> > > > > > > repopulate
> > > >> > >> > > > > > > > > >>> this
> > > >> > >> > > > > > > > > >>>>> field with the snapshot from disk along
> > with
> > > >> the
> > > >> > >> rest
> > > >> > >> > of
> > > >> > >> > > > the
> > > >> > >> > > > > > > > producer
> > > >> > >> > > > > > > > > >>>>> state.
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>> Let me know if there are further comments
> > > >> and/or
> > > >> > >> > > questions.
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>> Thanks,
> > > >> > >> > > > > > > > > >>>>> Justine
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>> On Tue, Nov 22, 2022 at 9:00 PM Jeff Kim
> > > >> > >> > > > > > > > > <jeff.kim@confluent.io.invalid
> > > >> > >> > > > > > > > > >>>>
> > > >> > >> > > > > > > > > >>>>> wrote:
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>>> Hi Justine,
> > > >> > >> > > > > > > > > >>>>>>
> > > >> > >> > > > > > > > > >>>>>> Thanks for the KIP! I have two
> questions:
> > > >> > >> > > > > > > > > >>>>>>
> > > >> > >> > > > > > > > > >>>>>> 1) For new clients, we can once again
> > return
> > > >> an
> > > >> > >> error
> > > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > >> > >> > > > > > > > > >>>>>> for sequences
> > > >> > >> > > > > > > > > >>>>>> that are non-zero when there is no
> > producer
> > > >> state
> > > >> > >> > > present
> > > >> > >> > > > on
> > > >> > >> > > > > > the
> > > >> > >> > > > > > > > > >>> server.
> > > >> > >> > > > > > > > > >>>>>> This will indicate we missed the 0
> > sequence
> > > >> and
> > > >> > we
> > > >> > >> > don't
> > > >> > >> > > > yet
> > > >> > >> > > > > > > want
> > > >> > >> > > > > > > > to
> > > >> > >> > > > > > > > > >>>>> write
> > > >> > >> > > > > > > > > >>>>>> to the log.
> > > >> > >> > > > > > > > > >>>>>>
> > > >> > >> > > > > > > > > >>>>>> I would like to understand the current
> > > >> behavior
> > > >> > to
> > > >> > >> > > handle
> > > >> > >> > > > > > older
> > > >> > >> > > > > > > > > >>> clients,
> > > >> > >> > > > > > > > > >>>>>> and if there are any changes we are
> > making.
> > > >> Maybe
> > > >> > >> I'm
> > > >> > >> > > > > missing
> > > >> > >> > > > > > > > > >>> something,
> > > >> > >> > > > > > > > > >>>>>> but we would want to identify whether we
> > > >> missed
> > > >> > >> the 0
> > > >> > >> > > > > sequence
> > > >> > >> > > > > > > for
> > > >> > >> > > > > > > > > >>> older
> > > >> > >> > > > > > > > > >>>>>> clients, no?
> > > >> > >> > > > > > > > > >>>>>>
> > > >> > >> > > > > > > > > >>>>>> 2) Upon returning from the transaction
> > > >> > >> coordinator, we
> > > >> > >> > > can
> > > >> > >> > > > > set
> > > >> > >> > > > > > > the
> > > >> > >> > > > > > > > > >>>>>> transaction
> > > >> > >> > > > > > > > > >>>>>> as ongoing on the leader by populating
> > > >> > >> > > > currentTxnFirstOffset
> > > >> > >> > > > > > > > > >>>>>> through the typical produce request
> > > handling.
> > > >> > >> > > > > > > > > >>>>>>
> > > >> > >> > > > > > > > > >>>>>> does the typical produce request path
> > append
> > > >> > >> records
> > > >> > >> > to
> > > >> > >> > > > > local
> > > >> > >> > > > > > > log
> > > >> > >> > > > > > > > > >>> along
> > > >> > >> > > > > > > > > >>>>>> with the currentTxnFirstOffset
> > information?
> > > I
> > > >> > would
> > > >> > >> > like
> > > >> > >> > > > to
> > > >> > >> > > > > > > > > understand
> > > >> > >> > > > > > > > > >>>>>> when the field is written to disk.
> > > >> > >> > > > > > > > > >>>>>>
> > > >> > >> > > > > > > > > >>>>>> Thanks,
> > > >> > >> > > > > > > > > >>>>>> Jeff
> > > >> > >> > > > > > > > > >>>>>>
> > > >> > >> > > > > > > > > >>>>>>
> > > >> > >> > > > > > > > > >>>>>> On Tue, Nov 22, 2022 at 4:44 PM Artem
> > > Livshits
> > > >> > >> > > > > > > > > >>>>>> <al...@confluent.io.invalid> wrote:
> > > >> > >> > > > > > > > > >>>>>>
> > > >> > >> > > > > > > > > >>>>>>> Hi Justine,
> > > >> > >> > > > > > > > > >>>>>>>
> > > >> > >> > > > > > > > > >>>>>>> Thank you for the KIP.  I have one
> > > question.
> > > >> > >> > > > > > > > > >>>>>>>
> > > >> > >> > > > > > > > > >>>>>>> 5) For new clients, we can once again
> > > return
> > > >> an
> > > >> > >> error
> > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > >> > >> > > > > > > > > >>>>>>>
> > > >> > >> > > > > > > > > >>>>>>> I believe we had problems in the past
> > with
> > > >> > >> returning
> > > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > >> > >> > > > > > > > > >>>>>>> because it was considered fatal and
> > > required
> > > >> > >> client
> > > >> > >> > > > > restart.
> > > >> > >> > > > > > > It
> > > >> > >> > > > > > > > > >>> would
> > > >> > >> > > > > > > > > >>>>> be
> > > >> > >> > > > > > > > > >>>>>>> good to spell out the new client
> behavior
> > > >> when
> > > >> > it
> > > >> > >> > > > receives
> > > >> > >> > > > > > the
> > > >> > >> > > > > > > > > error.
> > > >> > >> > > > > > > > > >>>>>>>
> > > >> > >> > > > > > > > > >>>>>>> -Artem
> > > >> > >> > > > > > > > > >>>>>>>
> > > >> > >> > > > > > > > > >>>>>>> On Tue, Nov 22, 2022 at 10:00 AM
> Justine
> > > >> Olshan
> > > >> > >> > > > > > > > > >>>>>>> <jo...@confluent.io.invalid> wrote:
> > > >> > >> > > > > > > > > >>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>> Thanks for taking a look Matthias.
> I've
> > > >> tried
> > > >> > to
> > > >> > >> > > answer
> > > >> > >> > > > > your
> > > >> > >> > > > > > > > > >>>>> questions
> > > >> > >> > > > > > > > > >>>>>>>> below:
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>> 10)
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>> Right — so the hanging transaction
> only
> > > >> occurs
> > > >> > >> when
> > > >> > >> > we
> > > >> > >> > > > > have
> > > >> > >> > > > > > a
> > > >> > >> > > > > > > > late
> > > >> > >> > > > > > > > > >>>>>>> message
> > > >> > >> > > > > > > > > >>>>>>>> come in and the partition is never
> added
> > > to
> > > >> a
> > > >> > >> > > > transaction
> > > >> > >> > > > > > > again.
> > > >> > >> > > > > > > > > If
> > > >> > >> > > > > > > > > >>>>> we
> > > >> > >> > > > > > > > > >>>>>>>> never add the partition to a
> > transaction,
> > > we
> > > >> > will
> > > >> > >> > > never
> > > >> > >> > > > > > write
> > > >> > >> > > > > > > a
> > > >> > >> > > > > > > > > >>>>> marker
> > > >> > >> > > > > > > > > >>>>>>> and
> > > >> > >> > > > > > > > > >>>>>>>> never advance the LSO.
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>> If we do end up adding the partition
> to
> > > the
> > > >> > >> > > transaction
> > > >> > >> > > > (I
> > > >> > >> > > > > > > > suppose
> > > >> > >> > > > > > > > > >>>>> this
> > > >> > >> > > > > > > > > >>>>>>> can
> > > >> > >> > > > > > > > > >>>>>>>> happen before or after the late
> message
> > > >> comes
> > > >> > in)
> > > >> > >> > then
> > > >> > >> > > > we
> > > >> > >> > > > > > will
> > > >> > >> > > > > > > > > >>>>> include
> > > >> > >> > > > > > > > > >>>>>>> the
> > > >> > >> > > > > > > > > >>>>>>>> late message in the next (incorrect)
> > > >> > transaction.
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>> So perhaps it is clearer to make the
> > > >> > distinction
> > > >> > >> > > between
> > > >> > >> > > > > > > > messages
> > > >> > >> > > > > > > > > >>>>> that
> > > >> > >> > > > > > > > > >>>>>>>> eventually get added to the
> transaction
> > > (but
> > > >> > the
> > > >> > >> > wrong
> > > >> > >> > > > > one)
> > > >> > >> > > > > > or
> > > >> > >> > > > > > > > > >>>>> messages
> > > >> > >> > > > > > > > > >>>>>>>> that never get added and become
> hanging.
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>> 20)
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>> The client side change for 2 is
> removing
> > > the
> > > >> > >> > > > addPartitions
> > > >> > >> > > > > > to
> > > >> > >> > > > > > > > > >>>>>> transaction
> > > >> > >> > > > > > > > > >>>>>>>> call. We don't need to make this from
> > the
> > > >> > >> producer
> > > >> > >> > to
> > > >> > >> > > > the
> > > >> > >> > > > > > txn
> > > >> > >> > > > > > > > > >>>>>>> coordinator,
> > > >> > >> > > > > > > > > >>>>>>>> only server side.
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>> In my opinion, the issue with the
> > > >> > >> addPartitionsToTxn
> > > >> > >> > > > call
> > > >> > >> > > > > > for
> > > >> > >> > > > > > > > > older
> > > >> > >> > > > > > > > > >>>>>>> clients
> > > >> > >> > > > > > > > > >>>>>>>> is that we don't have the epoch bump,
> so
> > > we
> > > >> > don't
> > > >> > >> > know
> > > >> > >> > > > if
> > > >> > >> > > > > > the
> > > >> > >> > > > > > > > > >>> message
> > > >> > >> > > > > > > > > >>>>>>>> belongs to the previous transaction or
> > > this
> > > >> > one.
> > > >> > >> We
> > > >> > >> > > need
> > > >> > >> > > > > to
> > > >> > >> > > > > > > > check
> > > >> > >> > > > > > > > > if
> > > >> > >> > > > > > > > > >>>>>> the
> > > >> > >> > > > > > > > > >>>>>>>> partition has been added to this
> > > >> transaction.
> > > >> > Of
> > > >> > >> > > course,
> > > >> > >> > > > > > this
> > > >> > >> > > > > > > > > means
> > > >> > >> > > > > > > > > >>>>> we
> > > >> > >> > > > > > > > > >>>>>>>> won't completely cover the case where
> we
> > > >> have a
> > > >> > >> > really
> > > >> > >> > > > > late
> > > >> > >> > > > > > > > > message
> > > >> > >> > > > > > > > > >>>>> and
> > > >> > >> > > > > > > > > >>>>>>> we
> > > >> > >> > > > > > > > > >>>>>>>> have added the partition to the new
> > > >> > transaction,
> > > >> > >> but
> > > >> > >> > > > > that's
> > > >> > >> > > > > > > > > >>>>>> unfortunately
> > > >> > >> > > > > > > > > >>>>>>>> something we will need the new clients
> > to
> > > >> > cover.
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>> 30)
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>> Transaction is ongoing = partition was
> > > >> added to
> > > >> > >> > > > > transaction
> > > >> > >> > > > > > > via
> > > >> > >> > > > > > > > > >>>>>>>> addPartitionsToTxn. We check this with
> > the
> > > >> > >> > > > > > > DescribeTransactions
> > > >> > >> > > > > > > > > >>> call.
> > > >> > >> > > > > > > > > >>>>>> Let
> > > >> > >> > > > > > > > > >>>>>>>> me know if this wasn't sufficiently
> > > >> explained
> > > >> > >> here:
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>
> > > >> > >> > > > > > > > > >>>>>>
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > >
> > > >> > >> > > > > > > >
> > > >> > >> > > > > > >
> > > >> > >> > > > > >
> > > >> > >> > > > >
> > > >> > >> > > >
> > > >> > >> > >
> > > >> > >> >
> > > >> > >>
> > > >> >
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>> 40)
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>> The idea here is that if any messages
> > > >> somehow
> > > >> > >> come
> > > >> > >> > in
> > > >> > >> > > > > before
> > > >> > >> > > > > > > we
> > > >> > >> > > > > > > > > get
> > > >> > >> > > > > > > > > >>>>> the
> > > >> > >> > > > > > > > > >>>>>>> new
> > > >> > >> > > > > > > > > >>>>>>>> epoch to the producer, they will be
> > > fenced.
> > > >> > >> However,
> > > >> > >> > > if
> > > >> > >> > > > we
> > > >> > >> > > > > > > don't
> > > >> > >> > > > > > > > > >>>>> think
> > > >> > >> > > > > > > > > >>>>>>> this
> > > >> > >> > > > > > > > > >>>>>>>> is necessary, it can be discussed
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>> 50)
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>> It should be synchronous because if we
> > > have
> > > >> an
> > > >> > >> event
> > > >> > >> > > > (ie,
> > > >> > >> > > > > an
> > > >> > >> > > > > > > > > error)
> > > >> > >> > > > > > > > > >>>>>> that
> > > >> > >> > > > > > > > > >>>>>>>> causes us to need to abort the
> > > transaction,
> > > >> we
> > > >> > >> need
> > > >> > >> > to
> > > >> > >> > > > > know
> > > >> > >> > > > > > > > which
> > > >> > >> > > > > > > > > >>>>>>>> partitions to send transaction markers
> > to.
> > > >> We
> > > >> > >> know
> > > >> > >> > the
> > > >> > >> > > > > > > > partitions
> > > >> > >> > > > > > > > > >>>>>> because
> > > >> > >> > > > > > > > > >>>>>>>> we added them to the coordinator via
> the
> > > >> > >> > > > > addPartitionsToTxn
> > > >> > >> > > > > > > > call.
> > > >> > >> > > > > > > > > >>>>>>>> Previously we have had asynchronous
> > calls
> > > in
> > > >> > the
> > > >> > >> > past
> > > >> > >> > > > (ie,
> > > >> > >> > > > > > > > writing
> > > >> > >> > > > > > > > > >>>>> the
> > > >> > >> > > > > > > > > >>>>>>>> commit markers when the transaction is
> > > >> > completed)
> > > >> > >> > but
> > > >> > >> > > > > often
> > > >> > >> > > > > > > this
> > > >> > >> > > > > > > > > >>> just
> > > >> > >> > > > > > > > > >>>>>>>> causes confusion as we need to wait
> for
> > > some
> > > >> > >> > > operations
> > > >> > >> > > > to
> > > >> > >> > > > > > > > > complete.
> > > >> > >> > > > > > > > > >>>>> In
> > > >> > >> > > > > > > > > >>>>>>> the
> > > >> > >> > > > > > > > > >>>>>>>> writing commit markers case, clients
> > often
> > > >> see
> > > >> > >> > > > > > > > > >>>>> CONCURRENT_TRANSACTIONs
> > > >> > >> > > > > > > > > >>>>>>>> error messages and that can be
> > confusing.
> > > >> For
> > > >> > >> that
> > > >> > >> > > > reason,
> > > >> > >> > > > > > it
> > > >> > >> > > > > > > > may
> > > >> > >> > > > > > > > > be
> > > >> > >> > > > > > > > > >>>>>>>> simpler to just have synchronous
> calls —
> > > >> > >> especially
> > > >> > >> > if
> > > >> > >> > > > we
> > > >> > >> > > > > > need
> > > >> > >> > > > > > > > to
> > > >> > >> > > > > > > > > >>>>> block
> > > >> > >> > > > > > > > > >>>>>>> on
> > > >> > >> > > > > > > > > >>>>>>>> some operation's completion anyway
> > before
> > > we
> > > >> > can
> > > >> > >> > start
> > > >> > >> > > > the
> > > >> > >> > > > > > > next
> > > >> > >> > > > > > > > > >>>>>>>> transaction. And yes, I meant
> > > coordinator. I
> > > >> > will
> > > >> > >> > fix
> > > >> > >> > > > > that.
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>> 60)
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>> When we are checking if the
> transaction
> > is
> > > >> > >> ongoing,
> > > >> > >> > we
> > > >> > >> > > > > need
> > > >> > >> > > > > > to
> > > >> > >> > > > > > > > > make
> > > >> > >> > > > > > > > > >>> a
> > > >> > >> > > > > > > > > >>>>>>> round
> > > >> > >> > > > > > > > > >>>>>>>> trip from the leader partition to the
> > > >> > transaction
> > > >> > >> > > > > > coordinator.
> > > >> > >> > > > > > > > In
> > > >> > >> > > > > > > > > >>> the
> > > >> > >> > > > > > > > > >>>>>>> time
> > > >> > >> > > > > > > > > >>>>>>>> we are waiting for this message to
> come
> > > >> back,
> > > >> > in
> > > >> > >> > > theory
> > > >> > >> > > > we
> > > >> > >> > > > > > > could
> > > >> > >> > > > > > > > > >>> have
> > > >> > >> > > > > > > > > >>>>>>> sent
> > > >> > >> > > > > > > > > >>>>>>>> a commit/abort call that would make
> the
> > > >> > original
> > > >> > >> > > result
> > > >> > >> > > > of
> > > >> > >> > > > > > the
> > > >> > >> > > > > > > > > check
> > > >> > >> > > > > > > > > >>>>>> out
> > > >> > >> > > > > > > > > >>>>>>> of
> > > >> > >> > > > > > > > > >>>>>>>> date. That is why we can check the
> > leader
> > > >> state
> > > >> > >> > before
> > > >> > >> > > > we
> > > >> > >> > > > > > > write
> > > >> > >> > > > > > > > to
> > > >> > >> > > > > > > > > >>>>> the
> > > >> > >> > > > > > > > > >>>>>>> log.
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>> I'm happy to update the KIP if some of
> > > these
> > > >> > >> things
> > > >> > >> > > were
> > > >> > >> > > > > not
> > > >> > >> > > > > > > > > clear.
> > > >> > >> > > > > > > > > >>>>>>>> Thanks,
> > > >> > >> > > > > > > > > >>>>>>>> Justine
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>> On Mon, Nov 21, 2022 at 7:11 PM
> Matthias
> > > J.
> > > >> > Sax <
> > > >> > >> > > > > > > > mjsax@apache.org
> > > >> > >> > > > > > > > > >
> > > >> > >> > > > > > > > > >>>>>>> wrote:
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>> Thanks for the KIP.
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>> Couple of clarification questions (I
> am
> > > >> not a
> > > >> > >> > broker
> > > >> > >> > > > > expert
> > > >> > >> > > > > > > do
> > > >> > >> > > > > > > > > >>>>> maybe
> > > >> > >> > > > > > > > > >>>>>>>>> some question are obvious for others,
> > but
> > > >> not
> > > >> > >> for
> > > >> > >> > me
> > > >> > >> > > > with
> > > >> > >> > > > > > my
> > > >> > >> > > > > > > > lack
> > > >> > >> > > > > > > > > >>>>> of
> > > >> > >> > > > > > > > > >>>>>>>>> broker knowledge).
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>> (10)
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>> 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.
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>> What happens if the message come in
> > > before
> > > >> the
> > > >> > >> next
> > > >> > >> > > > > > > > > >>>>>> addPartitionsToTxn
> > > >> > >> > > > > > > > > >>>>>>>>> request? It seems the broker hosting
> > the
> > > >> data
> > > >> > >> > > > partitions
> > > >> > >> > > > > > > won't
> > > >> > >> > > > > > > > > know
> > > >> > >> > > > > > > > > >>>>>>>>> anything about it and append it to
> the
> > > >> > >> partition,
> > > >> > >> > > too?
> > > >> > >> > > > > What
> > > >> > >> > > > > > > is
> > > >> > >> > > > > > > > > the
> > > >> > >> > > > > > > > > >>>>>>>>> difference between both cases?
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>> Also, it seems a TX would only hang,
> if
> > > >> there
> > > >> > >> is no
> > > >> > >> > > > > > following
> > > >> > >> > > > > > > > TX
> > > >> > >> > > > > > > > > >>>>> that
> > > >> > >> > > > > > > > > >>>>>>> is
> > > >> > >> > > > > > > > > >>>>>>>>> either committer or aborted? Thus,
> for
> > > the
> > > >> > case
> > > >> > >> > > above,
> > > >> > >> > > > > the
> > > >> > >> > > > > > TX
> > > >> > >> > > > > > > > > might
> > > >> > >> > > > > > > > > >>>>>>>>> actually not hang (of course, we
> might
> > > get
> > > >> an
> > > >> > >> EOS
> > > >> > >> > > > > violation
> > > >> > >> > > > > > > if
> > > >> > >> > > > > > > > > the
> > > >> > >> > > > > > > > > >>>>>>> first
> > > >> > >> > > > > > > > > >>>>>>>>> TX was aborted and the second
> > committed,
> > > or
> > > >> > the
> > > >> > >> > other
> > > >> > >> > > > way
> > > >> > >> > > > > > > > > around).
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>> (20)
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>> Of course, 1 and 2 require
> client-side
> > > >> > >> changes, so
> > > >> > >> > > for
> > > >> > >> > > > > > older
> > > >> > >> > > > > > > > > >>>>>> clients,
> > > >> > >> > > > > > > > > >>>>>>>>> those approaches won’t apply.
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>> For (1) I understand why a client
> > change
> > > is
> > > >> > >> > > necessary,
> > > >> > >> > > > > but
> > > >> > >> > > > > > > not
> > > >> > >> > > > > > > > > sure
> > > >> > >> > > > > > > > > >>>>>> why
> > > >> > >> > > > > > > > > >>>>>>>>> we need a client change for (2). Can
> > you
> > > >> > >> elaborate?
> > > >> > >> > > --
> > > >> > >> > > > > > Later
> > > >> > >> > > > > > > > you
> > > >> > >> > > > > > > > > >>>>>>> explain
> > > >> > >> > > > > > > > > >>>>>>>>> that we should send a
> > > >> > >> DescribeTransactionRequest,
> > > >> > >> > > but I
> > > >> > >> > > > > am
> > > >> > >> > > > > > > not
> > > >> > >> > > > > > > > > sure
> > > >> > >> > > > > > > > > >>>>>>> why?
> > > >> > >> > > > > > > > > >>>>>>>>> Can't we not just do an implicit
> > > >> > >> AddPartiitonToTx,
> > > >> > >> > > too?
> > > >> > >> > > > > If
> > > >> > >> > > > > > > the
> > > >> > >> > > > > > > > > old
> > > >> > >> > > > > > > > > >>>>>>>>> producer correctly registered the
> > > partition
> > > >> > >> > already,
> > > >> > >> > > > the
> > > >> > >> > > > > > > > > >>>>>> TX-coordinator
> > > >> > >> > > > > > > > > >>>>>>>>> can just ignore it as it's an
> > idempotent
> > > >> > >> operation?
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>> (30)
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>> To cover older clients, we will
> > ensure a
> > > >> > >> > transaction
> > > >> > >> > > > is
> > > >> > >> > > > > > > > ongoing
> > > >> > >> > > > > > > > > >>>>>>> before
> > > >> > >> > > > > > > > > >>>>>>>>> we write to a transaction
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>> Not sure what you mean by this? Can
> you
> > > >> > >> elaborate?
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>> (40)
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>> [the TX-coordinator] will write the
> > > >> prepare
> > > >> > >> commit
> > > >> > >> > > > > message
> > > >> > >> > > > > > > > with
> > > >> > >> > > > > > > > > a
> > > >> > >> > > > > > > > > >>>>>>>> bumped
> > > >> > >> > > > > > > > > >>>>>>>>> epoch and send WriteTxnMarkerRequests
> > > with
> > > >> the
> > > >> > >> > bumped
> > > >> > >> > > > > > epoch.
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>> Why do we use the bumped epoch for
> > both?
> > > It
> > > >> > >> seems
> > > >> > >> > > more
> > > >> > >> > > > > > > > intuitive
> > > >> > >> > > > > > > > > to
> > > >> > >> > > > > > > > > >>>>>> use
> > > >> > >> > > > > > > > > >>>>>>>>> the current epoch, and only return
> the
> > > >> bumped
> > > >> > >> epoch
> > > >> > >> > > to
> > > >> > >> > > > > the
> > > >> > >> > > > > > > > > >>>>> producer?
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>> (50) "Implicit
> > AddPartitionToTransaction"
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>> Why does the implicitly sent request
> > need
> > > >> to
> > > >> > be
> > > >> > >> > > > > > synchronous?
> > > >> > >> > > > > > > > The
> > > >> > >> > > > > > > > > >>>>> KIP
> > > >> > >> > > > > > > > > >>>>>>>>> also says
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>> in case we need to abort and need to
> > > know
> > > >> > which
> > > >> > >> > > > > partitions
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>> What do you mean by this?
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>> we don’t want to write to it before
> we
> > > >> store
> > > >> > in
> > > >> > >> > the
> > > >> > >> > > > > > > > transaction
> > > >> > >> > > > > > > > > >>>>>>> manager
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>> Do you mean TX-coordinator instead of
> > > >> > "manager"?
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>> (60)
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>> For older clients and ensuring that
> the
> > > TX
> > > >> is
> > > >> > >> > > ongoing,
> > > >> > >> > > > > you
> > > >> > >> > > > > > > > > >>>>> describe a
> > > >> > >> > > > > > > > > >>>>>>>>> race condition. I am not sure if I
> can
> > > >> follow
> > > >> > >> here.
> > > >> > >> > > Can
> > > >> > >> > > > > you
> > > >> > >> > > > > > > > > >>>>>> elaborate?
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>> -Matthias
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>> On 11/18/22 1:21 PM, Justine Olshan
> > > wrote:
> > > >> > >> > > > > > > > > >>>>>>>>>> Hey all!
> > > >> > >> > > > > > > > > >>>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>> I'd like to start a discussion on my
> > > >> proposal
> > > >> > >> to
> > > >> > >> > add
> > > >> > >> > > > > some
> > > >> > >> > > > > > > > > >>>>>> server-side
> > > >> > >> > > > > > > > > >>>>>>>>>> checks on transactions to avoid
> > hanging
> > > >> > >> > > transactions.
> > > >> > >> > > > I
> > > >> > >> > > > > > know
> > > >> > >> > > > > > > > > this
> > > >> > >> > > > > > > > > >>>>>> has
> > > >> > >> > > > > > > > > >>>>>>>>> been
> > > >> > >> > > > > > > > > >>>>>>>>>> an issue for some time, so I really
> > hope
> > > >> this
> > > >> > >> KIP
> > > >> > >> > > will
> > > >> > >> > > > > be
> > > >> > >> > > > > > > > > helpful
> > > >> > >> > > > > > > > > >>>>>> for
> > > >> > >> > > > > > > > > >>>>>>>>> many
> > > >> > >> > > > > > > > > >>>>>>>>>> users of EOS.
> > > >> > >> > > > > > > > > >>>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>> The KIP includes changes that will
> be
> > > >> > >> compatible
> > > >> > >> > > with
> > > >> > >> > > > > old
> > > >> > >> > > > > > > > > clients
> > > >> > >> > > > > > > > > >>>>>> and
> > > >> > >> > > > > > > > > >>>>>>>>>> changes to improve performance and
> > > >> > correctness
> > > >> > >> on
> > > >> > >> > > new
> > > >> > >> > > > > > > clients.
> > > >> > >> > > > > > > > > >>>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>> Please take a look and leave any
> > > comments
> > > >> you
> > > >> > >> may
> > > >> > >> > > > have!
> > > >> > >> > > > > > > > > >>>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>> KIP:
> > > >> > >> > > > > > > > > >>>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>
> > > >> > >> > > > > > > > > >>>>>>
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > >
> > > >> > >> > > > > > > >
> > > >> > >> > > > > > >
> > > >> > >> > > > > >
> > > >> > >> > > > >
> > > >> > >> > > >
> > > >> > >> > >
> > > >> > >> >
> > > >> > >>
> > > >> >
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > >> > >> > > > > > > > > >>>>>>>>>> JIRA:
> > > >> > >> > > > https://issues.apache.org/jira/browse/KAFKA-14402
> > > >> > >> > > > > > > > > >>>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>> Thanks!
> > > >> > >> > > > > > > > > >>>>>>>>>> Justine
> > > >> > >> > > > > > > > > >>>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>>
> > > >> > >> > > > > > > > > >>>>>>>
> > > >> > >> > > > > > > > > >>>>>>
> > > >> > >> > > > > > > > > >>>>>
> > > >> > >> > > > > > > > > >>>>
> > > >> > >> > > > > > > > > >>>
> > > >> > >> > > > > > > > > >>
> > > >> > >> > > > > > > > > >
> > > >> > >> > > > > > > > >
> > > >> > >> > > > > > > >
> > > >> > >> > > > > > >
> > > >> > >> > > > > >
> > > >> > >> > > > >
> > > >> > >> > > >
> > > >> > >> > >
> > > >> > >> >
> > > >> > >>
> > > >> > >
> > > >> >
> > > >>
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
Thanks for the discussion Artem.

With respect to the handling of fenced producers, we have some behavior
already in place. As of KIP-588:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts,
we handle timeouts more gracefully. The producer can recover.

Produce requests can also recover from epoch fencing by aborting the
transaction and starting over.

What other cases were you considering that would cause us to have a fenced
epoch but we'd want to recover?

The first point about handling epoch overflows is fair. I think there is
some logic we'd need to consider. (ie, if we are one away from the max
epoch, we need to reset the producer ID.) I'm still wondering if there is a
way to direct this from the response, or if everything should be done on
the client side. Let me know if you have any thoughts here.

Thanks,
Justine

On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits
<al...@confluent.io.invalid> wrote:

> There are some workflows in the client that are implied by protocol
> changes, e.g.:
>
> - for new clients, epoch changes with every transaction and can overflow,
> in old clients this condition was handled transparently, because epoch was
> bumped in InitProducerId and it would return a new producer id if epoch
> overflows, the new clients would need to implement some workflow to refresh
> producer id
> - how to handle fenced producers, for new clients epoch changes with every
> transaction, so in presence of failures during commits / aborts, the
> producer could get easily fenced, old clients would pretty much would get
> fenced when a new incarnation of the producer was initialized with
> InitProducerId so it's ok to treat as a fatal error, the new clients would
> need to implement some workflow to handle that error, otherwise they could
> get fenced by themselves
> - in particular (as a subset of the previous issue), what would the client
> do if it got a timeout during commit?  commit could've succeeded or failed
>
> Not sure if this has to be defined in the KIP as implementing those
> probably wouldn't require protocol changes, but we have multiple
> implementations of Kafka clients, so probably would be good to have some
> client implementation guidance.  Could also be done as a separate doc.
>
> -Artem
>
> On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan <jolshan@confluent.io.invalid
> >
> wrote:
>
> > Hey all, I've updated the KIP to incorporate Jason's suggestions.
> >
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> >
> >
> > 1. Use AddPartitionsToTxn + verify flag to check on old clients
> > 2. Updated AddPartitionsToTxn API to support transaction batching
> > 3. Mention IBP bump
> > 4. Mention auth change on new AddPartitionsToTxn version.
> >
> > I'm planning on opening a vote soon.
> > Thanks,
> > Justine
> >
> > On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <jo...@confluent.io>
> > wrote:
> >
> > > Thanks Jason. Those changes make sense to me. I will update the KIP.
> > >
> > >
> > >
> > > On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
> > <ja...@confluent.io.invalid>
> > > wrote:
> > >
> > >> Hey Justine,
> > >>
> > >> > I was wondering about compatibility here. When we send requests
> > >> between brokers, we want to ensure that the receiving broker
> understands
> > >> the request (specifically the new fields). Typically this is done via
> > >> IBP/metadata version.
> > >> I'm trying to think if there is a way around it but I'm not sure there
> > is.
> > >>
> > >> Yes. I think we would gate usage of this behind an IBP bump. Does that
> > >> seem
> > >> reasonable?
> > >>
> > >> > As for the improvements -- can you clarify how the multiple
> > >> transactional
> > >> IDs would help here? Were you thinking of a case where we wait/batch
> > >> multiple produce requests together? My understanding for now was 1
> > >> transactional ID and one validation per 1 produce request.
> > >>
> > >> Each call to `AddPartitionsToTxn` is essentially a write to the
> > >> transaction
> > >> log and must block on replication. The more we can fit into a single
> > >> request, the more writes we can do in parallel. The alternative is to
> > make
> > >> use of more connections, but usually we prefer batching since the
> > network
> > >> stack is not really optimized for high connection/request loads.
> > >>
> > >> > Finally with respect to the authorizations, I think it makes sense
> to
> > >> skip
> > >> topic authorizations, but I'm a bit confused by the "leader ID" field.
> > >> Wouldn't we just want to flag the request as from a broker (does it
> > matter
> > >> which one?).
> > >>
> > >> We could also make it version-based. For the next version, we could
> > >> require
> > >> CLUSTER auth. So clients would not be able to use the API anymore,
> which
> > >> is
> > >> probably what we want.
> > >>
> > >> -Jason
> > >>
> > >> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> > >> <jo...@confluent.io.invalid>
> > >> wrote:
> > >>
> > >> > As a follow up, I was just thinking about the batching a bit more.
> > >> > I suppose if we have one request in flight and we queue up the other
> > >> > produce requests in some sort of purgatory, we could send
> information
> > >> out
> > >> > for all of them rather than one by one. So that would be a benefit
> of
> > >> > batching partitions to add per transaction.
> > >> >
> > >> > I'll need to think a bit more on the design of this part of the KIP,
> > and
> > >> > will update the KIP in the next few days.
> > >> >
> > >> > Thanks,
> > >> > Justine
> > >> >
> > >> > On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan <
> jolshan@confluent.io>
> > >> > wrote:
> > >> >
> > >> > > Hey Jason -- thanks for the input -- I was just digging a bit
> deeper
> > >> into
> > >> > > the design + implementation of the validation calls here and what
> > you
> > >> say
> > >> > > makes sense.
> > >> > >
> > >> > > I was wondering about compatibility here. When we send requests
> > >> > > between brokers, we want to ensure that the receiving broker
> > >> understands
> > >> > > the request (specifically the new fields). Typically this is done
> > via
> > >> > > IBP/metadata version.
> > >> > > I'm trying to think if there is a way around it but I'm not sure
> > there
> > >> > is.
> > >> > >
> > >> > > As for the improvements -- can you clarify how the multiple
> > >> transactional
> > >> > > IDs would help here? Were you thinking of a case where we
> wait/batch
> > >> > > multiple produce requests together? My understanding for now was 1
> > >> > > transactional ID and one validation per 1 produce request.
> > >> > >
> > >> > > Finally with respect to the authorizations, I think it makes sense
> > to
> > >> > skip
> > >> > > topic authorizations, but I'm a bit confused by the "leader ID"
> > field.
> > >> > > Wouldn't we just want to flag the request as from a broker (does
> it
> > >> > matter
> > >> > > which one?).
> > >> > >
> > >> > > I think I want to adopt these suggestions, just had a few
> questions
> > on
> > >> > the
> > >> > > details.
> > >> > >
> > >> > > Thanks,
> > >> > > Justine
> > >> > >
> > >> > > On Thu, Jan 5, 2023 at 5:05 PM Jason Gustafson
> > >> > <ja...@confluent.io.invalid>
> > >> > > wrote:
> > >> > >
> > >> > >> Hi Justine,
> > >> > >>
> > >> > >> Thanks for the proposal.
> > >> > >>
> > >> > >> I was thinking about the implementation a little bit. In the
> > current
> > >> > >> proposal, the behavior depends on whether we have an old or new
> > >> client.
> > >> > >> For
> > >> > >> old clients, we send `DescribeTransactions` and verify the result
> > and
> > >> > for
> > >> > >> new clients, we send `AddPartitionsToTxn`. We might be able to
> > >> simplify
> > >> > >> the
> > >> > >> implementation if we can use the same request type. For example,
> > >> what if
> > >> > >> we
> > >> > >> bump the protocol version for `AddPartitionsToTxn` and add a
> > >> > >> `validateOnly`
> > >> > >> flag? For older versions, we can set `validateOnly=true` so that
> > the
> > >> > >> request only returns successfully if the partition had already
> been
> > >> > added.
> > >> > >> For new versions, we can set `validateOnly=false` and the
> partition
> > >> will
> > >> > >> be
> > >> > >> added to the transaction. The other slightly annoying thing that
> > this
> > >> > >> would
> > >> > >> get around is the need to collect the transaction state for all
> > >> > partitions
> > >> > >> even when we only care about a subset.
> > >> > >>
> > >> > >> Some additional improvements to consider:
> > >> > >>
> > >> > >> - We can give `AddPartitionsToTxn` better batch support for
> > >> inter-broker
> > >> > >> usage. Currently we only allow one `TransactionalId` to be
> > specified,
> > >> > but
> > >> > >> the broker may get some benefit being able to batch across
> multiple
> > >> > >> transactions.
> > >> > >> - Another small improvement is skipping topic authorization
> checks
> > >> for
> > >> > >> `AddPartitionsToTxn` when the request is from a broker. Perhaps
> we
> > >> can
> > >> > add
> > >> > >> a field for the `LeaderId` or something like that and require
> > CLUSTER
> > >> > >> permission when set.
> > >> > >>
> > >> > >> Best,
> > >> > >> Jason
> > >> > >>
> > >> > >>
> > >> > >>
> > >> > >> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao <jun@confluent.io.invalid
> >
> > >> > wrote:
> > >> > >>
> > >> > >> > Hi, Justine,
> > >> > >> >
> > >> > >> > Thanks for the explanation. It makes sense to me now.
> > >> > >> >
> > >> > >> > Jun
> > >> > >> >
> > >> > >> > On Mon, Dec 19, 2022 at 1:42 PM Justine Olshan
> > >> > >> > <jo...@confluent.io.invalid>
> > >> > >> > wrote:
> > >> > >> >
> > >> > >> > > Hi Jun,
> > >> > >> > >
> > >> > >> > > My understanding of the mechanism is that when we get to the
> > last
> > >> > >> epoch,
> > >> > >> > we
> > >> > >> > > increment to the fencing/last epoch and if any further
> requests
> > >> come
> > >> > >> in
> > >> > >> > for
> > >> > >> > > this producer ID they are fenced. Then the producer gets a
> new
> > ID
> > >> > and
> > >> > >> > > restarts with epoch/sequence 0. The fenced epoch sticks
> around
> > >> for
> > >> > the
> > >> > >> > > duration of producer.id.expiration.ms and blocks any late
> > >> messages
> > >> > >> > there.
> > >> > >> > > The new ID will get to take advantage of the improved
> semantics
> > >> > around
> > >> > >> > > non-zero start sequences. So I think we are covered.
> > >> > >> > >
> > >> > >> > > The only potential issue is overloading the cache, but
> > hopefully
> > >> the
> > >> > >> > > improvements (lowered producer.id.expiration.ms) will help
> > with
> > >> > that.
> > >> > >> > Let
> > >> > >> > > me know if you still have concerns.
> > >> > >> > >
> > >> > >> > > Thanks,
> > >> > >> > > Justine
> > >> > >> > >
> > >> > >> > > On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
> > >> <ju...@confluent.io.invalid>
> > >> > >> > wrote:
> > >> > >> > >
> > >> > >> > > > Hi, Justine,
> > >> > >> > > >
> > >> > >> > > > Thanks for the explanation.
> > >> > >> > > >
> > >> > >> > > > 70. The proposed fencing logic doesn't apply when pid
> > changes,
> > >> is
> > >> > >> that
> > >> > >> > > > right? If so, I am not sure how complete we are addressing
> > this
> > >> > >> issue
> > >> > >> > if
> > >> > >> > > > the pid changes more frequently.
> > >> > >> > > >
> > >> > >> > > > Thanks,
> > >> > >> > > >
> > >> > >> > > > Jun
> > >> > >> > > >
> > >> > >> > > >
> > >> > >> > > >
> > >> > >> > > > On Fri, Dec 16, 2022 at 9:16 AM Justine Olshan
> > >> > >> > > > <jo...@confluent.io.invalid>
> > >> > >> > > > wrote:
> > >> > >> > > >
> > >> > >> > > > > Hi Jun,
> > >> > >> > > > >
> > >> > >> > > > > Thanks for replying!
> > >> > >> > > > >
> > >> > >> > > > > 70.We already do the overflow mechanism, so my change
> would
> > >> just
> > >> > >> make
> > >> > >> > > it
> > >> > >> > > > > happen more often.
> > >> > >> > > > > I was also not suggesting a new field in the log, but in
> > the
> > >> > >> > response,
> > >> > >> > > > > which would be gated by the client version. Sorry if
> > >> something
> > >> > >> there
> > >> > >> > is
> > >> > >> > > > > unclear. I think we are starting to diverge.
> > >> > >> > > > > The goal of this KIP is to not change to the marker
> format
> > at
> > >> > all.
> > >> > >> > > > >
> > >> > >> > > > > 71. Yes, I guess I was going under the assumption that
> the
> > >> log
> > >> > >> would
> > >> > >> > > just
> > >> > >> > > > > look at its last epoch and treat it as the current
> epoch. I
> > >> > >> suppose
> > >> > >> > we
> > >> > >> > > > can
> > >> > >> > > > > have some special logic that if the last epoch was on a
> > >> marker
> > >> > we
> > >> > >> > > > actually
> > >> > >> > > > > expect the next epoch or something like that. We just
> need
> > to
> > >> > >> > > distinguish
> > >> > >> > > > > based on whether we had a commit/abort marker.
> > >> > >> > > > >
> > >> > >> > > > > 72.
> > >> > >> > > > > > if the producer epoch hasn't been bumped on the
> > >> > >> > > > > broker, it seems that the stucked message will fail the
> > >> sequence
> > >> > >> > > > validation
> > >> > >> > > > > and will be ignored. If the producer epoch has been
> bumped,
> > >> we
> > >> > >> ignore
> > >> > >> > > the
> > >> > >> > > > > sequence check and the stuck message could be appended to
> > the
> > >> > log.
> > >> > >> > So,
> > >> > >> > > is
> > >> > >> > > > > the latter case that we want to guard?
> > >> > >> > > > >
> > >> > >> > > > > I'm not sure I follow that "the message will fail the
> > >> sequence
> > >> > >> > > > validation".
> > >> > >> > > > > In some of these cases, we had an abort marker (due to an
> > >> error)
> > >> > >> and
> > >> > >> > > then
> > >> > >> > > > > the late message comes in with the correct sequence
> number.
> > >> This
> > >> > >> is a
> > >> > >> > > > case
> > >> > >> > > > > covered by the KIP.
> > >> > >> > > > > The latter case is actually not something we've
> considered
> > >> > here. I
> > >> > >> > > think
> > >> > >> > > > > generally when we bump the epoch, we are accepting that
> the
> > >> > >> sequence
> > >> > >> > > does
> > >> > >> > > > > not need to be checked anymore. My understanding is also
> > >> that we
> > >> > >> > don't
> > >> > >> > > > > typically bump epoch mid transaction (based on a quick
> look
> > >> at
> > >> > the
> > >> > >> > > code)
> > >> > >> > > > > but let me know if that is the case.
> > >> > >> > > > >
> > >> > >> > > > > Thanks,
> > >> > >> > > > > Justine
> > >> > >> > > > >
> > >> > >> > > > > On Thu, Dec 15, 2022 at 12:23 PM Jun Rao
> > >> > <jun@confluent.io.invalid
> > >> > >> >
> > >> > >> > > > wrote:
> > >> > >> > > > >
> > >> > >> > > > > > Hi, Justine,
> > >> > >> > > > > >
> > >> > >> > > > > > Thanks for the reply.
> > >> > >> > > > > >
> > >> > >> > > > > > 70. Assigning a new pid on int overflow seems a bit
> > hacky.
> > >> If
> > >> > we
> > >> > >> > > need a
> > >> > >> > > > > txn
> > >> > >> > > > > > level id, it will be better to model this explicitly.
> > >> Adding a
> > >> > >> new
> > >> > >> > > > field
> > >> > >> > > > > > would require a bit more work since it requires a new
> txn
> > >> > marker
> > >> > >> > > format
> > >> > >> > > > > in
> > >> > >> > > > > > the log. So, we probably need to guard it with an IBP
> or
> > >> > >> metadata
> > >> > >> > > > version
> > >> > >> > > > > > and document the impact on downgrade once the new
> format
> > is
> > >> > >> written
> > >> > >> > > to
> > >> > >> > > > > the
> > >> > >> > > > > > log.
> > >> > >> > > > > >
> > >> > >> > > > > > 71. Hmm, once the marker is written, the partition will
> > >> expect
> > >> > >> the
> > >> > >> > > next
> > >> > >> > > > > > append to be on the next epoch. Does that cover the
> case
> > >> you
> > >> > >> > > mentioned?
> > >> > >> > > > > >
> > >> > >> > > > > > 72. Also, just to be clear on the stucked message issue
> > >> > >> described
> > >> > >> > in
> > >> > >> > > > the
> > >> > >> > > > > > motivation. With EoS, we also validate the sequence id
> > for
> > >> > >> > > idempotency.
> > >> > >> > > > > So,
> > >> > >> > > > > > with the current logic, if the producer epoch hasn't
> been
> > >> > >> bumped on
> > >> > >> > > the
> > >> > >> > > > > > broker, it seems that the stucked message will fail the
> > >> > sequence
> > >> > >> > > > > validation
> > >> > >> > > > > > and will be ignored. If the producer epoch has been
> > >> bumped, we
> > >> > >> > ignore
> > >> > >> > > > the
> > >> > >> > > > > > sequence check and the stuck message could be appended
> to
> > >> the
> > >> > >> log.
> > >> > >> > > So,
> > >> > >> > > > is
> > >> > >> > > > > > the latter case that we want to guard?
> > >> > >> > > > > >
> > >> > >> > > > > > Thanks,
> > >> > >> > > > > >
> > >> > >> > > > > > Jun
> > >> > >> > > > > >
> > >> > >> > > > > > On Wed, Dec 14, 2022 at 10:44 AM Justine Olshan
> > >> > >> > > > > > <jo...@confluent.io.invalid> wrote:
> > >> > >> > > > > >
> > >> > >> > > > > > > Matthias — thanks again for taking time to look a
> this.
> > >> You
> > >> > >> said:
> > >> > >> > > > > > >
> > >> > >> > > > > > > > My proposal was only focusing to avoid dangling
> > >> > >> > > > > > >
> > >> > >> > > > > > > transactions if records are added without registered
> > >> > >> partition.
> > >> > >> > --
> > >> > >> > > > > Maybe
> > >> > >> > > > > > >
> > >> > >> > > > > > > you can add a few more details to the KIP about this
> > >> > scenario
> > >> > >> for
> > >> > >> > > > > better
> > >> > >> > > > > > >
> > >> > >> > > > > > > documentation purpose?
> > >> > >> > > > > > >
> > >> > >> > > > > > >
> > >> > >> > > > > > > I'm not sure I understand what you mean here. The
> > >> motivation
> > >> > >> > > section
> > >> > >> > > > > > > describes two scenarios about how the record can be
> > added
> > >> > >> > without a
> > >> > >> > > > > > > registered partition:
> > >> > >> > > > > > >
> > >> > >> > > > > > >
> > >> > >> > > > > > > > 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.
> > >> > >> > > > > > >
> > >> > >> > > > > > >
> > >> > >> > > > > > > > Another way hanging transactions can occur is that
> a
> > >> > client
> > >> > >> is
> > >> > >> > > > buggy
> > >> > >> > > > > > and
> > >> > >> > > > > > > may somehow try to write to a partition before it
> adds
> > >> the
> > >> > >> > > partition
> > >> > >> > > > to
> > >> > >> > > > > > the
> > >> > >> > > > > > > transaction.
> > >> > >> > > > > > >
> > >> > >> > > > > > >
> > >> > >> > > > > > >
> > >> > >> > > > > > > For the first example of this would it be helpful to
> > say
> > >> > that
> > >> > >> > this
> > >> > >> > > > > > message
> > >> > >> > > > > > > comes in after the abort, but before the partition is
> > >> added
> > >> > to
> > >> > >> > the
> > >> > >> > > > next
> > >> > >> > > > > > > transaction so it becomes "hanging." Perhaps the next
> > >> > sentence
> > >> > >> > > > > describing
> > >> > >> > > > > > > the message becoming part of the next transaction (a
> > >> > different
> > >> > >> > > case)
> > >> > >> > > > > was
> > >> > >> > > > > > > not properly differentiated.
> > >> > >> > > > > > >
> > >> > >> > > > > > >
> > >> > >> > > > > > >
> > >> > >> > > > > > > Jun — thanks for reading the KIP.
> > >> > >> > > > > > >
> > >> > >> > > > > > > 70. The int typing was a concern. Currently we have a
> > >> > >> mechanism
> > >> > >> > in
> > >> > >> > > > > place
> > >> > >> > > > > > to
> > >> > >> > > > > > > fence the final epoch when the epoch is about to
> > overflow
> > >> > and
> > >> > >> > > assign
> > >> > >> > > > a
> > >> > >> > > > > > new
> > >> > >> > > > > > > producer ID with epoch 0. Of course, this is a bit
> > tricky
> > >> > >> when it
> > >> > >> > > > comes
> > >> > >> > > > > > to
> > >> > >> > > > > > > the response back to the client.
> > >> > >> > > > > > > Making this a long could be another option, but I
> > wonder
> > >> are
> > >> > >> > there
> > >> > >> > > > any
> > >> > >> > > > > > > implications on changing this field if the epoch is
> > >> > persisted
> > >> > >> to
> > >> > >> > > > disk?
> > >> > >> > > > > > I'd
> > >> > >> > > > > > > need to check the usages.
> > >> > >> > > > > > >
> > >> > >> > > > > > > 71.This was something Matthias asked about as well. I
> > was
> > >> > >> > > > considering a
> > >> > >> > > > > > > possible edge case where a produce request from a new
> > >> > >> transaction
> > >> > >> > > > > somehow
> > >> > >> > > > > > > gets sent right after the marker is written, but
> before
> > >> the
> > >> > >> > > producer
> > >> > >> > > > is
> > >> > >> > > > > > > alerted of the newly bumped epoch. In this case, we
> may
> > >> > >> include
> > >> > >> > > this
> > >> > >> > > > > > record
> > >> > >> > > > > > > when we don't want to. I suppose we could try to do
> > >> > something
> > >> > >> > > client
> > >> > >> > > > > side
> > >> > >> > > > > > > to bump the epoch after sending an endTxn as well in
> > this
> > >> > >> > scenario
> > >> > >> > > —
> > >> > >> > > > > but
> > >> > >> > > > > > I
> > >> > >> > > > > > > wonder how it would work when the server is aborting
> > >> based
> > >> > on
> > >> > >> a
> > >> > >> > > > > > server-side
> > >> > >> > > > > > > error. I could also be missing something and this
> > >> scenario
> > >> > is
> > >> > >> > > > actually
> > >> > >> > > > > > not
> > >> > >> > > > > > > possible.
> > >> > >> > > > > > >
> > >> > >> > > > > > > Thanks again to everyone reading and commenting. Let
> me
> > >> know
> > >> > >> > about
> > >> > >> > > > any
> > >> > >> > > > > > > further questions or comments.
> > >> > >> > > > > > >
> > >> > >> > > > > > > Justine
> > >> > >> > > > > > >
> > >> > >> > > > > > > On Wed, Dec 14, 2022 at 9:41 AM Jun Rao
> > >> > >> <jun@confluent.io.invalid
> > >> > >> > >
> > >> > >> > > > > > wrote:
> > >> > >> > > > > > >
> > >> > >> > > > > > > > Hi, Justine,
> > >> > >> > > > > > > >
> > >> > >> > > > > > > > Thanks for the KIP. A couple of comments.
> > >> > >> > > > > > > >
> > >> > >> > > > > > > > 70. Currently, the producer epoch is an int. I am
> not
> > >> sure
> > >> > >> if
> > >> > >> > > it's
> > >> > >> > > > > > enough
> > >> > >> > > > > > > > to accommodate all transactions in the lifetime of
> a
> > >> > >> producer.
> > >> > >> > > > Should
> > >> > >> > > > > > we
> > >> > >> > > > > > > > change that to a long or add a new long field like
> > >> txnId?
> > >> > >> > > > > > > >
> > >> > >> > > > > > > > 71. "it will write the prepare commit message with
> a
> > >> > bumped
> > >> > >> > epoch
> > >> > >> > > > and
> > >> > >> > > > > > > send
> > >> > >> > > > > > > > WriteTxnMarkerRequests with the bumped epoch." Hmm,
> > the
> > >> > >> epoch
> > >> > >> > is
> > >> > >> > > > > > > associated
> > >> > >> > > > > > > > with the current txn right? So, it seems weird to
> > >> write a
> > >> > >> > commit
> > >> > >> > > > > > message
> > >> > >> > > > > > > > with a bumped epoch. Should we only bump up the
> epoch
> > >> in
> > >> > >> > > > > EndTxnResponse
> > >> > >> > > > > > > and
> > >> > >> > > > > > > > rename the field to sth like nextProducerEpoch?
> > >> > >> > > > > > > >
> > >> > >> > > > > > > > Thanks,
> > >> > >> > > > > > > >
> > >> > >> > > > > > > > Jun
> > >> > >> > > > > > > >
> > >> > >> > > > > > > >
> > >> > >> > > > > > > >
> > >> > >> > > > > > > > On Mon, Dec 12, 2022 at 8:54 PM Matthias J. Sax <
> > >> > >> > > mjsax@apache.org>
> > >> > >> > > > > > > wrote:
> > >> > >> > > > > > > >
> > >> > >> > > > > > > > > Thanks for the background.
> > >> > >> > > > > > > > >
> > >> > >> > > > > > > > > 20/30: SGTM. My proposal was only focusing to
> avoid
> > >> > >> dangling
> > >> > >> > > > > > > > > transactions if records are added without
> > registered
> > >> > >> > partition.
> > >> > >> > > > --
> > >> > >> > > > > > > Maybe
> > >> > >> > > > > > > > > you can add a few more details to the KIP about
> > this
> > >> > >> scenario
> > >> > >> > > for
> > >> > >> > > > > > > better
> > >> > >> > > > > > > > > documentation purpose?
> > >> > >> > > > > > > > >
> > >> > >> > > > > > > > > 40: I think you hit a fair point about race
> > >> conditions
> > >> > or
> > >> > >> > > client
> > >> > >> > > > > bugs
> > >> > >> > > > > > > > > (incorrectly not bumping the epoch). The
> > >> > >> complexity/confusion
> > >> > >> > > for
> > >> > >> > > > > > using
> > >> > >> > > > > > > > > the bumped epoch I see, is mainly for internal
> > >> > debugging,
> > >> > >> ie,
> > >> > >> > > > > > > inspecting
> > >> > >> > > > > > > > > log segment dumps -- it seems harder to reason
> > about
> > >> the
> > >> > >> > system
> > >> > >> > > > for
> > >> > >> > > > > > us
> > >> > >> > > > > > > > > humans. But if we get better guarantees, it would
> > be
> > >> > >> worth to
> > >> > >> > > use
> > >> > >> > > > > the
> > >> > >> > > > > > > > > bumped epoch.
> > >> > >> > > > > > > > >
> > >> > >> > > > > > > > > 60: as I mentioned already, I don't know the
> broker
> > >> > >> internals
> > >> > >> > > to
> > >> > >> > > > > > > provide
> > >> > >> > > > > > > > > more input. So if nobody else chimes in, we
> should
> > >> just
> > >> > >> move
> > >> > >> > > > > forward
> > >> > >> > > > > > > > > with your proposal.
> > >> > >> > > > > > > > >
> > >> > >> > > > > > > > >
> > >> > >> > > > > > > > > -Matthias
> > >> > >> > > > > > > > >
> > >> > >> > > > > > > > >
> > >> > >> > > > > > > > > On 12/6/22 4:22 PM, Justine Olshan wrote:
> > >> > >> > > > > > > > > > Hi all,
> > >> > >> > > > > > > > > > After Artem's questions about error behavior,
> > I've
> > >> > >> > > re-evaluated
> > >> > >> > > > > the
> > >> > >> > > > > > > > > > unknown producer ID exception and had some
> > >> discussions
> > >> > >> > > offline.
> > >> > >> > > > > > > > > >
> > >> > >> > > > > > > > > > I think generally it makes sense to simplify
> > error
> > >> > >> handling
> > >> > >> > > in
> > >> > >> > > > > > cases
> > >> > >> > > > > > > > like
> > >> > >> > > > > > > > > > this and the UNKNOWN_PRODUCER_ID error has a
> > pretty
> > >> > long
> > >> > >> > and
> > >> > >> > > > > > > > complicated
> > >> > >> > > > > > > > > > history. Because of this, I propose adding a
> new
> > >> error
> > >> > >> code
> > >> > >> > > > > > > > > ABORTABLE_ERROR
> > >> > >> > > > > > > > > > that when encountered by new clients (gated by
> > the
> > >> > >> produce
> > >> > >> > > > > request
> > >> > >> > > > > > > > > version)
> > >> > >> > > > > > > > > > will simply abort the transaction. This allows
> > the
> > >> > >> server
> > >> > >> > to
> > >> > >> > > > have
> > >> > >> > > > > > > some
> > >> > >> > > > > > > > > say
> > >> > >> > > > > > > > > > in whether the client aborts and makes handling
> > >> much
> > >> > >> > simpler.
> > >> > >> > > > In
> > >> > >> > > > > > the
> > >> > >> > > > > > > > > > future, we can also use this error in other
> > >> situations
> > >> > >> > where
> > >> > >> > > we
> > >> > >> > > > > > want
> > >> > >> > > > > > > to
> > >> > >> > > > > > > > > > abort the transactions. We can even use on
> other
> > >> apis.
> > >> > >> > > > > > > > > >
> > >> > >> > > > > > > > > > I've added this to the KIP. Let me know if
> there
> > >> are
> > >> > any
> > >> > >> > > > > questions
> > >> > >> > > > > > or
> > >> > >> > > > > > > > > > issues.
> > >> > >> > > > > > > > > >
> > >> > >> > > > > > > > > > Justine
> > >> > >> > > > > > > > > >
> > >> > >> > > > > > > > > > On Fri, Dec 2, 2022 at 10:22 AM Justine Olshan
> <
> > >> > >> > > > > > jolshan@confluent.io
> > >> > >> > > > > > > >
> > >> > >> > > > > > > > > wrote:
> > >> > >> > > > > > > > > >
> > >> > >> > > > > > > > > >> Hey Matthias,
> > >> > >> > > > > > > > > >>
> > >> > >> > > > > > > > > >>
> > >> > >> > > > > > > > > >> 20/30 — Maybe I also didn't express myself
> > >> clearly.
> > >> > For
> > >> > >> > > older
> > >> > >> > > > > > > clients
> > >> > >> > > > > > > > we
> > >> > >> > > > > > > > > >> don't have a way to distinguish between a
> > previous
> > >> > and
> > >> > >> the
> > >> > >> > > > > current
> > >> > >> > > > > > > > > >> transaction since we don't have the epoch
> bump.
> > >> This
> > >> > >> means
> > >> > >> > > > that
> > >> > >> > > > > a
> > >> > >> > > > > > > late
> > >> > >> > > > > > > > > >> message from the previous transaction may be
> > >> added to
> > >> > >> the
> > >> > >> > > new
> > >> > >> > > > > one.
> > >> > >> > > > > > > > With
> > >> > >> > > > > > > > > >> older clients — we can't guarantee this won't
> > >> happen
> > >> > >> if we
> > >> > >> > > > > already
> > >> > >> > > > > > > > sent
> > >> > >> > > > > > > > > the
> > >> > >> > > > > > > > > >> addPartitionsToTxn call (why we make changes
> for
> > >> the
> > >> > >> newer
> > >> > >> > > > > client)
> > >> > >> > > > > > > but
> > >> > >> > > > > > > > > we
> > >> > >> > > > > > > > > >> can at least gate some by ensuring that the
> > >> partition
> > >> > >> has
> > >> > >> > > been
> > >> > >> > > > > > added
> > >> > >> > > > > > > > to
> > >> > >> > > > > > > > > the
> > >> > >> > > > > > > > > >> transaction. The rationale here is that there
> > are
> > >> > >> likely
> > >> > >> > > LESS
> > >> > >> > > > > late
> > >> > >> > > > > > > > > arrivals
> > >> > >> > > > > > > > > >> as time goes on, so hopefully most late
> arrivals
> > >> will
> > >> > >> come
> > >> > >> > > in
> > >> > >> > > > > > BEFORE
> > >> > >> > > > > > > > the
> > >> > >> > > > > > > > > >> addPartitionsToTxn call. Those that arrive
> > before
> > >> > will
> > >> > >> be
> > >> > >> > > > > properly
> > >> > >> > > > > > > > gated
> > >> > >> > > > > > > > > >> with the describeTransactions approach.
> > >> > >> > > > > > > > > >>
> > >> > >> > > > > > > > > >> If we take the approach you suggested, ANY
> late
> > >> > arrival
> > >> > >> > > from a
> > >> > >> > > > > > > > previous
> > >> > >> > > > > > > > > >> transaction will be added. And we don't want
> > >> that. I
> > >> > >> also
> > >> > >> > > > don't
> > >> > >> > > > > > see
> > >> > >> > > > > > > > any
> > >> > >> > > > > > > > > >> benefit in sending addPartitionsToTxn over the
> > >> > >> > describeTxns
> > >> > >> > > > > call.
> > >> > >> > > > > > > They
> > >> > >> > > > > > > > > will
> > >> > >> > > > > > > > > >> both be one extra RPC to the Txn coordinator.
> > >> > >> > > > > > > > > >>
> > >> > >> > > > > > > > > >>
> > >> > >> > > > > > > > > >> To be clear — newer clients will use
> > >> > addPartitionsToTxn
> > >> > >> > > > instead
> > >> > >> > > > > of
> > >> > >> > > > > > > the
> > >> > >> > > > > > > > > >> DescribeTxns.
> > >> > >> > > > > > > > > >>
> > >> > >> > > > > > > > > >>
> > >> > >> > > > > > > > > >> 40)
> > >> > >> > > > > > > > > >> My concern is that if we have some delay in
> the
> > >> > client
> > >> > >> to
> > >> > >> > > bump
> > >> > >> > > > > the
> > >> > >> > > > > > > > > epoch,
> > >> > >> > > > > > > > > >> it could continue to send epoch 73 and those
> > >> records
> > >> > >> would
> > >> > >> > > not
> > >> > >> > > > > be
> > >> > >> > > > > > > > > fenced.
> > >> > >> > > > > > > > > >> Perhaps this is not an issue if we don't allow
> > the
> > >> > next
> > >> > >> > > > produce
> > >> > >> > > > > to
> > >> > >> > > > > > > go
> > >> > >> > > > > > > > > >> through before the EndTxn request returns. I'm
> > >> also
> > >> > >> > thinking
> > >> > >> > > > > about
> > >> > >> > > > > > > > > cases of
> > >> > >> > > > > > > > > >> failure. I will need to think on this a bit.
> > >> > >> > > > > > > > > >>
> > >> > >> > > > > > > > > >> I wasn't sure if it was that confusing. But if
> > we
> > >> > >> think it
> > >> > >> > > is,
> > >> > >> > > > > we
> > >> > >> > > > > > > can
> > >> > >> > > > > > > > > >> investigate other ways.
> > >> > >> > > > > > > > > >>
> > >> > >> > > > > > > > > >>
> > >> > >> > > > > > > > > >> 60)
> > >> > >> > > > > > > > > >>
> > >> > >> > > > > > > > > >> I'm not sure these are the same purgatories
> > since
> > >> one
> > >> > >> is a
> > >> > >> > > > > produce
> > >> > >> > > > > > > > > >> purgatory (I was planning on using a callback
> > >> rather
> > >> > >> than
> > >> > >> > > > > > purgatory)
> > >> > >> > > > > > > > and
> > >> > >> > > > > > > > > >> the other is simply a request to append to the
> > >> log.
> > >> > Not
> > >> > >> > sure
> > >> > >> > > > we
> > >> > >> > > > > > have
> > >> > >> > > > > > > > any
> > >> > >> > > > > > > > > >> structure here for ordering, but my
> > understanding
> > >> is
> > >> > >> that
> > >> > >> > > the
> > >> > >> > > > > > broker
> > >> > >> > > > > > > > > could
> > >> > >> > > > > > > > > >> handle the write request before it hears back
> > from
> > >> > the
> > >> > >> Txn
> > >> > >> > > > > > > > Coordinator.
> > >> > >> > > > > > > > > >>
> > >> > >> > > > > > > > > >> Let me know if I misunderstood something or
> > >> something
> > >> > >> was
> > >> > >> > > > > unclear.
> > >> > >> > > > > > > > > >>
> > >> > >> > > > > > > > > >> Justine
> > >> > >> > > > > > > > > >>
> > >> > >> > > > > > > > > >> On Thu, Dec 1, 2022 at 12:15 PM Matthias J.
> Sax
> > <
> > >> > >> > > > > mjsax@apache.org
> > >> > >> > > > > > >
> > >> > >> > > > > > > > > wrote:
> > >> > >> > > > > > > > > >>
> > >> > >> > > > > > > > > >>> Thanks for the details Justine!
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > > >>>> 20)
> > >> > >> > > > > > > > > >>>>
> > >> > >> > > > > > > > > >>>> The client side change for 2 is removing the
> > >> > >> > addPartitions
> > >> > >> > > > to
> > >> > >> > > > > > > > > >>> transaction
> > >> > >> > > > > > > > > >>>> call. We don't need to make this from the
> > >> producer
> > >> > to
> > >> > >> > the
> > >> > >> > > > txn
> > >> > >> > > > > > > > > >>> coordinator,
> > >> > >> > > > > > > > > >>>> only server side.
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > > >>> I think I did not express myself clearly. I
> > >> > understand
> > >> > >> > that
> > >> > >> > > > we
> > >> > >> > > > > > can
> > >> > >> > > > > > > > (and
> > >> > >> > > > > > > > > >>> should) change the producer to not send the
> > >> > >> > `addPartitions`
> > >> > >> > > > > > request
> > >> > >> > > > > > > > any
> > >> > >> > > > > > > > > >>> longer. But I don't thinks it's requirement
> to
> > >> > change
> > >> > >> the
> > >> > >> > > > > broker?
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > > >>> What I am trying to say is: as a safe-guard
> and
> > >> > >> > improvement
> > >> > >> > > > for
> > >> > >> > > > > > > older
> > >> > >> > > > > > > > > >>> producers, the partition leader can just send
> > the
> > >> > >> > > > > `addPartitions`
> > >> > >> > > > > > > > > >>> request to the TX-coordinator in any case --
> if
> > >> the
> > >> > >> old
> > >> > >> > > > > producer
> > >> > >> > > > > > > > > >>> correctly did send the `addPartition` request
> > to
> > >> the
> > >> > >> > > > > > TX-coordinator
> > >> > >> > > > > > > > > >>> already, the TX-coordinator can just "ignore"
> > is
> > >> as
> > >> > >> > > > idempotent.
> > >> > >> > > > > > > > > However,
> > >> > >> > > > > > > > > >>> if the old producer has a bug and did forget
> to
> > >> sent
> > >> > >> the
> > >> > >> > > > > > > > `addPartition`
> > >> > >> > > > > > > > > >>> request, we would now ensure that the
> partition
> > >> is
> > >> > >> indeed
> > >> > >> > > > added
> > >> > >> > > > > > to
> > >> > >> > > > > > > > the
> > >> > >> > > > > > > > > >>> TX and thus fix a potential producer bug
> (even
> > >> if we
> > >> > >> > don't
> > >> > >> > > > get
> > >> > >> > > > > > the
> > >> > >> > > > > > > > > >>> fencing via the bump epoch). -- It seems to
> be
> > a
> > >> > good
> > >> > >> > > > > > improvement?
> > >> > >> > > > > > > Or
> > >> > >> > > > > > > > > is
> > >> > >> > > > > > > > > >>> there a reason to not do this?
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > > >>>> 30)
> > >> > >> > > > > > > > > >>>>
> > >> > >> > > > > > > > > >>>> Transaction is ongoing = partition was added
> > to
> > >> > >> > > transaction
> > >> > >> > > > > via
> > >> > >> > > > > > > > > >>>> addPartitionsToTxn. We check this with the
> > >> > >> > > > > DescribeTransactions
> > >> > >> > > > > > > > call.
> > >> > >> > > > > > > > > >>> Let
> > >> > >> > > > > > > > > >>>> me know if this wasn't sufficiently
> explained
> > >> here:
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > > >>> If we do what I propose in (20), we don't
> > really
> > >> > need
> > >> > >> to
> > >> > >> > > make
> > >> > >> > > > > > this
> > >> > >> > > > > > > > > >>> `DescribeTransaction` call, as the partition
> > >> leader
> > >> > >> adds
> > >> > >> > > the
> > >> > >> > > > > > > > partition
> > >> > >> > > > > > > > > >>> for older clients and we get this check for
> > free.
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > > >>>> 40)
> > >> > >> > > > > > > > > >>>>
> > >> > >> > > > > > > > > >>>> The idea here is that if any messages
> somehow
> > >> come
> > >> > in
> > >> > >> > > before
> > >> > >> > > > > we
> > >> > >> > > > > > > get
> > >> > >> > > > > > > > > the
> > >> > >> > > > > > > > > >>> new
> > >> > >> > > > > > > > > >>>> epoch to the producer, they will be fenced.
> > >> > However,
> > >> > >> if
> > >> > >> > we
> > >> > >> > > > > don't
> > >> > >> > > > > > > > think
> > >> > >> > > > > > > > > >>> this
> > >> > >> > > > > > > > > >>>> is necessary, it can be discussed
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > > >>> I agree that we should have epoch fencing. My
> > >> > >> question is
> > >> > >> > > > > > > different:
> > >> > >> > > > > > > > > >>> Assume we are at epoch 73, and we have an
> > ongoing
> > >> > >> > > > transaction,
> > >> > >> > > > > > that
> > >> > >> > > > > > > > is
> > >> > >> > > > > > > > > >>> committed. It seems natural to write the
> > "prepare
> > >> > >> commit"
> > >> > >> > > > > marker
> > >> > >> > > > > > > and
> > >> > >> > > > > > > > > the
> > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` both with epoch 73,
> too,
> > >> as
> > >> > it
> > >> > >> > > belongs
> > >> > >> > > > > to
> > >> > >> > > > > > > the
> > >> > >> > > > > > > > > >>> current transaction. Of course, we now also
> > bump
> > >> the
> > >> > >> > epoch
> > >> > >> > > > and
> > >> > >> > > > > > > expect
> > >> > >> > > > > > > > > >>> the next requests to have epoch 74, and would
> > >> reject
> > >> > >> an
> > >> > >> > > > request
> > >> > >> > > > > > > with
> > >> > >> > > > > > > > > >>> epoch 73, as the corresponding TX for epoch
> 73
> > >> was
> > >> > >> > already
> > >> > >> > > > > > > committed.
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > > >>> It seems you propose to write the "prepare
> > commit
> > >> > >> marker"
> > >> > >> > > and
> > >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` with epoch 74 though,
> > what
> > >> > >> would
> > >> > >> > > work,
> > >> > >> > > > > but
> > >> > >> > > > > > > it
> > >> > >> > > > > > > > > >>> seems confusing. Is there a reason why we
> would
> > >> use
> > >> > >> the
> > >> > >> > > > bumped
> > >> > >> > > > > > > epoch
> > >> > >> > > > > > > > 74
> > >> > >> > > > > > > > > >>> instead of the current epoch 73?
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > > >>>> 60)
> > >> > >> > > > > > > > > >>>>
> > >> > >> > > > > > > > > >>>> When we are checking if the transaction is
> > >> ongoing,
> > >> > >> we
> > >> > >> > > need
> > >> > >> > > > to
> > >> > >> > > > > > > make
> > >> > >> > > > > > > > a
> > >> > >> > > > > > > > > >>> round
> > >> > >> > > > > > > > > >>>> trip from the leader partition to the
> > >> transaction
> > >> > >> > > > coordinator.
> > >> > >> > > > > > In
> > >> > >> > > > > > > > the
> > >> > >> > > > > > > > > >>> time
> > >> > >> > > > > > > > > >>>> we are waiting for this message to come
> back,
> > in
> > >> > >> theory
> > >> > >> > we
> > >> > >> > > > > could
> > >> > >> > > > > > > > have
> > >> > >> > > > > > > > > >>> sent
> > >> > >> > > > > > > > > >>>> a commit/abort call that would make the
> > original
> > >> > >> result
> > >> > >> > of
> > >> > >> > > > the
> > >> > >> > > > > > > check
> > >> > >> > > > > > > > > >>> out of
> > >> > >> > > > > > > > > >>>> date. That is why we can check the leader
> > state
> > >> > >> before
> > >> > >> > we
> > >> > >> > > > > write
> > >> > >> > > > > > to
> > >> > >> > > > > > > > the
> > >> > >> > > > > > > > > >>> log.
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > > >>> Thanks. Got it.
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > > >>> However, is this really an issue? We put the
> > >> produce
> > >> > >> > > request
> > >> > >> > > > in
> > >> > >> > > > > > > > > >>> purgatory, so how could we process the
> > >> > >> > > > `WriteTxnMarkerRequest`
> > >> > >> > > > > > > first?
> > >> > >> > > > > > > > > >>> Don't we need to put the
> > `WriteTxnMarkerRequest`
> > >> > into
> > >> > >> > > > > purgatory,
> > >> > >> > > > > > > too,
> > >> > >> > > > > > > > > >>> for this case, and process both request
> > in-order?
> > >> > >> (Again,
> > >> > >> > > my
> > >> > >> > > > > > broker
> > >> > >> > > > > > > > > >>> knowledge is limited and maybe we don't
> > maintain
> > >> > >> request
> > >> > >> > > > order
> > >> > >> > > > > > for
> > >> > >> > > > > > > > this
> > >> > >> > > > > > > > > >>> case, what seems to be an issue IMHO, and I
> am
> > >> > >> wondering
> > >> > >> > if
> > >> > >> > > > > > > changing
> > >> > >> > > > > > > > > >>> request handling to preserve order for this
> > case
> > >> > >> might be
> > >> > >> > > the
> > >> > >> > > > > > > cleaner
> > >> > >> > > > > > > > > >>> solution?)
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > > >>> -Matthias
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > > >>> On 11/30/22 3:28 PM, Artem Livshits wrote:
> > >> > >> > > > > > > > > >>>> Hi Justine,
> > >> > >> > > > > > > > > >>>>
> > >> > >> > > > > > > > > >>>> I think the interesting part is not in this
> > >> logic
> > >> > >> > (because
> > >> > >> > > > it
> > >> > >> > > > > > > tries
> > >> > >> > > > > > > > to
> > >> > >> > > > > > > > > >>>> figure out when UNKNOWN_PRODUCER_ID is
> > retriable
> > >> > and
> > >> > >> if
> > >> > >> > > it's
> > >> > >> > > > > > > > > retryable,
> > >> > >> > > > > > > > > >>>> it's definitely not fatal), but what happens
> > >> when
> > >> > >> this
> > >> > >> > > logic
> > >> > >> > > > > > > doesn't
> > >> > >> > > > > > > > > >>> return
> > >> > >> > > > > > > > > >>>> 'true' and falls through.  In the old
> clients
> > it
> > >> > >> seems
> > >> > >> > to
> > >> > >> > > be
> > >> > >> > > > > > > fatal,
> > >> > >> > > > > > > > if
> > >> > >> > > > > > > > > >>> we
> > >> > >> > > > > > > > > >>>> keep the behavior in the new clients, I'd
> > >> expect it
> > >> > >> > would
> > >> > >> > > be
> > >> > >> > > > > > fatal
> > >> > >> > > > > > > > as
> > >> > >> > > > > > > > > >>> well.
> > >> > >> > > > > > > > > >>>>
> > >> > >> > > > > > > > > >>>> -Artem
> > >> > >> > > > > > > > > >>>>
> > >> > >> > > > > > > > > >>>> On Tue, Nov 29, 2022 at 11:57 AM Justine
> > Olshan
> > >> > >> > > > > > > > > >>>> <jo...@confluent.io.invalid> wrote:
> > >> > >> > > > > > > > > >>>>
> > >> > >> > > > > > > > > >>>>> Hi Artem and Jeff,
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>> Thanks for taking a look and sorry for the
> > slow
> > >> > >> > response.
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>> You both mentioned the change to handle
> > >> > >> > > UNKNOWN_PRODUCER_ID
> > >> > >> > > > > > > errors.
> > >> > >> > > > > > > > > To
> > >> > >> > > > > > > > > >>> be
> > >> > >> > > > > > > > > >>>>> clear — this error code will only be sent
> > again
> > >> > when
> > >> > >> > the
> > >> > >> > > > > > client's
> > >> > >> > > > > > > > > >>> request
> > >> > >> > > > > > > > > >>>>> version is high enough to ensure we handle
> it
> > >> > >> > correctly.
> > >> > >> > > > > > > > > >>>>> The current (Java) client handles this by
> the
> > >> > >> following
> > >> > >> > > > > > (somewhat
> > >> > >> > > > > > > > > long)
> > >> > >> > > > > > > > > >>>>> code snippet:
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>> // An UNKNOWN_PRODUCER_ID means that we
> have
> > >> lost
> > >> > >> the
> > >> > >> > > > > producer
> > >> > >> > > > > > > > state
> > >> > >> > > > > > > > > >>> on the
> > >> > >> > > > > > > > > >>>>> broker. Depending on the log start
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>> // offset, we may want to retry these, as
> > >> > described
> > >> > >> for
> > >> > >> > > > each
> > >> > >> > > > > > case
> > >> > >> > > > > > > > > >>> below. If
> > >> > >> > > > > > > > > >>>>> none of those apply, then for the
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>> // idempotent producer, we will locally
> bump
> > >> the
> > >> > >> epoch
> > >> > >> > > and
> > >> > >> > > > > > reset
> > >> > >> > > > > > > > the
> > >> > >> > > > > > > > > >>>>> sequence numbers of in-flight batches from
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>> // sequence 0, then retry the failed batch,
> > >> which
> > >> > >> > should
> > >> > >> > > > now
> > >> > >> > > > > > > > succeed.
> > >> > >> > > > > > > > > >>> For
> > >> > >> > > > > > > > > >>>>> the transactional producer, allow the
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>> // batch to fail. When processing the
> failed
> > >> > batch,
> > >> > >> we
> > >> > >> > > will
> > >> > >> > > > > > > > > transition
> > >> > >> > > > > > > > > >>> to
> > >> > >> > > > > > > > > >>>>> an abortable error and set a flag
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>> // indicating that we need to bump the
> epoch
> > >> (if
> > >> > >> > > supported
> > >> > >> > > > by
> > >> > >> > > > > > the
> > >> > >> > > > > > > > > >>> broker).
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>> if (error == Errors.*UNKNOWN_PRODUCER_ID*)
> {
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>       if (response.logStartOffset == -1) {
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           // We don't know the log start
> > offset
> > >> > with
> > >> > >> > this
> > >> > >> > > > > > > response.
> > >> > >> > > > > > > > > We
> > >> > >> > > > > > > > > >>> should
> > >> > >> > > > > > > > > >>>>> just retry the request until we get it.
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           // The UNKNOWN_PRODUCER_ID error
> > code
> > >> > was
> > >> > >> > added
> > >> > >> > > > > along
> > >> > >> > > > > > > > with
> > >> > >> > > > > > > > > >>> the new
> > >> > >> > > > > > > > > >>>>> ProduceResponse which includes the
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           // logStartOffset. So the '-1'
> > >> sentinel
> > >> > is
> > >> > >> > not
> > >> > >> > > > for
> > >> > >> > > > > > > > backward
> > >> > >> > > > > > > > > >>>>> compatibility. Instead, it is possible for
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           // a broker to not know the
> > >> > >> logStartOffset at
> > >> > >> > > > when
> > >> > >> > > > > it
> > >> > >> > > > > > > is
> > >> > >> > > > > > > > > >>> returning
> > >> > >> > > > > > > > > >>>>> the response because the partition
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           // may have moved away from the
> > >> broker
> > >> > >> from
> > >> > >> > the
> > >> > >> > > > > time
> > >> > >> > > > > > > the
> > >> > >> > > > > > > > > >>> error was
> > >> > >> > > > > > > > > >>>>> initially raised to the time the
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           // response was being
> constructed.
> > In
> > >> > >> these
> > >> > >> > > > cases,
> > >> > >> > > > > we
> > >> > >> > > > > > > > > should
> > >> > >> > > > > > > > > >>> just
> > >> > >> > > > > > > > > >>>>> retry the request: we are guaranteed
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           // to eventually get a
> > logStartOffset
> > >> > once
> > >> > >> > > things
> > >> > >> > > > > > > settle
> > >> > >> > > > > > > > > down.
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           return true;
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>       }
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>       if (batch.sequenceHasBeenReset()) {
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           // When the first inflight batch
> > >> fails
> > >> > >> due to
> > >> > >> > > the
> > >> > >> > > > > > > > > truncation
> > >> > >> > > > > > > > > >>> case,
> > >> > >> > > > > > > > > >>>>> then the sequences of all the other
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           // in flight batches would have
> > been
> > >> > >> > restarted
> > >> > >> > > > from
> > >> > >> > > > > > the
> > >> > >> > > > > > > > > >>> beginning.
> > >> > >> > > > > > > > > >>>>> However, when those responses
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           // come back from the broker,
> they
> > >> would
> > >> > >> also
> > >> > >> > > > come
> > >> > >> > > > > > with
> > >> > >> > > > > > > > an
> > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID error. In this case, we
> > >> should
> > >> > >> not
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           // reset the sequence numbers to
> > the
> > >> > >> > beginning.
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           return true;
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>       } else if
> > >> > >> > > > > (lastAckedOffset(batch.topicPartition).orElse(
> > >> > >> > > > > > > > > >>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > >> > >> > > > response.logStartOffset) {
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           // The head of the log has been
> > >> removed,
> > >> > >> > > probably
> > >> > >> > > > > due
> > >> > >> > > > > > > to
> > >> > >> > > > > > > > > the
> > >> > >> > > > > > > > > >>>>> retention time elapsing. In this case,
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           // we expect to lose the producer
> > >> state.
> > >> > >> For
> > >> > >> > > the
> > >> > >> > > > > > > > > transactional
> > >> > >> > > > > > > > > >>>>> producer, reset the sequences of all
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           // inflight batches to be from
> the
> > >> > >> beginning
> > >> > >> > > and
> > >> > >> > > > > > retry
> > >> > >> > > > > > > > > them,
> > >> > >> > > > > > > > > >>> so
> > >> > >> > > > > > > > > >>>>> that the transaction does not need to
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           // be aborted. For the idempotent
> > >> > >> producer,
> > >> > >> > > bump
> > >> > >> > > > > the
> > >> > >> > > > > > > > epoch
> > >> > >> > > > > > > > > to
> > >> > >> > > > > > > > > >>> avoid
> > >> > >> > > > > > > > > >>>>> reusing (sequence, epoch) pairs
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           if (isTransactional()) {
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>
> > >> > >> > > >
> > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > >> > >> > > > > > > > > >>>>> this.producerIdAndEpoch);
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           } else {
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > >  requestEpochBumpForPartition(batch.topicPartition);
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           }
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           return true;
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>       }
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>       if (!isTransactional()) {
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           // For the idempotent producer,
> > >> always
> > >> > >> retry
> > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > >> > >> > > > > > > > > >>>>> errors. If the batch has the current
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           // producer ID and epoch,
> request a
> > >> bump
> > >> > >> of
> > >> > >> > the
> > >> > >> > > > > > epoch.
> > >> > >> > > > > > > > > >>> Otherwise
> > >> > >> > > > > > > > > >>>>> just retry the produce.
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > >  requestEpochBumpForPartition(batch.topicPartition);
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>           return true;
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>       }
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>> }
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>> I was considering keeping this behavior —
> but
> > >> am
> > >> > >> open
> > >> > >> > to
> > >> > >> > > > > > > > simplifying
> > >> > >> > > > > > > > > >>> it.
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>> We are leaving changes to older clients off
> > the
> > >> > >> table
> > >> > >> > > here
> > >> > >> > > > > > since
> > >> > >> > > > > > > it
> > >> > >> > > > > > > > > >>> caused
> > >> > >> > > > > > > > > >>>>> many issues for clients in the past.
> > Previously
> > >> > this
> > >> > >> > was
> > >> > >> > > a
> > >> > >> > > > > > fatal
> > >> > >> > > > > > > > > error
> > >> > >> > > > > > > > > >>> and
> > >> > >> > > > > > > > > >>>>> we didn't have the mechanisms in place to
> > >> detect
> > >> > >> when
> > >> > >> > > this
> > >> > >> > > > > was
> > >> > >> > > > > > a
> > >> > >> > > > > > > > > >>> legitimate
> > >> > >> > > > > > > > > >>>>> case vs some bug or gap in the protocol.
> > >> Ensuring
> > >> > >> each
> > >> > >> > > > > > > transaction
> > >> > >> > > > > > > > > has
> > >> > >> > > > > > > > > >>> its
> > >> > >> > > > > > > > > >>>>> own epoch should close this gap.
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>> And to address Jeff's second point:
> > >> > >> > > > > > > > > >>>>> *does the typical produce request path
> append
> > >> > >> records
> > >> > >> > to
> > >> > >> > > > > local
> > >> > >> > > > > > > log
> > >> > >> > > > > > > > > >>> along*
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>> *with the currentTxnFirstOffset
> information?
> > I
> > >> > would
> > >> > >> > like
> > >> > >> > > > to
> > >> > >> > > > > > > > > >>> understand*
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>> *when the field is written to disk.*
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>> Yes, the first produce request populates
> this
> > >> > field
> > >> > >> and
> > >> > >> > > > > writes
> > >> > >> > > > > > > the
> > >> > >> > > > > > > > > >>> offset
> > >> > >> > > > > > > > > >>>>> as part of the record batch and also to the
> > >> > producer
> > >> > >> > > state
> > >> > >> > > > > > > > snapshot.
> > >> > >> > > > > > > > > >>> When
> > >> > >> > > > > > > > > >>>>> we reload the records on restart and/or
> > >> > >> reassignment,
> > >> > >> > we
> > >> > >> > > > > > > repopulate
> > >> > >> > > > > > > > > >>> this
> > >> > >> > > > > > > > > >>>>> field with the snapshot from disk along
> with
> > >> the
> > >> > >> rest
> > >> > >> > of
> > >> > >> > > > the
> > >> > >> > > > > > > > producer
> > >> > >> > > > > > > > > >>>>> state.
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>> Let me know if there are further comments
> > >> and/or
> > >> > >> > > questions.
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>> Thanks,
> > >> > >> > > > > > > > > >>>>> Justine
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>> On Tue, Nov 22, 2022 at 9:00 PM Jeff Kim
> > >> > >> > > > > > > > > <jeff.kim@confluent.io.invalid
> > >> > >> > > > > > > > > >>>>
> > >> > >> > > > > > > > > >>>>> wrote:
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>>> Hi Justine,
> > >> > >> > > > > > > > > >>>>>>
> > >> > >> > > > > > > > > >>>>>> Thanks for the KIP! I have two questions:
> > >> > >> > > > > > > > > >>>>>>
> > >> > >> > > > > > > > > >>>>>> 1) For new clients, we can once again
> return
> > >> an
> > >> > >> error
> > >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > >> > >> > > > > > > > > >>>>>> for sequences
> > >> > >> > > > > > > > > >>>>>> that are non-zero when there is no
> producer
> > >> state
> > >> > >> > > present
> > >> > >> > > > on
> > >> > >> > > > > > the
> > >> > >> > > > > > > > > >>> server.
> > >> > >> > > > > > > > > >>>>>> This will indicate we missed the 0
> sequence
> > >> and
> > >> > we
> > >> > >> > don't
> > >> > >> > > > yet
> > >> > >> > > > > > > want
> > >> > >> > > > > > > > to
> > >> > >> > > > > > > > > >>>>> write
> > >> > >> > > > > > > > > >>>>>> to the log.
> > >> > >> > > > > > > > > >>>>>>
> > >> > >> > > > > > > > > >>>>>> I would like to understand the current
> > >> behavior
> > >> > to
> > >> > >> > > handle
> > >> > >> > > > > > older
> > >> > >> > > > > > > > > >>> clients,
> > >> > >> > > > > > > > > >>>>>> and if there are any changes we are
> making.
> > >> Maybe
> > >> > >> I'm
> > >> > >> > > > > missing
> > >> > >> > > > > > > > > >>> something,
> > >> > >> > > > > > > > > >>>>>> but we would want to identify whether we
> > >> missed
> > >> > >> the 0
> > >> > >> > > > > sequence
> > >> > >> > > > > > > for
> > >> > >> > > > > > > > > >>> older
> > >> > >> > > > > > > > > >>>>>> clients, no?
> > >> > >> > > > > > > > > >>>>>>
> > >> > >> > > > > > > > > >>>>>> 2) Upon returning from the transaction
> > >> > >> coordinator, we
> > >> > >> > > can
> > >> > >> > > > > set
> > >> > >> > > > > > > the
> > >> > >> > > > > > > > > >>>>>> transaction
> > >> > >> > > > > > > > > >>>>>> as ongoing on the leader by populating
> > >> > >> > > > currentTxnFirstOffset
> > >> > >> > > > > > > > > >>>>>> through the typical produce request
> > handling.
> > >> > >> > > > > > > > > >>>>>>
> > >> > >> > > > > > > > > >>>>>> does the typical produce request path
> append
> > >> > >> records
> > >> > >> > to
> > >> > >> > > > > local
> > >> > >> > > > > > > log
> > >> > >> > > > > > > > > >>> along
> > >> > >> > > > > > > > > >>>>>> with the currentTxnFirstOffset
> information?
> > I
> > >> > would
> > >> > >> > like
> > >> > >> > > > to
> > >> > >> > > > > > > > > understand
> > >> > >> > > > > > > > > >>>>>> when the field is written to disk.
> > >> > >> > > > > > > > > >>>>>>
> > >> > >> > > > > > > > > >>>>>> Thanks,
> > >> > >> > > > > > > > > >>>>>> Jeff
> > >> > >> > > > > > > > > >>>>>>
> > >> > >> > > > > > > > > >>>>>>
> > >> > >> > > > > > > > > >>>>>> On Tue, Nov 22, 2022 at 4:44 PM Artem
> > Livshits
> > >> > >> > > > > > > > > >>>>>> <al...@confluent.io.invalid> wrote:
> > >> > >> > > > > > > > > >>>>>>
> > >> > >> > > > > > > > > >>>>>>> Hi Justine,
> > >> > >> > > > > > > > > >>>>>>>
> > >> > >> > > > > > > > > >>>>>>> Thank you for the KIP.  I have one
> > question.
> > >> > >> > > > > > > > > >>>>>>>
> > >> > >> > > > > > > > > >>>>>>> 5) For new clients, we can once again
> > return
> > >> an
> > >> > >> error
> > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > >> > >> > > > > > > > > >>>>>>>
> > >> > >> > > > > > > > > >>>>>>> I believe we had problems in the past
> with
> > >> > >> returning
> > >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > >> > >> > > > > > > > > >>>>>>> because it was considered fatal and
> > required
> > >> > >> client
> > >> > >> > > > > restart.
> > >> > >> > > > > > > It
> > >> > >> > > > > > > > > >>> would
> > >> > >> > > > > > > > > >>>>> be
> > >> > >> > > > > > > > > >>>>>>> good to spell out the new client behavior
> > >> when
> > >> > it
> > >> > >> > > > receives
> > >> > >> > > > > > the
> > >> > >> > > > > > > > > error.
> > >> > >> > > > > > > > > >>>>>>>
> > >> > >> > > > > > > > > >>>>>>> -Artem
> > >> > >> > > > > > > > > >>>>>>>
> > >> > >> > > > > > > > > >>>>>>> On Tue, Nov 22, 2022 at 10:00 AM Justine
> > >> Olshan
> > >> > >> > > > > > > > > >>>>>>> <jo...@confluent.io.invalid> wrote:
> > >> > >> > > > > > > > > >>>>>>>
> > >> > >> > > > > > > > > >>>>>>>> Thanks for taking a look Matthias. I've
> > >> tried
> > >> > to
> > >> > >> > > answer
> > >> > >> > > > > your
> > >> > >> > > > > > > > > >>>>> questions
> > >> > >> > > > > > > > > >>>>>>>> below:
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>> 10)
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>> Right — so the hanging transaction only
> > >> occurs
> > >> > >> when
> > >> > >> > we
> > >> > >> > > > > have
> > >> > >> > > > > > a
> > >> > >> > > > > > > > late
> > >> > >> > > > > > > > > >>>>>>> message
> > >> > >> > > > > > > > > >>>>>>>> come in and the partition is never added
> > to
> > >> a
> > >> > >> > > > transaction
> > >> > >> > > > > > > again.
> > >> > >> > > > > > > > > If
> > >> > >> > > > > > > > > >>>>> we
> > >> > >> > > > > > > > > >>>>>>>> never add the partition to a
> transaction,
> > we
> > >> > will
> > >> > >> > > never
> > >> > >> > > > > > write
> > >> > >> > > > > > > a
> > >> > >> > > > > > > > > >>>>> marker
> > >> > >> > > > > > > > > >>>>>>> and
> > >> > >> > > > > > > > > >>>>>>>> never advance the LSO.
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>> If we do end up adding the partition to
> > the
> > >> > >> > > transaction
> > >> > >> > > > (I
> > >> > >> > > > > > > > suppose
> > >> > >> > > > > > > > > >>>>> this
> > >> > >> > > > > > > > > >>>>>>> can
> > >> > >> > > > > > > > > >>>>>>>> happen before or after the late message
> > >> comes
> > >> > in)
> > >> > >> > then
> > >> > >> > > > we
> > >> > >> > > > > > will
> > >> > >> > > > > > > > > >>>>> include
> > >> > >> > > > > > > > > >>>>>>> the
> > >> > >> > > > > > > > > >>>>>>>> late message in the next (incorrect)
> > >> > transaction.
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>> So perhaps it is clearer to make the
> > >> > distinction
> > >> > >> > > between
> > >> > >> > > > > > > > messages
> > >> > >> > > > > > > > > >>>>> that
> > >> > >> > > > > > > > > >>>>>>>> eventually get added to the transaction
> > (but
> > >> > the
> > >> > >> > wrong
> > >> > >> > > > > one)
> > >> > >> > > > > > or
> > >> > >> > > > > > > > > >>>>> messages
> > >> > >> > > > > > > > > >>>>>>>> that never get added and become hanging.
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>> 20)
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>> The client side change for 2 is removing
> > the
> > >> > >> > > > addPartitions
> > >> > >> > > > > > to
> > >> > >> > > > > > > > > >>>>>> transaction
> > >> > >> > > > > > > > > >>>>>>>> call. We don't need to make this from
> the
> > >> > >> producer
> > >> > >> > to
> > >> > >> > > > the
> > >> > >> > > > > > txn
> > >> > >> > > > > > > > > >>>>>>> coordinator,
> > >> > >> > > > > > > > > >>>>>>>> only server side.
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>> In my opinion, the issue with the
> > >> > >> addPartitionsToTxn
> > >> > >> > > > call
> > >> > >> > > > > > for
> > >> > >> > > > > > > > > older
> > >> > >> > > > > > > > > >>>>>>> clients
> > >> > >> > > > > > > > > >>>>>>>> is that we don't have the epoch bump, so
> > we
> > >> > don't
> > >> > >> > know
> > >> > >> > > > if
> > >> > >> > > > > > the
> > >> > >> > > > > > > > > >>> message
> > >> > >> > > > > > > > > >>>>>>>> belongs to the previous transaction or
> > this
> > >> > one.
> > >> > >> We
> > >> > >> > > need
> > >> > >> > > > > to
> > >> > >> > > > > > > > check
> > >> > >> > > > > > > > > if
> > >> > >> > > > > > > > > >>>>>> the
> > >> > >> > > > > > > > > >>>>>>>> partition has been added to this
> > >> transaction.
> > >> > Of
> > >> > >> > > course,
> > >> > >> > > > > > this
> > >> > >> > > > > > > > > means
> > >> > >> > > > > > > > > >>>>> we
> > >> > >> > > > > > > > > >>>>>>>> won't completely cover the case where we
> > >> have a
> > >> > >> > really
> > >> > >> > > > > late
> > >> > >> > > > > > > > > message
> > >> > >> > > > > > > > > >>>>> and
> > >> > >> > > > > > > > > >>>>>>> we
> > >> > >> > > > > > > > > >>>>>>>> have added the partition to the new
> > >> > transaction,
> > >> > >> but
> > >> > >> > > > > that's
> > >> > >> > > > > > > > > >>>>>> unfortunately
> > >> > >> > > > > > > > > >>>>>>>> something we will need the new clients
> to
> > >> > cover.
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>> 30)
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>> Transaction is ongoing = partition was
> > >> added to
> > >> > >> > > > > transaction
> > >> > >> > > > > > > via
> > >> > >> > > > > > > > > >>>>>>>> addPartitionsToTxn. We check this with
> the
> > >> > >> > > > > > > DescribeTransactions
> > >> > >> > > > > > > > > >>> call.
> > >> > >> > > > > > > > > >>>>>> Let
> > >> > >> > > > > > > > > >>>>>>>> me know if this wasn't sufficiently
> > >> explained
> > >> > >> here:
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>
> > >> > >> > > > > > > > > >>>>>>
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > >
> > >> > >> > > > > > > >
> > >> > >> > > > > > >
> > >> > >> > > > > >
> > >> > >> > > > >
> > >> > >> > > >
> > >> > >> > >
> > >> > >> >
> > >> > >>
> > >> >
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>> 40)
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>> The idea here is that if any messages
> > >> somehow
> > >> > >> come
> > >> > >> > in
> > >> > >> > > > > before
> > >> > >> > > > > > > we
> > >> > >> > > > > > > > > get
> > >> > >> > > > > > > > > >>>>> the
> > >> > >> > > > > > > > > >>>>>>> new
> > >> > >> > > > > > > > > >>>>>>>> epoch to the producer, they will be
> > fenced.
> > >> > >> However,
> > >> > >> > > if
> > >> > >> > > > we
> > >> > >> > > > > > > don't
> > >> > >> > > > > > > > > >>>>> think
> > >> > >> > > > > > > > > >>>>>>> this
> > >> > >> > > > > > > > > >>>>>>>> is necessary, it can be discussed
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>> 50)
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>> It should be synchronous because if we
> > have
> > >> an
> > >> > >> event
> > >> > >> > > > (ie,
> > >> > >> > > > > an
> > >> > >> > > > > > > > > error)
> > >> > >> > > > > > > > > >>>>>> that
> > >> > >> > > > > > > > > >>>>>>>> causes us to need to abort the
> > transaction,
> > >> we
> > >> > >> need
> > >> > >> > to
> > >> > >> > > > > know
> > >> > >> > > > > > > > which
> > >> > >> > > > > > > > > >>>>>>>> partitions to send transaction markers
> to.
> > >> We
> > >> > >> know
> > >> > >> > the
> > >> > >> > > > > > > > partitions
> > >> > >> > > > > > > > > >>>>>> because
> > >> > >> > > > > > > > > >>>>>>>> we added them to the coordinator via the
> > >> > >> > > > > addPartitionsToTxn
> > >> > >> > > > > > > > call.
> > >> > >> > > > > > > > > >>>>>>>> Previously we have had asynchronous
> calls
> > in
> > >> > the
> > >> > >> > past
> > >> > >> > > > (ie,
> > >> > >> > > > > > > > writing
> > >> > >> > > > > > > > > >>>>> the
> > >> > >> > > > > > > > > >>>>>>>> commit markers when the transaction is
> > >> > completed)
> > >> > >> > but
> > >> > >> > > > > often
> > >> > >> > > > > > > this
> > >> > >> > > > > > > > > >>> just
> > >> > >> > > > > > > > > >>>>>>>> causes confusion as we need to wait for
> > some
> > >> > >> > > operations
> > >> > >> > > > to
> > >> > >> > > > > > > > > complete.
> > >> > >> > > > > > > > > >>>>> In
> > >> > >> > > > > > > > > >>>>>>> the
> > >> > >> > > > > > > > > >>>>>>>> writing commit markers case, clients
> often
> > >> see
> > >> > >> > > > > > > > > >>>>> CONCURRENT_TRANSACTIONs
> > >> > >> > > > > > > > > >>>>>>>> error messages and that can be
> confusing.
> > >> For
> > >> > >> that
> > >> > >> > > > reason,
> > >> > >> > > > > > it
> > >> > >> > > > > > > > may
> > >> > >> > > > > > > > > be
> > >> > >> > > > > > > > > >>>>>>>> simpler to just have synchronous calls —
> > >> > >> especially
> > >> > >> > if
> > >> > >> > > > we
> > >> > >> > > > > > need
> > >> > >> > > > > > > > to
> > >> > >> > > > > > > > > >>>>> block
> > >> > >> > > > > > > > > >>>>>>> on
> > >> > >> > > > > > > > > >>>>>>>> some operation's completion anyway
> before
> > we
> > >> > can
> > >> > >> > start
> > >> > >> > > > the
> > >> > >> > > > > > > next
> > >> > >> > > > > > > > > >>>>>>>> transaction. And yes, I meant
> > coordinator. I
> > >> > will
> > >> > >> > fix
> > >> > >> > > > > that.
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>> 60)
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>> When we are checking if the transaction
> is
> > >> > >> ongoing,
> > >> > >> > we
> > >> > >> > > > > need
> > >> > >> > > > > > to
> > >> > >> > > > > > > > > make
> > >> > >> > > > > > > > > >>> a
> > >> > >> > > > > > > > > >>>>>>> round
> > >> > >> > > > > > > > > >>>>>>>> trip from the leader partition to the
> > >> > transaction
> > >> > >> > > > > > coordinator.
> > >> > >> > > > > > > > In
> > >> > >> > > > > > > > > >>> the
> > >> > >> > > > > > > > > >>>>>>> time
> > >> > >> > > > > > > > > >>>>>>>> we are waiting for this message to come
> > >> back,
> > >> > in
> > >> > >> > > theory
> > >> > >> > > > we
> > >> > >> > > > > > > could
> > >> > >> > > > > > > > > >>> have
> > >> > >> > > > > > > > > >>>>>>> sent
> > >> > >> > > > > > > > > >>>>>>>> a commit/abort call that would make the
> > >> > original
> > >> > >> > > result
> > >> > >> > > > of
> > >> > >> > > > > > the
> > >> > >> > > > > > > > > check
> > >> > >> > > > > > > > > >>>>>> out
> > >> > >> > > > > > > > > >>>>>>> of
> > >> > >> > > > > > > > > >>>>>>>> date. That is why we can check the
> leader
> > >> state
> > >> > >> > before
> > >> > >> > > > we
> > >> > >> > > > > > > write
> > >> > >> > > > > > > > to
> > >> > >> > > > > > > > > >>>>> the
> > >> > >> > > > > > > > > >>>>>>> log.
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>> I'm happy to update the KIP if some of
> > these
> > >> > >> things
> > >> > >> > > were
> > >> > >> > > > > not
> > >> > >> > > > > > > > > clear.
> > >> > >> > > > > > > > > >>>>>>>> Thanks,
> > >> > >> > > > > > > > > >>>>>>>> Justine
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>> On Mon, Nov 21, 2022 at 7:11 PM Matthias
> > J.
> > >> > Sax <
> > >> > >> > > > > > > > mjsax@apache.org
> > >> > >> > > > > > > > > >
> > >> > >> > > > > > > > > >>>>>>> wrote:
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>> Thanks for the KIP.
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>> Couple of clarification questions (I am
> > >> not a
> > >> > >> > broker
> > >> > >> > > > > expert
> > >> > >> > > > > > > do
> > >> > >> > > > > > > > > >>>>> maybe
> > >> > >> > > > > > > > > >>>>>>>>> some question are obvious for others,
> but
> > >> not
> > >> > >> for
> > >> > >> > me
> > >> > >> > > > with
> > >> > >> > > > > > my
> > >> > >> > > > > > > > lack
> > >> > >> > > > > > > > > >>>>> of
> > >> > >> > > > > > > > > >>>>>>>>> broker knowledge).
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>> (10)
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>> 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.
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>> What happens if the message come in
> > before
> > >> the
> > >> > >> next
> > >> > >> > > > > > > > > >>>>>> addPartitionsToTxn
> > >> > >> > > > > > > > > >>>>>>>>> request? It seems the broker hosting
> the
> > >> data
> > >> > >> > > > partitions
> > >> > >> > > > > > > won't
> > >> > >> > > > > > > > > know
> > >> > >> > > > > > > > > >>>>>>>>> anything about it and append it to the
> > >> > >> partition,
> > >> > >> > > too?
> > >> > >> > > > > What
> > >> > >> > > > > > > is
> > >> > >> > > > > > > > > the
> > >> > >> > > > > > > > > >>>>>>>>> difference between both cases?
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>> Also, it seems a TX would only hang, if
> > >> there
> > >> > >> is no
> > >> > >> > > > > > following
> > >> > >> > > > > > > > TX
> > >> > >> > > > > > > > > >>>>> that
> > >> > >> > > > > > > > > >>>>>>> is
> > >> > >> > > > > > > > > >>>>>>>>> either committer or aborted? Thus, for
> > the
> > >> > case
> > >> > >> > > above,
> > >> > >> > > > > the
> > >> > >> > > > > > TX
> > >> > >> > > > > > > > > might
> > >> > >> > > > > > > > > >>>>>>>>> actually not hang (of course, we might
> > get
> > >> an
> > >> > >> EOS
> > >> > >> > > > > violation
> > >> > >> > > > > > > if
> > >> > >> > > > > > > > > the
> > >> > >> > > > > > > > > >>>>>>> first
> > >> > >> > > > > > > > > >>>>>>>>> TX was aborted and the second
> committed,
> > or
> > >> > the
> > >> > >> > other
> > >> > >> > > > way
> > >> > >> > > > > > > > > around).
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>> (20)
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>> Of course, 1 and 2 require client-side
> > >> > >> changes, so
> > >> > >> > > for
> > >> > >> > > > > > older
> > >> > >> > > > > > > > > >>>>>> clients,
> > >> > >> > > > > > > > > >>>>>>>>> those approaches won’t apply.
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>> For (1) I understand why a client
> change
> > is
> > >> > >> > > necessary,
> > >> > >> > > > > but
> > >> > >> > > > > > > not
> > >> > >> > > > > > > > > sure
> > >> > >> > > > > > > > > >>>>>> why
> > >> > >> > > > > > > > > >>>>>>>>> we need a client change for (2). Can
> you
> > >> > >> elaborate?
> > >> > >> > > --
> > >> > >> > > > > > Later
> > >> > >> > > > > > > > you
> > >> > >> > > > > > > > > >>>>>>> explain
> > >> > >> > > > > > > > > >>>>>>>>> that we should send a
> > >> > >> DescribeTransactionRequest,
> > >> > >> > > but I
> > >> > >> > > > > am
> > >> > >> > > > > > > not
> > >> > >> > > > > > > > > sure
> > >> > >> > > > > > > > > >>>>>>> why?
> > >> > >> > > > > > > > > >>>>>>>>> Can't we not just do an implicit
> > >> > >> AddPartiitonToTx,
> > >> > >> > > too?
> > >> > >> > > > > If
> > >> > >> > > > > > > the
> > >> > >> > > > > > > > > old
> > >> > >> > > > > > > > > >>>>>>>>> producer correctly registered the
> > partition
> > >> > >> > already,
> > >> > >> > > > the
> > >> > >> > > > > > > > > >>>>>> TX-coordinator
> > >> > >> > > > > > > > > >>>>>>>>> can just ignore it as it's an
> idempotent
> > >> > >> operation?
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>> (30)
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>> To cover older clients, we will
> ensure a
> > >> > >> > transaction
> > >> > >> > > > is
> > >> > >> > > > > > > > ongoing
> > >> > >> > > > > > > > > >>>>>>> before
> > >> > >> > > > > > > > > >>>>>>>>> we write to a transaction
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>> Not sure what you mean by this? Can you
> > >> > >> elaborate?
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>> (40)
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>> [the TX-coordinator] will write the
> > >> prepare
> > >> > >> commit
> > >> > >> > > > > message
> > >> > >> > > > > > > > with
> > >> > >> > > > > > > > > a
> > >> > >> > > > > > > > > >>>>>>>> bumped
> > >> > >> > > > > > > > > >>>>>>>>> epoch and send WriteTxnMarkerRequests
> > with
> > >> the
> > >> > >> > bumped
> > >> > >> > > > > > epoch.
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>> Why do we use the bumped epoch for
> both?
> > It
> > >> > >> seems
> > >> > >> > > more
> > >> > >> > > > > > > > intuitive
> > >> > >> > > > > > > > > to
> > >> > >> > > > > > > > > >>>>>> use
> > >> > >> > > > > > > > > >>>>>>>>> the current epoch, and only return the
> > >> bumped
> > >> > >> epoch
> > >> > >> > > to
> > >> > >> > > > > the
> > >> > >> > > > > > > > > >>>>> producer?
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>> (50) "Implicit
> AddPartitionToTransaction"
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>> Why does the implicitly sent request
> need
> > >> to
> > >> > be
> > >> > >> > > > > > synchronous?
> > >> > >> > > > > > > > The
> > >> > >> > > > > > > > > >>>>> KIP
> > >> > >> > > > > > > > > >>>>>>>>> also says
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>> in case we need to abort and need to
> > know
> > >> > which
> > >> > >> > > > > partitions
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>> What do you mean by this?
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>> we don’t want to write to it before we
> > >> store
> > >> > in
> > >> > >> > the
> > >> > >> > > > > > > > transaction
> > >> > >> > > > > > > > > >>>>>>> manager
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>> Do you mean TX-coordinator instead of
> > >> > "manager"?
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>> (60)
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>> For older clients and ensuring that the
> > TX
> > >> is
> > >> > >> > > ongoing,
> > >> > >> > > > > you
> > >> > >> > > > > > > > > >>>>> describe a
> > >> > >> > > > > > > > > >>>>>>>>> race condition. I am not sure if I can
> > >> follow
> > >> > >> here.
> > >> > >> > > Can
> > >> > >> > > > > you
> > >> > >> > > > > > > > > >>>>>> elaborate?
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>> -Matthias
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>> On 11/18/22 1:21 PM, Justine Olshan
> > wrote:
> > >> > >> > > > > > > > > >>>>>>>>>> Hey all!
> > >> > >> > > > > > > > > >>>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>> I'd like to start a discussion on my
> > >> proposal
> > >> > >> to
> > >> > >> > add
> > >> > >> > > > > some
> > >> > >> > > > > > > > > >>>>>> server-side
> > >> > >> > > > > > > > > >>>>>>>>>> checks on transactions to avoid
> hanging
> > >> > >> > > transactions.
> > >> > >> > > > I
> > >> > >> > > > > > know
> > >> > >> > > > > > > > > this
> > >> > >> > > > > > > > > >>>>>> has
> > >> > >> > > > > > > > > >>>>>>>>> been
> > >> > >> > > > > > > > > >>>>>>>>>> an issue for some time, so I really
> hope
> > >> this
> > >> > >> KIP
> > >> > >> > > will
> > >> > >> > > > > be
> > >> > >> > > > > > > > > helpful
> > >> > >> > > > > > > > > >>>>>> for
> > >> > >> > > > > > > > > >>>>>>>>> many
> > >> > >> > > > > > > > > >>>>>>>>>> users of EOS.
> > >> > >> > > > > > > > > >>>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>> The KIP includes changes that will be
> > >> > >> compatible
> > >> > >> > > with
> > >> > >> > > > > old
> > >> > >> > > > > > > > > clients
> > >> > >> > > > > > > > > >>>>>> and
> > >> > >> > > > > > > > > >>>>>>>>>> changes to improve performance and
> > >> > correctness
> > >> > >> on
> > >> > >> > > new
> > >> > >> > > > > > > clients.
> > >> > >> > > > > > > > > >>>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>> Please take a look and leave any
> > comments
> > >> you
> > >> > >> may
> > >> > >> > > > have!
> > >> > >> > > > > > > > > >>>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>> KIP:
> > >> > >> > > > > > > > > >>>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>
> > >> > >> > > > > > > > > >>>>>>
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > >
> > >> > >> > > > > > > >
> > >> > >> > > > > > >
> > >> > >> > > > > >
> > >> > >> > > > >
> > >> > >> > > >
> > >> > >> > >
> > >> > >> >
> > >> > >>
> > >> >
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > >> > >> > > > > > > > > >>>>>>>>>> JIRA:
> > >> > >> > > > https://issues.apache.org/jira/browse/KAFKA-14402
> > >> > >> > > > > > > > > >>>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>> Thanks!
> > >> > >> > > > > > > > > >>>>>>>>>> Justine
> > >> > >> > > > > > > > > >>>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>>
> > >> > >> > > > > > > > > >>>>>>>
> > >> > >> > > > > > > > > >>>>>>
> > >> > >> > > > > > > > > >>>>>
> > >> > >> > > > > > > > > >>>>
> > >> > >> > > > > > > > > >>>
> > >> > >> > > > > > > > > >>
> > >> > >> > > > > > > > > >
> > >> > >> > > > > > > > >
> > >> > >> > > > > > > >
> > >> > >> > > > > > >
> > >> > >> > > > > >
> > >> > >> > > > >
> > >> > >> > > >
> > >> > >> > >
> > >> > >> >
> > >> > >>
> > >> > >
> > >> >
> > >>
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Artem Livshits <al...@confluent.io.INVALID>.
There are some workflows in the client that are implied by protocol
changes, e.g.:

- for new clients, epoch changes with every transaction and can overflow,
in old clients this condition was handled transparently, because epoch was
bumped in InitProducerId and it would return a new producer id if epoch
overflows, the new clients would need to implement some workflow to refresh
producer id
- how to handle fenced producers, for new clients epoch changes with every
transaction, so in presence of failures during commits / aborts, the
producer could get easily fenced, old clients would pretty much would get
fenced when a new incarnation of the producer was initialized with
InitProducerId so it's ok to treat as a fatal error, the new clients would
need to implement some workflow to handle that error, otherwise they could
get fenced by themselves
- in particular (as a subset of the previous issue), what would the client
do if it got a timeout during commit?  commit could've succeeded or failed

Not sure if this has to be defined in the KIP as implementing those
probably wouldn't require protocol changes, but we have multiple
implementations of Kafka clients, so probably would be good to have some
client implementation guidance.  Could also be done as a separate doc.

-Artem

On Mon, Jan 9, 2023 at 3:38 PM Justine Olshan <jo...@confluent.io.invalid>
wrote:

> Hey all, I've updated the KIP to incorporate Jason's suggestions.
>
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
>
>
> 1. Use AddPartitionsToTxn + verify flag to check on old clients
> 2. Updated AddPartitionsToTxn API to support transaction batching
> 3. Mention IBP bump
> 4. Mention auth change on new AddPartitionsToTxn version.
>
> I'm planning on opening a vote soon.
> Thanks,
> Justine
>
> On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <jo...@confluent.io>
> wrote:
>
> > Thanks Jason. Those changes make sense to me. I will update the KIP.
> >
> >
> >
> > On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson
> <ja...@confluent.io.invalid>
> > wrote:
> >
> >> Hey Justine,
> >>
> >> > I was wondering about compatibility here. When we send requests
> >> between brokers, we want to ensure that the receiving broker understands
> >> the request (specifically the new fields). Typically this is done via
> >> IBP/metadata version.
> >> I'm trying to think if there is a way around it but I'm not sure there
> is.
> >>
> >> Yes. I think we would gate usage of this behind an IBP bump. Does that
> >> seem
> >> reasonable?
> >>
> >> > As for the improvements -- can you clarify how the multiple
> >> transactional
> >> IDs would help here? Were you thinking of a case where we wait/batch
> >> multiple produce requests together? My understanding for now was 1
> >> transactional ID and one validation per 1 produce request.
> >>
> >> Each call to `AddPartitionsToTxn` is essentially a write to the
> >> transaction
> >> log and must block on replication. The more we can fit into a single
> >> request, the more writes we can do in parallel. The alternative is to
> make
> >> use of more connections, but usually we prefer batching since the
> network
> >> stack is not really optimized for high connection/request loads.
> >>
> >> > Finally with respect to the authorizations, I think it makes sense to
> >> skip
> >> topic authorizations, but I'm a bit confused by the "leader ID" field.
> >> Wouldn't we just want to flag the request as from a broker (does it
> matter
> >> which one?).
> >>
> >> We could also make it version-based. For the next version, we could
> >> require
> >> CLUSTER auth. So clients would not be able to use the API anymore, which
> >> is
> >> probably what we want.
> >>
> >> -Jason
> >>
> >> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> >> <jo...@confluent.io.invalid>
> >> wrote:
> >>
> >> > As a follow up, I was just thinking about the batching a bit more.
> >> > I suppose if we have one request in flight and we queue up the other
> >> > produce requests in some sort of purgatory, we could send information
> >> out
> >> > for all of them rather than one by one. So that would be a benefit of
> >> > batching partitions to add per transaction.
> >> >
> >> > I'll need to think a bit more on the design of this part of the KIP,
> and
> >> > will update the KIP in the next few days.
> >> >
> >> > Thanks,
> >> > Justine
> >> >
> >> > On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan <jo...@confluent.io>
> >> > wrote:
> >> >
> >> > > Hey Jason -- thanks for the input -- I was just digging a bit deeper
> >> into
> >> > > the design + implementation of the validation calls here and what
> you
> >> say
> >> > > makes sense.
> >> > >
> >> > > I was wondering about compatibility here. When we send requests
> >> > > between brokers, we want to ensure that the receiving broker
> >> understands
> >> > > the request (specifically the new fields). Typically this is done
> via
> >> > > IBP/metadata version.
> >> > > I'm trying to think if there is a way around it but I'm not sure
> there
> >> > is.
> >> > >
> >> > > As for the improvements -- can you clarify how the multiple
> >> transactional
> >> > > IDs would help here? Were you thinking of a case where we wait/batch
> >> > > multiple produce requests together? My understanding for now was 1
> >> > > transactional ID and one validation per 1 produce request.
> >> > >
> >> > > Finally with respect to the authorizations, I think it makes sense
> to
> >> > skip
> >> > > topic authorizations, but I'm a bit confused by the "leader ID"
> field.
> >> > > Wouldn't we just want to flag the request as from a broker (does it
> >> > matter
> >> > > which one?).
> >> > >
> >> > > I think I want to adopt these suggestions, just had a few questions
> on
> >> > the
> >> > > details.
> >> > >
> >> > > Thanks,
> >> > > Justine
> >> > >
> >> > > On Thu, Jan 5, 2023 at 5:05 PM Jason Gustafson
> >> > <ja...@confluent.io.invalid>
> >> > > wrote:
> >> > >
> >> > >> Hi Justine,
> >> > >>
> >> > >> Thanks for the proposal.
> >> > >>
> >> > >> I was thinking about the implementation a little bit. In the
> current
> >> > >> proposal, the behavior depends on whether we have an old or new
> >> client.
> >> > >> For
> >> > >> old clients, we send `DescribeTransactions` and verify the result
> and
> >> > for
> >> > >> new clients, we send `AddPartitionsToTxn`. We might be able to
> >> simplify
> >> > >> the
> >> > >> implementation if we can use the same request type. For example,
> >> what if
> >> > >> we
> >> > >> bump the protocol version for `AddPartitionsToTxn` and add a
> >> > >> `validateOnly`
> >> > >> flag? For older versions, we can set `validateOnly=true` so that
> the
> >> > >> request only returns successfully if the partition had already been
> >> > added.
> >> > >> For new versions, we can set `validateOnly=false` and the partition
> >> will
> >> > >> be
> >> > >> added to the transaction. The other slightly annoying thing that
> this
> >> > >> would
> >> > >> get around is the need to collect the transaction state for all
> >> > partitions
> >> > >> even when we only care about a subset.
> >> > >>
> >> > >> Some additional improvements to consider:
> >> > >>
> >> > >> - We can give `AddPartitionsToTxn` better batch support for
> >> inter-broker
> >> > >> usage. Currently we only allow one `TransactionalId` to be
> specified,
> >> > but
> >> > >> the broker may get some benefit being able to batch across multiple
> >> > >> transactions.
> >> > >> - Another small improvement is skipping topic authorization checks
> >> for
> >> > >> `AddPartitionsToTxn` when the request is from a broker. Perhaps we
> >> can
> >> > add
> >> > >> a field for the `LeaderId` or something like that and require
> CLUSTER
> >> > >> permission when set.
> >> > >>
> >> > >> Best,
> >> > >> Jason
> >> > >>
> >> > >>
> >> > >>
> >> > >> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao <ju...@confluent.io.invalid>
> >> > wrote:
> >> > >>
> >> > >> > Hi, Justine,
> >> > >> >
> >> > >> > Thanks for the explanation. It makes sense to me now.
> >> > >> >
> >> > >> > Jun
> >> > >> >
> >> > >> > On Mon, Dec 19, 2022 at 1:42 PM Justine Olshan
> >> > >> > <jo...@confluent.io.invalid>
> >> > >> > wrote:
> >> > >> >
> >> > >> > > Hi Jun,
> >> > >> > >
> >> > >> > > My understanding of the mechanism is that when we get to the
> last
> >> > >> epoch,
> >> > >> > we
> >> > >> > > increment to the fencing/last epoch and if any further requests
> >> come
> >> > >> in
> >> > >> > for
> >> > >> > > this producer ID they are fenced. Then the producer gets a new
> ID
> >> > and
> >> > >> > > restarts with epoch/sequence 0. The fenced epoch sticks around
> >> for
> >> > the
> >> > >> > > duration of producer.id.expiration.ms and blocks any late
> >> messages
> >> > >> > there.
> >> > >> > > The new ID will get to take advantage of the improved semantics
> >> > around
> >> > >> > > non-zero start sequences. So I think we are covered.
> >> > >> > >
> >> > >> > > The only potential issue is overloading the cache, but
> hopefully
> >> the
> >> > >> > > improvements (lowered producer.id.expiration.ms) will help
> with
> >> > that.
> >> > >> > Let
> >> > >> > > me know if you still have concerns.
> >> > >> > >
> >> > >> > > Thanks,
> >> > >> > > Justine
> >> > >> > >
> >> > >> > > On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
> >> <ju...@confluent.io.invalid>
> >> > >> > wrote:
> >> > >> > >
> >> > >> > > > Hi, Justine,
> >> > >> > > >
> >> > >> > > > Thanks for the explanation.
> >> > >> > > >
> >> > >> > > > 70. The proposed fencing logic doesn't apply when pid
> changes,
> >> is
> >> > >> that
> >> > >> > > > right? If so, I am not sure how complete we are addressing
> this
> >> > >> issue
> >> > >> > if
> >> > >> > > > the pid changes more frequently.
> >> > >> > > >
> >> > >> > > > Thanks,
> >> > >> > > >
> >> > >> > > > Jun
> >> > >> > > >
> >> > >> > > >
> >> > >> > > >
> >> > >> > > > On Fri, Dec 16, 2022 at 9:16 AM Justine Olshan
> >> > >> > > > <jo...@confluent.io.invalid>
> >> > >> > > > wrote:
> >> > >> > > >
> >> > >> > > > > Hi Jun,
> >> > >> > > > >
> >> > >> > > > > Thanks for replying!
> >> > >> > > > >
> >> > >> > > > > 70.We already do the overflow mechanism, so my change would
> >> just
> >> > >> make
> >> > >> > > it
> >> > >> > > > > happen more often.
> >> > >> > > > > I was also not suggesting a new field in the log, but in
> the
> >> > >> > response,
> >> > >> > > > > which would be gated by the client version. Sorry if
> >> something
> >> > >> there
> >> > >> > is
> >> > >> > > > > unclear. I think we are starting to diverge.
> >> > >> > > > > The goal of this KIP is to not change to the marker format
> at
> >> > all.
> >> > >> > > > >
> >> > >> > > > > 71. Yes, I guess I was going under the assumption that the
> >> log
> >> > >> would
> >> > >> > > just
> >> > >> > > > > look at its last epoch and treat it as the current epoch. I
> >> > >> suppose
> >> > >> > we
> >> > >> > > > can
> >> > >> > > > > have some special logic that if the last epoch was on a
> >> marker
> >> > we
> >> > >> > > > actually
> >> > >> > > > > expect the next epoch or something like that. We just need
> to
> >> > >> > > distinguish
> >> > >> > > > > based on whether we had a commit/abort marker.
> >> > >> > > > >
> >> > >> > > > > 72.
> >> > >> > > > > > if the producer epoch hasn't been bumped on the
> >> > >> > > > > broker, it seems that the stucked message will fail the
> >> sequence
> >> > >> > > > validation
> >> > >> > > > > and will be ignored. If the producer epoch has been bumped,
> >> we
> >> > >> ignore
> >> > >> > > the
> >> > >> > > > > sequence check and the stuck message could be appended to
> the
> >> > log.
> >> > >> > So,
> >> > >> > > is
> >> > >> > > > > the latter case that we want to guard?
> >> > >> > > > >
> >> > >> > > > > I'm not sure I follow that "the message will fail the
> >> sequence
> >> > >> > > > validation".
> >> > >> > > > > In some of these cases, we had an abort marker (due to an
> >> error)
> >> > >> and
> >> > >> > > then
> >> > >> > > > > the late message comes in with the correct sequence number.
> >> This
> >> > >> is a
> >> > >> > > > case
> >> > >> > > > > covered by the KIP.
> >> > >> > > > > The latter case is actually not something we've considered
> >> > here. I
> >> > >> > > think
> >> > >> > > > > generally when we bump the epoch, we are accepting that the
> >> > >> sequence
> >> > >> > > does
> >> > >> > > > > not need to be checked anymore. My understanding is also
> >> that we
> >> > >> > don't
> >> > >> > > > > typically bump epoch mid transaction (based on a quick look
> >> at
> >> > the
> >> > >> > > code)
> >> > >> > > > > but let me know if that is the case.
> >> > >> > > > >
> >> > >> > > > > Thanks,
> >> > >> > > > > Justine
> >> > >> > > > >
> >> > >> > > > > On Thu, Dec 15, 2022 at 12:23 PM Jun Rao
> >> > <jun@confluent.io.invalid
> >> > >> >
> >> > >> > > > wrote:
> >> > >> > > > >
> >> > >> > > > > > Hi, Justine,
> >> > >> > > > > >
> >> > >> > > > > > Thanks for the reply.
> >> > >> > > > > >
> >> > >> > > > > > 70. Assigning a new pid on int overflow seems a bit
> hacky.
> >> If
> >> > we
> >> > >> > > need a
> >> > >> > > > > txn
> >> > >> > > > > > level id, it will be better to model this explicitly.
> >> Adding a
> >> > >> new
> >> > >> > > > field
> >> > >> > > > > > would require a bit more work since it requires a new txn
> >> > marker
> >> > >> > > format
> >> > >> > > > > in
> >> > >> > > > > > the log. So, we probably need to guard it with an IBP or
> >> > >> metadata
> >> > >> > > > version
> >> > >> > > > > > and document the impact on downgrade once the new format
> is
> >> > >> written
> >> > >> > > to
> >> > >> > > > > the
> >> > >> > > > > > log.
> >> > >> > > > > >
> >> > >> > > > > > 71. Hmm, once the marker is written, the partition will
> >> expect
> >> > >> the
> >> > >> > > next
> >> > >> > > > > > append to be on the next epoch. Does that cover the case
> >> you
> >> > >> > > mentioned?
> >> > >> > > > > >
> >> > >> > > > > > 72. Also, just to be clear on the stucked message issue
> >> > >> described
> >> > >> > in
> >> > >> > > > the
> >> > >> > > > > > motivation. With EoS, we also validate the sequence id
> for
> >> > >> > > idempotency.
> >> > >> > > > > So,
> >> > >> > > > > > with the current logic, if the producer epoch hasn't been
> >> > >> bumped on
> >> > >> > > the
> >> > >> > > > > > broker, it seems that the stucked message will fail the
> >> > sequence
> >> > >> > > > > validation
> >> > >> > > > > > and will be ignored. If the producer epoch has been
> >> bumped, we
> >> > >> > ignore
> >> > >> > > > the
> >> > >> > > > > > sequence check and the stuck message could be appended to
> >> the
> >> > >> log.
> >> > >> > > So,
> >> > >> > > > is
> >> > >> > > > > > the latter case that we want to guard?
> >> > >> > > > > >
> >> > >> > > > > > Thanks,
> >> > >> > > > > >
> >> > >> > > > > > Jun
> >> > >> > > > > >
> >> > >> > > > > > On Wed, Dec 14, 2022 at 10:44 AM Justine Olshan
> >> > >> > > > > > <jo...@confluent.io.invalid> wrote:
> >> > >> > > > > >
> >> > >> > > > > > > Matthias — thanks again for taking time to look a this.
> >> You
> >> > >> said:
> >> > >> > > > > > >
> >> > >> > > > > > > > My proposal was only focusing to avoid dangling
> >> > >> > > > > > >
> >> > >> > > > > > > transactions if records are added without registered
> >> > >> partition.
> >> > >> > --
> >> > >> > > > > Maybe
> >> > >> > > > > > >
> >> > >> > > > > > > you can add a few more details to the KIP about this
> >> > scenario
> >> > >> for
> >> > >> > > > > better
> >> > >> > > > > > >
> >> > >> > > > > > > documentation purpose?
> >> > >> > > > > > >
> >> > >> > > > > > >
> >> > >> > > > > > > I'm not sure I understand what you mean here. The
> >> motivation
> >> > >> > > section
> >> > >> > > > > > > describes two scenarios about how the record can be
> added
> >> > >> > without a
> >> > >> > > > > > > registered partition:
> >> > >> > > > > > >
> >> > >> > > > > > >
> >> > >> > > > > > > > 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.
> >> > >> > > > > > >
> >> > >> > > > > > >
> >> > >> > > > > > > > Another way hanging transactions can occur is that a
> >> > client
> >> > >> is
> >> > >> > > > buggy
> >> > >> > > > > > and
> >> > >> > > > > > > may somehow try to write to a partition before it adds
> >> the
> >> > >> > > partition
> >> > >> > > > to
> >> > >> > > > > > the
> >> > >> > > > > > > transaction.
> >> > >> > > > > > >
> >> > >> > > > > > >
> >> > >> > > > > > >
> >> > >> > > > > > > For the first example of this would it be helpful to
> say
> >> > that
> >> > >> > this
> >> > >> > > > > > message
> >> > >> > > > > > > comes in after the abort, but before the partition is
> >> added
> >> > to
> >> > >> > the
> >> > >> > > > next
> >> > >> > > > > > > transaction so it becomes "hanging." Perhaps the next
> >> > sentence
> >> > >> > > > > describing
> >> > >> > > > > > > the message becoming part of the next transaction (a
> >> > different
> >> > >> > > case)
> >> > >> > > > > was
> >> > >> > > > > > > not properly differentiated.
> >> > >> > > > > > >
> >> > >> > > > > > >
> >> > >> > > > > > >
> >> > >> > > > > > > Jun — thanks for reading the KIP.
> >> > >> > > > > > >
> >> > >> > > > > > > 70. The int typing was a concern. Currently we have a
> >> > >> mechanism
> >> > >> > in
> >> > >> > > > > place
> >> > >> > > > > > to
> >> > >> > > > > > > fence the final epoch when the epoch is about to
> overflow
> >> > and
> >> > >> > > assign
> >> > >> > > > a
> >> > >> > > > > > new
> >> > >> > > > > > > producer ID with epoch 0. Of course, this is a bit
> tricky
> >> > >> when it
> >> > >> > > > comes
> >> > >> > > > > > to
> >> > >> > > > > > > the response back to the client.
> >> > >> > > > > > > Making this a long could be another option, but I
> wonder
> >> are
> >> > >> > there
> >> > >> > > > any
> >> > >> > > > > > > implications on changing this field if the epoch is
> >> > persisted
> >> > >> to
> >> > >> > > > disk?
> >> > >> > > > > > I'd
> >> > >> > > > > > > need to check the usages.
> >> > >> > > > > > >
> >> > >> > > > > > > 71.This was something Matthias asked about as well. I
> was
> >> > >> > > > considering a
> >> > >> > > > > > > possible edge case where a produce request from a new
> >> > >> transaction
> >> > >> > > > > somehow
> >> > >> > > > > > > gets sent right after the marker is written, but before
> >> the
> >> > >> > > producer
> >> > >> > > > is
> >> > >> > > > > > > alerted of the newly bumped epoch. In this case, we may
> >> > >> include
> >> > >> > > this
> >> > >> > > > > > record
> >> > >> > > > > > > when we don't want to. I suppose we could try to do
> >> > something
> >> > >> > > client
> >> > >> > > > > side
> >> > >> > > > > > > to bump the epoch after sending an endTxn as well in
> this
> >> > >> > scenario
> >> > >> > > —
> >> > >> > > > > but
> >> > >> > > > > > I
> >> > >> > > > > > > wonder how it would work when the server is aborting
> >> based
> >> > on
> >> > >> a
> >> > >> > > > > > server-side
> >> > >> > > > > > > error. I could also be missing something and this
> >> scenario
> >> > is
> >> > >> > > > actually
> >> > >> > > > > > not
> >> > >> > > > > > > possible.
> >> > >> > > > > > >
> >> > >> > > > > > > Thanks again to everyone reading and commenting. Let me
> >> know
> >> > >> > about
> >> > >> > > > any
> >> > >> > > > > > > further questions or comments.
> >> > >> > > > > > >
> >> > >> > > > > > > Justine
> >> > >> > > > > > >
> >> > >> > > > > > > On Wed, Dec 14, 2022 at 9:41 AM Jun Rao
> >> > >> <jun@confluent.io.invalid
> >> > >> > >
> >> > >> > > > > > wrote:
> >> > >> > > > > > >
> >> > >> > > > > > > > Hi, Justine,
> >> > >> > > > > > > >
> >> > >> > > > > > > > Thanks for the KIP. A couple of comments.
> >> > >> > > > > > > >
> >> > >> > > > > > > > 70. Currently, the producer epoch is an int. I am not
> >> sure
> >> > >> if
> >> > >> > > it's
> >> > >> > > > > > enough
> >> > >> > > > > > > > to accommodate all transactions in the lifetime of a
> >> > >> producer.
> >> > >> > > > Should
> >> > >> > > > > > we
> >> > >> > > > > > > > change that to a long or add a new long field like
> >> txnId?
> >> > >> > > > > > > >
> >> > >> > > > > > > > 71. "it will write the prepare commit message with a
> >> > bumped
> >> > >> > epoch
> >> > >> > > > and
> >> > >> > > > > > > send
> >> > >> > > > > > > > WriteTxnMarkerRequests with the bumped epoch." Hmm,
> the
> >> > >> epoch
> >> > >> > is
> >> > >> > > > > > > associated
> >> > >> > > > > > > > with the current txn right? So, it seems weird to
> >> write a
> >> > >> > commit
> >> > >> > > > > > message
> >> > >> > > > > > > > with a bumped epoch. Should we only bump up the epoch
> >> in
> >> > >> > > > > EndTxnResponse
> >> > >> > > > > > > and
> >> > >> > > > > > > > rename the field to sth like nextProducerEpoch?
> >> > >> > > > > > > >
> >> > >> > > > > > > > Thanks,
> >> > >> > > > > > > >
> >> > >> > > > > > > > Jun
> >> > >> > > > > > > >
> >> > >> > > > > > > >
> >> > >> > > > > > > >
> >> > >> > > > > > > > On Mon, Dec 12, 2022 at 8:54 PM Matthias J. Sax <
> >> > >> > > mjsax@apache.org>
> >> > >> > > > > > > wrote:
> >> > >> > > > > > > >
> >> > >> > > > > > > > > Thanks for the background.
> >> > >> > > > > > > > >
> >> > >> > > > > > > > > 20/30: SGTM. My proposal was only focusing to avoid
> >> > >> dangling
> >> > >> > > > > > > > > transactions if records are added without
> registered
> >> > >> > partition.
> >> > >> > > > --
> >> > >> > > > > > > Maybe
> >> > >> > > > > > > > > you can add a few more details to the KIP about
> this
> >> > >> scenario
> >> > >> > > for
> >> > >> > > > > > > better
> >> > >> > > > > > > > > documentation purpose?
> >> > >> > > > > > > > >
> >> > >> > > > > > > > > 40: I think you hit a fair point about race
> >> conditions
> >> > or
> >> > >> > > client
> >> > >> > > > > bugs
> >> > >> > > > > > > > > (incorrectly not bumping the epoch). The
> >> > >> complexity/confusion
> >> > >> > > for
> >> > >> > > > > > using
> >> > >> > > > > > > > > the bumped epoch I see, is mainly for internal
> >> > debugging,
> >> > >> ie,
> >> > >> > > > > > > inspecting
> >> > >> > > > > > > > > log segment dumps -- it seems harder to reason
> about
> >> the
> >> > >> > system
> >> > >> > > > for
> >> > >> > > > > > us
> >> > >> > > > > > > > > humans. But if we get better guarantees, it would
> be
> >> > >> worth to
> >> > >> > > use
> >> > >> > > > > the
> >> > >> > > > > > > > > bumped epoch.
> >> > >> > > > > > > > >
> >> > >> > > > > > > > > 60: as I mentioned already, I don't know the broker
> >> > >> internals
> >> > >> > > to
> >> > >> > > > > > > provide
> >> > >> > > > > > > > > more input. So if nobody else chimes in, we should
> >> just
> >> > >> move
> >> > >> > > > > forward
> >> > >> > > > > > > > > with your proposal.
> >> > >> > > > > > > > >
> >> > >> > > > > > > > >
> >> > >> > > > > > > > > -Matthias
> >> > >> > > > > > > > >
> >> > >> > > > > > > > >
> >> > >> > > > > > > > > On 12/6/22 4:22 PM, Justine Olshan wrote:
> >> > >> > > > > > > > > > Hi all,
> >> > >> > > > > > > > > > After Artem's questions about error behavior,
> I've
> >> > >> > > re-evaluated
> >> > >> > > > > the
> >> > >> > > > > > > > > > unknown producer ID exception and had some
> >> discussions
> >> > >> > > offline.
> >> > >> > > > > > > > > >
> >> > >> > > > > > > > > > I think generally it makes sense to simplify
> error
> >> > >> handling
> >> > >> > > in
> >> > >> > > > > > cases
> >> > >> > > > > > > > like
> >> > >> > > > > > > > > > this and the UNKNOWN_PRODUCER_ID error has a
> pretty
> >> > long
> >> > >> > and
> >> > >> > > > > > > > complicated
> >> > >> > > > > > > > > > history. Because of this, I propose adding a new
> >> error
> >> > >> code
> >> > >> > > > > > > > > ABORTABLE_ERROR
> >> > >> > > > > > > > > > that when encountered by new clients (gated by
> the
> >> > >> produce
> >> > >> > > > > request
> >> > >> > > > > > > > > version)
> >> > >> > > > > > > > > > will simply abort the transaction. This allows
> the
> >> > >> server
> >> > >> > to
> >> > >> > > > have
> >> > >> > > > > > > some
> >> > >> > > > > > > > > say
> >> > >> > > > > > > > > > in whether the client aborts and makes handling
> >> much
> >> > >> > simpler.
> >> > >> > > > In
> >> > >> > > > > > the
> >> > >> > > > > > > > > > future, we can also use this error in other
> >> situations
> >> > >> > where
> >> > >> > > we
> >> > >> > > > > > want
> >> > >> > > > > > > to
> >> > >> > > > > > > > > > abort the transactions. We can even use on other
> >> apis.
> >> > >> > > > > > > > > >
> >> > >> > > > > > > > > > I've added this to the KIP. Let me know if there
> >> are
> >> > any
> >> > >> > > > > questions
> >> > >> > > > > > or
> >> > >> > > > > > > > > > issues.
> >> > >> > > > > > > > > >
> >> > >> > > > > > > > > > Justine
> >> > >> > > > > > > > > >
> >> > >> > > > > > > > > > On Fri, Dec 2, 2022 at 10:22 AM Justine Olshan <
> >> > >> > > > > > jolshan@confluent.io
> >> > >> > > > > > > >
> >> > >> > > > > > > > > wrote:
> >> > >> > > > > > > > > >
> >> > >> > > > > > > > > >> Hey Matthias,
> >> > >> > > > > > > > > >>
> >> > >> > > > > > > > > >>
> >> > >> > > > > > > > > >> 20/30 — Maybe I also didn't express myself
> >> clearly.
> >> > For
> >> > >> > > older
> >> > >> > > > > > > clients
> >> > >> > > > > > > > we
> >> > >> > > > > > > > > >> don't have a way to distinguish between a
> previous
> >> > and
> >> > >> the
> >> > >> > > > > current
> >> > >> > > > > > > > > >> transaction since we don't have the epoch bump.
> >> This
> >> > >> means
> >> > >> > > > that
> >> > >> > > > > a
> >> > >> > > > > > > late
> >> > >> > > > > > > > > >> message from the previous transaction may be
> >> added to
> >> > >> the
> >> > >> > > new
> >> > >> > > > > one.
> >> > >> > > > > > > > With
> >> > >> > > > > > > > > >> older clients — we can't guarantee this won't
> >> happen
> >> > >> if we
> >> > >> > > > > already
> >> > >> > > > > > > > sent
> >> > >> > > > > > > > > the
> >> > >> > > > > > > > > >> addPartitionsToTxn call (why we make changes for
> >> the
> >> > >> newer
> >> > >> > > > > client)
> >> > >> > > > > > > but
> >> > >> > > > > > > > > we
> >> > >> > > > > > > > > >> can at least gate some by ensuring that the
> >> partition
> >> > >> has
> >> > >> > > been
> >> > >> > > > > > added
> >> > >> > > > > > > > to
> >> > >> > > > > > > > > the
> >> > >> > > > > > > > > >> transaction. The rationale here is that there
> are
> >> > >> likely
> >> > >> > > LESS
> >> > >> > > > > late
> >> > >> > > > > > > > > arrivals
> >> > >> > > > > > > > > >> as time goes on, so hopefully most late arrivals
> >> will
> >> > >> come
> >> > >> > > in
> >> > >> > > > > > BEFORE
> >> > >> > > > > > > > the
> >> > >> > > > > > > > > >> addPartitionsToTxn call. Those that arrive
> before
> >> > will
> >> > >> be
> >> > >> > > > > properly
> >> > >> > > > > > > > gated
> >> > >> > > > > > > > > >> with the describeTransactions approach.
> >> > >> > > > > > > > > >>
> >> > >> > > > > > > > > >> If we take the approach you suggested, ANY late
> >> > arrival
> >> > >> > > from a
> >> > >> > > > > > > > previous
> >> > >> > > > > > > > > >> transaction will be added. And we don't want
> >> that. I
> >> > >> also
> >> > >> > > > don't
> >> > >> > > > > > see
> >> > >> > > > > > > > any
> >> > >> > > > > > > > > >> benefit in sending addPartitionsToTxn over the
> >> > >> > describeTxns
> >> > >> > > > > call.
> >> > >> > > > > > > They
> >> > >> > > > > > > > > will
> >> > >> > > > > > > > > >> both be one extra RPC to the Txn coordinator.
> >> > >> > > > > > > > > >>
> >> > >> > > > > > > > > >>
> >> > >> > > > > > > > > >> To be clear — newer clients will use
> >> > addPartitionsToTxn
> >> > >> > > > instead
> >> > >> > > > > of
> >> > >> > > > > > > the
> >> > >> > > > > > > > > >> DescribeTxns.
> >> > >> > > > > > > > > >>
> >> > >> > > > > > > > > >>
> >> > >> > > > > > > > > >> 40)
> >> > >> > > > > > > > > >> My concern is that if we have some delay in the
> >> > client
> >> > >> to
> >> > >> > > bump
> >> > >> > > > > the
> >> > >> > > > > > > > > epoch,
> >> > >> > > > > > > > > >> it could continue to send epoch 73 and those
> >> records
> >> > >> would
> >> > >> > > not
> >> > >> > > > > be
> >> > >> > > > > > > > > fenced.
> >> > >> > > > > > > > > >> Perhaps this is not an issue if we don't allow
> the
> >> > next
> >> > >> > > > produce
> >> > >> > > > > to
> >> > >> > > > > > > go
> >> > >> > > > > > > > > >> through before the EndTxn request returns. I'm
> >> also
> >> > >> > thinking
> >> > >> > > > > about
> >> > >> > > > > > > > > cases of
> >> > >> > > > > > > > > >> failure. I will need to think on this a bit.
> >> > >> > > > > > > > > >>
> >> > >> > > > > > > > > >> I wasn't sure if it was that confusing. But if
> we
> >> > >> think it
> >> > >> > > is,
> >> > >> > > > > we
> >> > >> > > > > > > can
> >> > >> > > > > > > > > >> investigate other ways.
> >> > >> > > > > > > > > >>
> >> > >> > > > > > > > > >>
> >> > >> > > > > > > > > >> 60)
> >> > >> > > > > > > > > >>
> >> > >> > > > > > > > > >> I'm not sure these are the same purgatories
> since
> >> one
> >> > >> is a
> >> > >> > > > > produce
> >> > >> > > > > > > > > >> purgatory (I was planning on using a callback
> >> rather
> >> > >> than
> >> > >> > > > > > purgatory)
> >> > >> > > > > > > > and
> >> > >> > > > > > > > > >> the other is simply a request to append to the
> >> log.
> >> > Not
> >> > >> > sure
> >> > >> > > > we
> >> > >> > > > > > have
> >> > >> > > > > > > > any
> >> > >> > > > > > > > > >> structure here for ordering, but my
> understanding
> >> is
> >> > >> that
> >> > >> > > the
> >> > >> > > > > > broker
> >> > >> > > > > > > > > could
> >> > >> > > > > > > > > >> handle the write request before it hears back
> from
> >> > the
> >> > >> Txn
> >> > >> > > > > > > > Coordinator.
> >> > >> > > > > > > > > >>
> >> > >> > > > > > > > > >> Let me know if I misunderstood something or
> >> something
> >> > >> was
> >> > >> > > > > unclear.
> >> > >> > > > > > > > > >>
> >> > >> > > > > > > > > >> Justine
> >> > >> > > > > > > > > >>
> >> > >> > > > > > > > > >> On Thu, Dec 1, 2022 at 12:15 PM Matthias J. Sax
> <
> >> > >> > > > > mjsax@apache.org
> >> > >> > > > > > >
> >> > >> > > > > > > > > wrote:
> >> > >> > > > > > > > > >>
> >> > >> > > > > > > > > >>> Thanks for the details Justine!
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > > >>>> 20)
> >> > >> > > > > > > > > >>>>
> >> > >> > > > > > > > > >>>> The client side change for 2 is removing the
> >> > >> > addPartitions
> >> > >> > > > to
> >> > >> > > > > > > > > >>> transaction
> >> > >> > > > > > > > > >>>> call. We don't need to make this from the
> >> producer
> >> > to
> >> > >> > the
> >> > >> > > > txn
> >> > >> > > > > > > > > >>> coordinator,
> >> > >> > > > > > > > > >>>> only server side.
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > > >>> I think I did not express myself clearly. I
> >> > understand
> >> > >> > that
> >> > >> > > > we
> >> > >> > > > > > can
> >> > >> > > > > > > > (and
> >> > >> > > > > > > > > >>> should) change the producer to not send the
> >> > >> > `addPartitions`
> >> > >> > > > > > request
> >> > >> > > > > > > > any
> >> > >> > > > > > > > > >>> longer. But I don't thinks it's requirement to
> >> > change
> >> > >> the
> >> > >> > > > > broker?
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > > >>> What I am trying to say is: as a safe-guard and
> >> > >> > improvement
> >> > >> > > > for
> >> > >> > > > > > > older
> >> > >> > > > > > > > > >>> producers, the partition leader can just send
> the
> >> > >> > > > > `addPartitions`
> >> > >> > > > > > > > > >>> request to the TX-coordinator in any case -- if
> >> the
> >> > >> old
> >> > >> > > > > producer
> >> > >> > > > > > > > > >>> correctly did send the `addPartition` request
> to
> >> the
> >> > >> > > > > > TX-coordinator
> >> > >> > > > > > > > > >>> already, the TX-coordinator can just "ignore"
> is
> >> as
> >> > >> > > > idempotent.
> >> > >> > > > > > > > > However,
> >> > >> > > > > > > > > >>> if the old producer has a bug and did forget to
> >> sent
> >> > >> the
> >> > >> > > > > > > > `addPartition`
> >> > >> > > > > > > > > >>> request, we would now ensure that the partition
> >> is
> >> > >> indeed
> >> > >> > > > added
> >> > >> > > > > > to
> >> > >> > > > > > > > the
> >> > >> > > > > > > > > >>> TX and thus fix a potential producer bug (even
> >> if we
> >> > >> > don't
> >> > >> > > > get
> >> > >> > > > > > the
> >> > >> > > > > > > > > >>> fencing via the bump epoch). -- It seems to be
> a
> >> > good
> >> > >> > > > > > improvement?
> >> > >> > > > > > > Or
> >> > >> > > > > > > > > is
> >> > >> > > > > > > > > >>> there a reason to not do this?
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > > >>>> 30)
> >> > >> > > > > > > > > >>>>
> >> > >> > > > > > > > > >>>> Transaction is ongoing = partition was added
> to
> >> > >> > > transaction
> >> > >> > > > > via
> >> > >> > > > > > > > > >>>> addPartitionsToTxn. We check this with the
> >> > >> > > > > DescribeTransactions
> >> > >> > > > > > > > call.
> >> > >> > > > > > > > > >>> Let
> >> > >> > > > > > > > > >>>> me know if this wasn't sufficiently explained
> >> here:
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > > >>> If we do what I propose in (20), we don't
> really
> >> > need
> >> > >> to
> >> > >> > > make
> >> > >> > > > > > this
> >> > >> > > > > > > > > >>> `DescribeTransaction` call, as the partition
> >> leader
> >> > >> adds
> >> > >> > > the
> >> > >> > > > > > > > partition
> >> > >> > > > > > > > > >>> for older clients and we get this check for
> free.
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > > >>>> 40)
> >> > >> > > > > > > > > >>>>
> >> > >> > > > > > > > > >>>> The idea here is that if any messages somehow
> >> come
> >> > in
> >> > >> > > before
> >> > >> > > > > we
> >> > >> > > > > > > get
> >> > >> > > > > > > > > the
> >> > >> > > > > > > > > >>> new
> >> > >> > > > > > > > > >>>> epoch to the producer, they will be fenced.
> >> > However,
> >> > >> if
> >> > >> > we
> >> > >> > > > > don't
> >> > >> > > > > > > > think
> >> > >> > > > > > > > > >>> this
> >> > >> > > > > > > > > >>>> is necessary, it can be discussed
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > > >>> I agree that we should have epoch fencing. My
> >> > >> question is
> >> > >> > > > > > > different:
> >> > >> > > > > > > > > >>> Assume we are at epoch 73, and we have an
> ongoing
> >> > >> > > > transaction,
> >> > >> > > > > > that
> >> > >> > > > > > > > is
> >> > >> > > > > > > > > >>> committed. It seems natural to write the
> "prepare
> >> > >> commit"
> >> > >> > > > > marker
> >> > >> > > > > > > and
> >> > >> > > > > > > > > the
> >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` both with epoch 73, too,
> >> as
> >> > it
> >> > >> > > belongs
> >> > >> > > > > to
> >> > >> > > > > > > the
> >> > >> > > > > > > > > >>> current transaction. Of course, we now also
> bump
> >> the
> >> > >> > epoch
> >> > >> > > > and
> >> > >> > > > > > > expect
> >> > >> > > > > > > > > >>> the next requests to have epoch 74, and would
> >> reject
> >> > >> an
> >> > >> > > > request
> >> > >> > > > > > > with
> >> > >> > > > > > > > > >>> epoch 73, as the corresponding TX for epoch 73
> >> was
> >> > >> > already
> >> > >> > > > > > > committed.
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > > >>> It seems you propose to write the "prepare
> commit
> >> > >> marker"
> >> > >> > > and
> >> > >> > > > > > > > > >>> `WriteTxMarkerRequest` with epoch 74 though,
> what
> >> > >> would
> >> > >> > > work,
> >> > >> > > > > but
> >> > >> > > > > > > it
> >> > >> > > > > > > > > >>> seems confusing. Is there a reason why we would
> >> use
> >> > >> the
> >> > >> > > > bumped
> >> > >> > > > > > > epoch
> >> > >> > > > > > > > 74
> >> > >> > > > > > > > > >>> instead of the current epoch 73?
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > > >>>> 60)
> >> > >> > > > > > > > > >>>>
> >> > >> > > > > > > > > >>>> When we are checking if the transaction is
> >> ongoing,
> >> > >> we
> >> > >> > > need
> >> > >> > > > to
> >> > >> > > > > > > make
> >> > >> > > > > > > > a
> >> > >> > > > > > > > > >>> round
> >> > >> > > > > > > > > >>>> trip from the leader partition to the
> >> transaction
> >> > >> > > > coordinator.
> >> > >> > > > > > In
> >> > >> > > > > > > > the
> >> > >> > > > > > > > > >>> time
> >> > >> > > > > > > > > >>>> we are waiting for this message to come back,
> in
> >> > >> theory
> >> > >> > we
> >> > >> > > > > could
> >> > >> > > > > > > > have
> >> > >> > > > > > > > > >>> sent
> >> > >> > > > > > > > > >>>> a commit/abort call that would make the
> original
> >> > >> result
> >> > >> > of
> >> > >> > > > the
> >> > >> > > > > > > check
> >> > >> > > > > > > > > >>> out of
> >> > >> > > > > > > > > >>>> date. That is why we can check the leader
> state
> >> > >> before
> >> > >> > we
> >> > >> > > > > write
> >> > >> > > > > > to
> >> > >> > > > > > > > the
> >> > >> > > > > > > > > >>> log.
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > > >>> Thanks. Got it.
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > > >>> However, is this really an issue? We put the
> >> produce
> >> > >> > > request
> >> > >> > > > in
> >> > >> > > > > > > > > >>> purgatory, so how could we process the
> >> > >> > > > `WriteTxnMarkerRequest`
> >> > >> > > > > > > first?
> >> > >> > > > > > > > > >>> Don't we need to put the
> `WriteTxnMarkerRequest`
> >> > into
> >> > >> > > > > purgatory,
> >> > >> > > > > > > too,
> >> > >> > > > > > > > > >>> for this case, and process both request
> in-order?
> >> > >> (Again,
> >> > >> > > my
> >> > >> > > > > > broker
> >> > >> > > > > > > > > >>> knowledge is limited and maybe we don't
> maintain
> >> > >> request
> >> > >> > > > order
> >> > >> > > > > > for
> >> > >> > > > > > > > this
> >> > >> > > > > > > > > >>> case, what seems to be an issue IMHO, and I am
> >> > >> wondering
> >> > >> > if
> >> > >> > > > > > > changing
> >> > >> > > > > > > > > >>> request handling to preserve order for this
> case
> >> > >> might be
> >> > >> > > the
> >> > >> > > > > > > cleaner
> >> > >> > > > > > > > > >>> solution?)
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > > >>> -Matthias
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > > >>> On 11/30/22 3:28 PM, Artem Livshits wrote:
> >> > >> > > > > > > > > >>>> Hi Justine,
> >> > >> > > > > > > > > >>>>
> >> > >> > > > > > > > > >>>> I think the interesting part is not in this
> >> logic
> >> > >> > (because
> >> > >> > > > it
> >> > >> > > > > > > tries
> >> > >> > > > > > > > to
> >> > >> > > > > > > > > >>>> figure out when UNKNOWN_PRODUCER_ID is
> retriable
> >> > and
> >> > >> if
> >> > >> > > it's
> >> > >> > > > > > > > > retryable,
> >> > >> > > > > > > > > >>>> it's definitely not fatal), but what happens
> >> when
> >> > >> this
> >> > >> > > logic
> >> > >> > > > > > > doesn't
> >> > >> > > > > > > > > >>> return
> >> > >> > > > > > > > > >>>> 'true' and falls through.  In the old clients
> it
> >> > >> seems
> >> > >> > to
> >> > >> > > be
> >> > >> > > > > > > fatal,
> >> > >> > > > > > > > if
> >> > >> > > > > > > > > >>> we
> >> > >> > > > > > > > > >>>> keep the behavior in the new clients, I'd
> >> expect it
> >> > >> > would
> >> > >> > > be
> >> > >> > > > > > fatal
> >> > >> > > > > > > > as
> >> > >> > > > > > > > > >>> well.
> >> > >> > > > > > > > > >>>>
> >> > >> > > > > > > > > >>>> -Artem
> >> > >> > > > > > > > > >>>>
> >> > >> > > > > > > > > >>>> On Tue, Nov 29, 2022 at 11:57 AM Justine
> Olshan
> >> > >> > > > > > > > > >>>> <jo...@confluent.io.invalid> wrote:
> >> > >> > > > > > > > > >>>>
> >> > >> > > > > > > > > >>>>> Hi Artem and Jeff,
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>> Thanks for taking a look and sorry for the
> slow
> >> > >> > response.
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>> You both mentioned the change to handle
> >> > >> > > UNKNOWN_PRODUCER_ID
> >> > >> > > > > > > errors.
> >> > >> > > > > > > > > To
> >> > >> > > > > > > > > >>> be
> >> > >> > > > > > > > > >>>>> clear — this error code will only be sent
> again
> >> > when
> >> > >> > the
> >> > >> > > > > > client's
> >> > >> > > > > > > > > >>> request
> >> > >> > > > > > > > > >>>>> version is high enough to ensure we handle it
> >> > >> > correctly.
> >> > >> > > > > > > > > >>>>> The current (Java) client handles this by the
> >> > >> following
> >> > >> > > > > > (somewhat
> >> > >> > > > > > > > > long)
> >> > >> > > > > > > > > >>>>> code snippet:
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>> // An UNKNOWN_PRODUCER_ID means that we have
> >> lost
> >> > >> the
> >> > >> > > > > producer
> >> > >> > > > > > > > state
> >> > >> > > > > > > > > >>> on the
> >> > >> > > > > > > > > >>>>> broker. Depending on the log start
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>> // offset, we may want to retry these, as
> >> > described
> >> > >> for
> >> > >> > > > each
> >> > >> > > > > > case
> >> > >> > > > > > > > > >>> below. If
> >> > >> > > > > > > > > >>>>> none of those apply, then for the
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>> // idempotent producer, we will locally bump
> >> the
> >> > >> epoch
> >> > >> > > and
> >> > >> > > > > > reset
> >> > >> > > > > > > > the
> >> > >> > > > > > > > > >>>>> sequence numbers of in-flight batches from
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>> // sequence 0, then retry the failed batch,
> >> which
> >> > >> > should
> >> > >> > > > now
> >> > >> > > > > > > > succeed.
> >> > >> > > > > > > > > >>> For
> >> > >> > > > > > > > > >>>>> the transactional producer, allow the
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>> // batch to fail. When processing the failed
> >> > batch,
> >> > >> we
> >> > >> > > will
> >> > >> > > > > > > > > transition
> >> > >> > > > > > > > > >>> to
> >> > >> > > > > > > > > >>>>> an abortable error and set a flag
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>> // indicating that we need to bump the epoch
> >> (if
> >> > >> > > supported
> >> > >> > > > by
> >> > >> > > > > > the
> >> > >> > > > > > > > > >>> broker).
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>> if (error == Errors.*UNKNOWN_PRODUCER_ID*) {
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>       if (response.logStartOffset == -1) {
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           // We don't know the log start
> offset
> >> > with
> >> > >> > this
> >> > >> > > > > > > response.
> >> > >> > > > > > > > > We
> >> > >> > > > > > > > > >>> should
> >> > >> > > > > > > > > >>>>> just retry the request until we get it.
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           // The UNKNOWN_PRODUCER_ID error
> code
> >> > was
> >> > >> > added
> >> > >> > > > > along
> >> > >> > > > > > > > with
> >> > >> > > > > > > > > >>> the new
> >> > >> > > > > > > > > >>>>> ProduceResponse which includes the
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           // logStartOffset. So the '-1'
> >> sentinel
> >> > is
> >> > >> > not
> >> > >> > > > for
> >> > >> > > > > > > > backward
> >> > >> > > > > > > > > >>>>> compatibility. Instead, it is possible for
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           // a broker to not know the
> >> > >> logStartOffset at
> >> > >> > > > when
> >> > >> > > > > it
> >> > >> > > > > > > is
> >> > >> > > > > > > > > >>> returning
> >> > >> > > > > > > > > >>>>> the response because the partition
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           // may have moved away from the
> >> broker
> >> > >> from
> >> > >> > the
> >> > >> > > > > time
> >> > >> > > > > > > the
> >> > >> > > > > > > > > >>> error was
> >> > >> > > > > > > > > >>>>> initially raised to the time the
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           // response was being constructed.
> In
> >> > >> these
> >> > >> > > > cases,
> >> > >> > > > > we
> >> > >> > > > > > > > > should
> >> > >> > > > > > > > > >>> just
> >> > >> > > > > > > > > >>>>> retry the request: we are guaranteed
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           // to eventually get a
> logStartOffset
> >> > once
> >> > >> > > things
> >> > >> > > > > > > settle
> >> > >> > > > > > > > > down.
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           return true;
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>       }
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>       if (batch.sequenceHasBeenReset()) {
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           // When the first inflight batch
> >> fails
> >> > >> due to
> >> > >> > > the
> >> > >> > > > > > > > > truncation
> >> > >> > > > > > > > > >>> case,
> >> > >> > > > > > > > > >>>>> then the sequences of all the other
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           // in flight batches would have
> been
> >> > >> > restarted
> >> > >> > > > from
> >> > >> > > > > > the
> >> > >> > > > > > > > > >>> beginning.
> >> > >> > > > > > > > > >>>>> However, when those responses
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           // come back from the broker, they
> >> would
> >> > >> also
> >> > >> > > > come
> >> > >> > > > > > with
> >> > >> > > > > > > > an
> >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID error. In this case, we
> >> should
> >> > >> not
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           // reset the sequence numbers to
> the
> >> > >> > beginning.
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           return true;
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>       } else if
> >> > >> > > > > (lastAckedOffset(batch.topicPartition).orElse(
> >> > >> > > > > > > > > >>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> >> > >> > > > response.logStartOffset) {
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           // The head of the log has been
> >> removed,
> >> > >> > > probably
> >> > >> > > > > due
> >> > >> > > > > > > to
> >> > >> > > > > > > > > the
> >> > >> > > > > > > > > >>>>> retention time elapsing. In this case,
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           // we expect to lose the producer
> >> state.
> >> > >> For
> >> > >> > > the
> >> > >> > > > > > > > > transactional
> >> > >> > > > > > > > > >>>>> producer, reset the sequences of all
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           // inflight batches to be from the
> >> > >> beginning
> >> > >> > > and
> >> > >> > > > > > retry
> >> > >> > > > > > > > > them,
> >> > >> > > > > > > > > >>> so
> >> > >> > > > > > > > > >>>>> that the transaction does not need to
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           // be aborted. For the idempotent
> >> > >> producer,
> >> > >> > > bump
> >> > >> > > > > the
> >> > >> > > > > > > > epoch
> >> > >> > > > > > > > > to
> >> > >> > > > > > > > > >>> avoid
> >> > >> > > > > > > > > >>>>> reusing (sequence, epoch) pairs
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           if (isTransactional()) {
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>
> >> > >> > > >
> txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> >> > >> > > > > > > > > >>>>> this.producerIdAndEpoch);
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           } else {
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > >  requestEpochBumpForPartition(batch.topicPartition);
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           }
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           return true;
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>       }
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>       if (!isTransactional()) {
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           // For the idempotent producer,
> >> always
> >> > >> retry
> >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> >> > >> > > > > > > > > >>>>> errors. If the batch has the current
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           // producer ID and epoch, request a
> >> bump
> >> > >> of
> >> > >> > the
> >> > >> > > > > > epoch.
> >> > >> > > > > > > > > >>> Otherwise
> >> > >> > > > > > > > > >>>>> just retry the produce.
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>
> >> > >> > > >  requestEpochBumpForPartition(batch.topicPartition);
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>           return true;
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>       }
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>> }
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>> I was considering keeping this behavior — but
> >> am
> >> > >> open
> >> > >> > to
> >> > >> > > > > > > > simplifying
> >> > >> > > > > > > > > >>> it.
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>> We are leaving changes to older clients off
> the
> >> > >> table
> >> > >> > > here
> >> > >> > > > > > since
> >> > >> > > > > > > it
> >> > >> > > > > > > > > >>> caused
> >> > >> > > > > > > > > >>>>> many issues for clients in the past.
> Previously
> >> > this
> >> > >> > was
> >> > >> > > a
> >> > >> > > > > > fatal
> >> > >> > > > > > > > > error
> >> > >> > > > > > > > > >>> and
> >> > >> > > > > > > > > >>>>> we didn't have the mechanisms in place to
> >> detect
> >> > >> when
> >> > >> > > this
> >> > >> > > > > was
> >> > >> > > > > > a
> >> > >> > > > > > > > > >>> legitimate
> >> > >> > > > > > > > > >>>>> case vs some bug or gap in the protocol.
> >> Ensuring
> >> > >> each
> >> > >> > > > > > > transaction
> >> > >> > > > > > > > > has
> >> > >> > > > > > > > > >>> its
> >> > >> > > > > > > > > >>>>> own epoch should close this gap.
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>> And to address Jeff's second point:
> >> > >> > > > > > > > > >>>>> *does the typical produce request path append
> >> > >> records
> >> > >> > to
> >> > >> > > > > local
> >> > >> > > > > > > log
> >> > >> > > > > > > > > >>> along*
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>> *with the currentTxnFirstOffset information?
> I
> >> > would
> >> > >> > like
> >> > >> > > > to
> >> > >> > > > > > > > > >>> understand*
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>> *when the field is written to disk.*
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>> Yes, the first produce request populates this
> >> > field
> >> > >> and
> >> > >> > > > > writes
> >> > >> > > > > > > the
> >> > >> > > > > > > > > >>> offset
> >> > >> > > > > > > > > >>>>> as part of the record batch and also to the
> >> > producer
> >> > >> > > state
> >> > >> > > > > > > > snapshot.
> >> > >> > > > > > > > > >>> When
> >> > >> > > > > > > > > >>>>> we reload the records on restart and/or
> >> > >> reassignment,
> >> > >> > we
> >> > >> > > > > > > repopulate
> >> > >> > > > > > > > > >>> this
> >> > >> > > > > > > > > >>>>> field with the snapshot from disk along with
> >> the
> >> > >> rest
> >> > >> > of
> >> > >> > > > the
> >> > >> > > > > > > > producer
> >> > >> > > > > > > > > >>>>> state.
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>> Let me know if there are further comments
> >> and/or
> >> > >> > > questions.
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>> Thanks,
> >> > >> > > > > > > > > >>>>> Justine
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>> On Tue, Nov 22, 2022 at 9:00 PM Jeff Kim
> >> > >> > > > > > > > > <jeff.kim@confluent.io.invalid
> >> > >> > > > > > > > > >>>>
> >> > >> > > > > > > > > >>>>> wrote:
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>>> Hi Justine,
> >> > >> > > > > > > > > >>>>>>
> >> > >> > > > > > > > > >>>>>> Thanks for the KIP! I have two questions:
> >> > >> > > > > > > > > >>>>>>
> >> > >> > > > > > > > > >>>>>> 1) For new clients, we can once again return
> >> an
> >> > >> error
> >> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> >> > >> > > > > > > > > >>>>>> for sequences
> >> > >> > > > > > > > > >>>>>> that are non-zero when there is no producer
> >> state
> >> > >> > > present
> >> > >> > > > on
> >> > >> > > > > > the
> >> > >> > > > > > > > > >>> server.
> >> > >> > > > > > > > > >>>>>> This will indicate we missed the 0 sequence
> >> and
> >> > we
> >> > >> > don't
> >> > >> > > > yet
> >> > >> > > > > > > want
> >> > >> > > > > > > > to
> >> > >> > > > > > > > > >>>>> write
> >> > >> > > > > > > > > >>>>>> to the log.
> >> > >> > > > > > > > > >>>>>>
> >> > >> > > > > > > > > >>>>>> I would like to understand the current
> >> behavior
> >> > to
> >> > >> > > handle
> >> > >> > > > > > older
> >> > >> > > > > > > > > >>> clients,
> >> > >> > > > > > > > > >>>>>> and if there are any changes we are making.
> >> Maybe
> >> > >> I'm
> >> > >> > > > > missing
> >> > >> > > > > > > > > >>> something,
> >> > >> > > > > > > > > >>>>>> but we would want to identify whether we
> >> missed
> >> > >> the 0
> >> > >> > > > > sequence
> >> > >> > > > > > > for
> >> > >> > > > > > > > > >>> older
> >> > >> > > > > > > > > >>>>>> clients, no?
> >> > >> > > > > > > > > >>>>>>
> >> > >> > > > > > > > > >>>>>> 2) Upon returning from the transaction
> >> > >> coordinator, we
> >> > >> > > can
> >> > >> > > > > set
> >> > >> > > > > > > the
> >> > >> > > > > > > > > >>>>>> transaction
> >> > >> > > > > > > > > >>>>>> as ongoing on the leader by populating
> >> > >> > > > currentTxnFirstOffset
> >> > >> > > > > > > > > >>>>>> through the typical produce request
> handling.
> >> > >> > > > > > > > > >>>>>>
> >> > >> > > > > > > > > >>>>>> does the typical produce request path append
> >> > >> records
> >> > >> > to
> >> > >> > > > > local
> >> > >> > > > > > > log
> >> > >> > > > > > > > > >>> along
> >> > >> > > > > > > > > >>>>>> with the currentTxnFirstOffset information?
> I
> >> > would
> >> > >> > like
> >> > >> > > > to
> >> > >> > > > > > > > > understand
> >> > >> > > > > > > > > >>>>>> when the field is written to disk.
> >> > >> > > > > > > > > >>>>>>
> >> > >> > > > > > > > > >>>>>> Thanks,
> >> > >> > > > > > > > > >>>>>> Jeff
> >> > >> > > > > > > > > >>>>>>
> >> > >> > > > > > > > > >>>>>>
> >> > >> > > > > > > > > >>>>>> On Tue, Nov 22, 2022 at 4:44 PM Artem
> Livshits
> >> > >> > > > > > > > > >>>>>> <al...@confluent.io.invalid> wrote:
> >> > >> > > > > > > > > >>>>>>
> >> > >> > > > > > > > > >>>>>>> Hi Justine,
> >> > >> > > > > > > > > >>>>>>>
> >> > >> > > > > > > > > >>>>>>> Thank you for the KIP.  I have one
> question.
> >> > >> > > > > > > > > >>>>>>>
> >> > >> > > > > > > > > >>>>>>> 5) For new clients, we can once again
> return
> >> an
> >> > >> error
> >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> >> > >> > > > > > > > > >>>>>>>
> >> > >> > > > > > > > > >>>>>>> I believe we had problems in the past with
> >> > >> returning
> >> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> >> > >> > > > > > > > > >>>>>>> because it was considered fatal and
> required
> >> > >> client
> >> > >> > > > > restart.
> >> > >> > > > > > > It
> >> > >> > > > > > > > > >>> would
> >> > >> > > > > > > > > >>>>> be
> >> > >> > > > > > > > > >>>>>>> good to spell out the new client behavior
> >> when
> >> > it
> >> > >> > > > receives
> >> > >> > > > > > the
> >> > >> > > > > > > > > error.
> >> > >> > > > > > > > > >>>>>>>
> >> > >> > > > > > > > > >>>>>>> -Artem
> >> > >> > > > > > > > > >>>>>>>
> >> > >> > > > > > > > > >>>>>>> On Tue, Nov 22, 2022 at 10:00 AM Justine
> >> Olshan
> >> > >> > > > > > > > > >>>>>>> <jo...@confluent.io.invalid> wrote:
> >> > >> > > > > > > > > >>>>>>>
> >> > >> > > > > > > > > >>>>>>>> Thanks for taking a look Matthias. I've
> >> tried
> >> > to
> >> > >> > > answer
> >> > >> > > > > your
> >> > >> > > > > > > > > >>>>> questions
> >> > >> > > > > > > > > >>>>>>>> below:
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>> 10)
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>> Right — so the hanging transaction only
> >> occurs
> >> > >> when
> >> > >> > we
> >> > >> > > > > have
> >> > >> > > > > > a
> >> > >> > > > > > > > late
> >> > >> > > > > > > > > >>>>>>> message
> >> > >> > > > > > > > > >>>>>>>> come in and the partition is never added
> to
> >> a
> >> > >> > > > transaction
> >> > >> > > > > > > again.
> >> > >> > > > > > > > > If
> >> > >> > > > > > > > > >>>>> we
> >> > >> > > > > > > > > >>>>>>>> never add the partition to a transaction,
> we
> >> > will
> >> > >> > > never
> >> > >> > > > > > write
> >> > >> > > > > > > a
> >> > >> > > > > > > > > >>>>> marker
> >> > >> > > > > > > > > >>>>>>> and
> >> > >> > > > > > > > > >>>>>>>> never advance the LSO.
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>> If we do end up adding the partition to
> the
> >> > >> > > transaction
> >> > >> > > > (I
> >> > >> > > > > > > > suppose
> >> > >> > > > > > > > > >>>>> this
> >> > >> > > > > > > > > >>>>>>> can
> >> > >> > > > > > > > > >>>>>>>> happen before or after the late message
> >> comes
> >> > in)
> >> > >> > then
> >> > >> > > > we
> >> > >> > > > > > will
> >> > >> > > > > > > > > >>>>> include
> >> > >> > > > > > > > > >>>>>>> the
> >> > >> > > > > > > > > >>>>>>>> late message in the next (incorrect)
> >> > transaction.
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>> So perhaps it is clearer to make the
> >> > distinction
> >> > >> > > between
> >> > >> > > > > > > > messages
> >> > >> > > > > > > > > >>>>> that
> >> > >> > > > > > > > > >>>>>>>> eventually get added to the transaction
> (but
> >> > the
> >> > >> > wrong
> >> > >> > > > > one)
> >> > >> > > > > > or
> >> > >> > > > > > > > > >>>>> messages
> >> > >> > > > > > > > > >>>>>>>> that never get added and become hanging.
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>> 20)
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>> The client side change for 2 is removing
> the
> >> > >> > > > addPartitions
> >> > >> > > > > > to
> >> > >> > > > > > > > > >>>>>> transaction
> >> > >> > > > > > > > > >>>>>>>> call. We don't need to make this from the
> >> > >> producer
> >> > >> > to
> >> > >> > > > the
> >> > >> > > > > > txn
> >> > >> > > > > > > > > >>>>>>> coordinator,
> >> > >> > > > > > > > > >>>>>>>> only server side.
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>> In my opinion, the issue with the
> >> > >> addPartitionsToTxn
> >> > >> > > > call
> >> > >> > > > > > for
> >> > >> > > > > > > > > older
> >> > >> > > > > > > > > >>>>>>> clients
> >> > >> > > > > > > > > >>>>>>>> is that we don't have the epoch bump, so
> we
> >> > don't
> >> > >> > know
> >> > >> > > > if
> >> > >> > > > > > the
> >> > >> > > > > > > > > >>> message
> >> > >> > > > > > > > > >>>>>>>> belongs to the previous transaction or
> this
> >> > one.
> >> > >> We
> >> > >> > > need
> >> > >> > > > > to
> >> > >> > > > > > > > check
> >> > >> > > > > > > > > if
> >> > >> > > > > > > > > >>>>>> the
> >> > >> > > > > > > > > >>>>>>>> partition has been added to this
> >> transaction.
> >> > Of
> >> > >> > > course,
> >> > >> > > > > > this
> >> > >> > > > > > > > > means
> >> > >> > > > > > > > > >>>>> we
> >> > >> > > > > > > > > >>>>>>>> won't completely cover the case where we
> >> have a
> >> > >> > really
> >> > >> > > > > late
> >> > >> > > > > > > > > message
> >> > >> > > > > > > > > >>>>> and
> >> > >> > > > > > > > > >>>>>>> we
> >> > >> > > > > > > > > >>>>>>>> have added the partition to the new
> >> > transaction,
> >> > >> but
> >> > >> > > > > that's
> >> > >> > > > > > > > > >>>>>> unfortunately
> >> > >> > > > > > > > > >>>>>>>> something we will need the new clients to
> >> > cover.
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>> 30)
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>> Transaction is ongoing = partition was
> >> added to
> >> > >> > > > > transaction
> >> > >> > > > > > > via
> >> > >> > > > > > > > > >>>>>>>> addPartitionsToTxn. We check this with the
> >> > >> > > > > > > DescribeTransactions
> >> > >> > > > > > > > > >>> call.
> >> > >> > > > > > > > > >>>>>> Let
> >> > >> > > > > > > > > >>>>>>>> me know if this wasn't sufficiently
> >> explained
> >> > >> here:
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>
> >> > >> > > > > > > > > >>>>>>
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > >
> >> > >> > > > > > > >
> >> > >> > > > > > >
> >> > >> > > > > >
> >> > >> > > > >
> >> > >> > > >
> >> > >> > >
> >> > >> >
> >> > >>
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>> 40)
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>> The idea here is that if any messages
> >> somehow
> >> > >> come
> >> > >> > in
> >> > >> > > > > before
> >> > >> > > > > > > we
> >> > >> > > > > > > > > get
> >> > >> > > > > > > > > >>>>> the
> >> > >> > > > > > > > > >>>>>>> new
> >> > >> > > > > > > > > >>>>>>>> epoch to the producer, they will be
> fenced.
> >> > >> However,
> >> > >> > > if
> >> > >> > > > we
> >> > >> > > > > > > don't
> >> > >> > > > > > > > > >>>>> think
> >> > >> > > > > > > > > >>>>>>> this
> >> > >> > > > > > > > > >>>>>>>> is necessary, it can be discussed
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>> 50)
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>> It should be synchronous because if we
> have
> >> an
> >> > >> event
> >> > >> > > > (ie,
> >> > >> > > > > an
> >> > >> > > > > > > > > error)
> >> > >> > > > > > > > > >>>>>> that
> >> > >> > > > > > > > > >>>>>>>> causes us to need to abort the
> transaction,
> >> we
> >> > >> need
> >> > >> > to
> >> > >> > > > > know
> >> > >> > > > > > > > which
> >> > >> > > > > > > > > >>>>>>>> partitions to send transaction markers to.
> >> We
> >> > >> know
> >> > >> > the
> >> > >> > > > > > > > partitions
> >> > >> > > > > > > > > >>>>>> because
> >> > >> > > > > > > > > >>>>>>>> we added them to the coordinator via the
> >> > >> > > > > addPartitionsToTxn
> >> > >> > > > > > > > call.
> >> > >> > > > > > > > > >>>>>>>> Previously we have had asynchronous calls
> in
> >> > the
> >> > >> > past
> >> > >> > > > (ie,
> >> > >> > > > > > > > writing
> >> > >> > > > > > > > > >>>>> the
> >> > >> > > > > > > > > >>>>>>>> commit markers when the transaction is
> >> > completed)
> >> > >> > but
> >> > >> > > > > often
> >> > >> > > > > > > this
> >> > >> > > > > > > > > >>> just
> >> > >> > > > > > > > > >>>>>>>> causes confusion as we need to wait for
> some
> >> > >> > > operations
> >> > >> > > > to
> >> > >> > > > > > > > > complete.
> >> > >> > > > > > > > > >>>>> In
> >> > >> > > > > > > > > >>>>>>> the
> >> > >> > > > > > > > > >>>>>>>> writing commit markers case, clients often
> >> see
> >> > >> > > > > > > > > >>>>> CONCURRENT_TRANSACTIONs
> >> > >> > > > > > > > > >>>>>>>> error messages and that can be confusing.
> >> For
> >> > >> that
> >> > >> > > > reason,
> >> > >> > > > > > it
> >> > >> > > > > > > > may
> >> > >> > > > > > > > > be
> >> > >> > > > > > > > > >>>>>>>> simpler to just have synchronous calls —
> >> > >> especially
> >> > >> > if
> >> > >> > > > we
> >> > >> > > > > > need
> >> > >> > > > > > > > to
> >> > >> > > > > > > > > >>>>> block
> >> > >> > > > > > > > > >>>>>>> on
> >> > >> > > > > > > > > >>>>>>>> some operation's completion anyway before
> we
> >> > can
> >> > >> > start
> >> > >> > > > the
> >> > >> > > > > > > next
> >> > >> > > > > > > > > >>>>>>>> transaction. And yes, I meant
> coordinator. I
> >> > will
> >> > >> > fix
> >> > >> > > > > that.
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>> 60)
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>> When we are checking if the transaction is
> >> > >> ongoing,
> >> > >> > we
> >> > >> > > > > need
> >> > >> > > > > > to
> >> > >> > > > > > > > > make
> >> > >> > > > > > > > > >>> a
> >> > >> > > > > > > > > >>>>>>> round
> >> > >> > > > > > > > > >>>>>>>> trip from the leader partition to the
> >> > transaction
> >> > >> > > > > > coordinator.
> >> > >> > > > > > > > In
> >> > >> > > > > > > > > >>> the
> >> > >> > > > > > > > > >>>>>>> time
> >> > >> > > > > > > > > >>>>>>>> we are waiting for this message to come
> >> back,
> >> > in
> >> > >> > > theory
> >> > >> > > > we
> >> > >> > > > > > > could
> >> > >> > > > > > > > > >>> have
> >> > >> > > > > > > > > >>>>>>> sent
> >> > >> > > > > > > > > >>>>>>>> a commit/abort call that would make the
> >> > original
> >> > >> > > result
> >> > >> > > > of
> >> > >> > > > > > the
> >> > >> > > > > > > > > check
> >> > >> > > > > > > > > >>>>>> out
> >> > >> > > > > > > > > >>>>>>> of
> >> > >> > > > > > > > > >>>>>>>> date. That is why we can check the leader
> >> state
> >> > >> > before
> >> > >> > > > we
> >> > >> > > > > > > write
> >> > >> > > > > > > > to
> >> > >> > > > > > > > > >>>>> the
> >> > >> > > > > > > > > >>>>>>> log.
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>> I'm happy to update the KIP if some of
> these
> >> > >> things
> >> > >> > > were
> >> > >> > > > > not
> >> > >> > > > > > > > > clear.
> >> > >> > > > > > > > > >>>>>>>> Thanks,
> >> > >> > > > > > > > > >>>>>>>> Justine
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>> On Mon, Nov 21, 2022 at 7:11 PM Matthias
> J.
> >> > Sax <
> >> > >> > > > > > > > mjsax@apache.org
> >> > >> > > > > > > > > >
> >> > >> > > > > > > > > >>>>>>> wrote:
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>> Thanks for the KIP.
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>> Couple of clarification questions (I am
> >> not a
> >> > >> > broker
> >> > >> > > > > expert
> >> > >> > > > > > > do
> >> > >> > > > > > > > > >>>>> maybe
> >> > >> > > > > > > > > >>>>>>>>> some question are obvious for others, but
> >> not
> >> > >> for
> >> > >> > me
> >> > >> > > > with
> >> > >> > > > > > my
> >> > >> > > > > > > > lack
> >> > >> > > > > > > > > >>>>> of
> >> > >> > > > > > > > > >>>>>>>>> broker knowledge).
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>> (10)
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>> 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.
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>> What happens if the message come in
> before
> >> the
> >> > >> next
> >> > >> > > > > > > > > >>>>>> addPartitionsToTxn
> >> > >> > > > > > > > > >>>>>>>>> request? It seems the broker hosting the
> >> data
> >> > >> > > > partitions
> >> > >> > > > > > > won't
> >> > >> > > > > > > > > know
> >> > >> > > > > > > > > >>>>>>>>> anything about it and append it to the
> >> > >> partition,
> >> > >> > > too?
> >> > >> > > > > What
> >> > >> > > > > > > is
> >> > >> > > > > > > > > the
> >> > >> > > > > > > > > >>>>>>>>> difference between both cases?
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>> Also, it seems a TX would only hang, if
> >> there
> >> > >> is no
> >> > >> > > > > > following
> >> > >> > > > > > > > TX
> >> > >> > > > > > > > > >>>>> that
> >> > >> > > > > > > > > >>>>>>> is
> >> > >> > > > > > > > > >>>>>>>>> either committer or aborted? Thus, for
> the
> >> > case
> >> > >> > > above,
> >> > >> > > > > the
> >> > >> > > > > > TX
> >> > >> > > > > > > > > might
> >> > >> > > > > > > > > >>>>>>>>> actually not hang (of course, we might
> get
> >> an
> >> > >> EOS
> >> > >> > > > > violation
> >> > >> > > > > > > if
> >> > >> > > > > > > > > the
> >> > >> > > > > > > > > >>>>>>> first
> >> > >> > > > > > > > > >>>>>>>>> TX was aborted and the second committed,
> or
> >> > the
> >> > >> > other
> >> > >> > > > way
> >> > >> > > > > > > > > around).
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>> (20)
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>> Of course, 1 and 2 require client-side
> >> > >> changes, so
> >> > >> > > for
> >> > >> > > > > > older
> >> > >> > > > > > > > > >>>>>> clients,
> >> > >> > > > > > > > > >>>>>>>>> those approaches won’t apply.
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>> For (1) I understand why a client change
> is
> >> > >> > > necessary,
> >> > >> > > > > but
> >> > >> > > > > > > not
> >> > >> > > > > > > > > sure
> >> > >> > > > > > > > > >>>>>> why
> >> > >> > > > > > > > > >>>>>>>>> we need a client change for (2). Can you
> >> > >> elaborate?
> >> > >> > > --
> >> > >> > > > > > Later
> >> > >> > > > > > > > you
> >> > >> > > > > > > > > >>>>>>> explain
> >> > >> > > > > > > > > >>>>>>>>> that we should send a
> >> > >> DescribeTransactionRequest,
> >> > >> > > but I
> >> > >> > > > > am
> >> > >> > > > > > > not
> >> > >> > > > > > > > > sure
> >> > >> > > > > > > > > >>>>>>> why?
> >> > >> > > > > > > > > >>>>>>>>> Can't we not just do an implicit
> >> > >> AddPartiitonToTx,
> >> > >> > > too?
> >> > >> > > > > If
> >> > >> > > > > > > the
> >> > >> > > > > > > > > old
> >> > >> > > > > > > > > >>>>>>>>> producer correctly registered the
> partition
> >> > >> > already,
> >> > >> > > > the
> >> > >> > > > > > > > > >>>>>> TX-coordinator
> >> > >> > > > > > > > > >>>>>>>>> can just ignore it as it's an idempotent
> >> > >> operation?
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>> (30)
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>> To cover older clients, we will ensure a
> >> > >> > transaction
> >> > >> > > > is
> >> > >> > > > > > > > ongoing
> >> > >> > > > > > > > > >>>>>>> before
> >> > >> > > > > > > > > >>>>>>>>> we write to a transaction
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>> Not sure what you mean by this? Can you
> >> > >> elaborate?
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>> (40)
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>> [the TX-coordinator] will write the
> >> prepare
> >> > >> commit
> >> > >> > > > > message
> >> > >> > > > > > > > with
> >> > >> > > > > > > > > a
> >> > >> > > > > > > > > >>>>>>>> bumped
> >> > >> > > > > > > > > >>>>>>>>> epoch and send WriteTxnMarkerRequests
> with
> >> the
> >> > >> > bumped
> >> > >> > > > > > epoch.
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>> Why do we use the bumped epoch for both?
> It
> >> > >> seems
> >> > >> > > more
> >> > >> > > > > > > > intuitive
> >> > >> > > > > > > > > to
> >> > >> > > > > > > > > >>>>>> use
> >> > >> > > > > > > > > >>>>>>>>> the current epoch, and only return the
> >> bumped
> >> > >> epoch
> >> > >> > > to
> >> > >> > > > > the
> >> > >> > > > > > > > > >>>>> producer?
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>> (50) "Implicit AddPartitionToTransaction"
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>> Why does the implicitly sent request need
> >> to
> >> > be
> >> > >> > > > > > synchronous?
> >> > >> > > > > > > > The
> >> > >> > > > > > > > > >>>>> KIP
> >> > >> > > > > > > > > >>>>>>>>> also says
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>> in case we need to abort and need to
> know
> >> > which
> >> > >> > > > > partitions
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>> What do you mean by this?
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>> we don’t want to write to it before we
> >> store
> >> > in
> >> > >> > the
> >> > >> > > > > > > > transaction
> >> > >> > > > > > > > > >>>>>>> manager
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>> Do you mean TX-coordinator instead of
> >> > "manager"?
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>> (60)
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>> For older clients and ensuring that the
> TX
> >> is
> >> > >> > > ongoing,
> >> > >> > > > > you
> >> > >> > > > > > > > > >>>>> describe a
> >> > >> > > > > > > > > >>>>>>>>> race condition. I am not sure if I can
> >> follow
> >> > >> here.
> >> > >> > > Can
> >> > >> > > > > you
> >> > >> > > > > > > > > >>>>>> elaborate?
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>> -Matthias
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>> On 11/18/22 1:21 PM, Justine Olshan
> wrote:
> >> > >> > > > > > > > > >>>>>>>>>> Hey all!
> >> > >> > > > > > > > > >>>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>> I'd like to start a discussion on my
> >> proposal
> >> > >> to
> >> > >> > add
> >> > >> > > > > some
> >> > >> > > > > > > > > >>>>>> server-side
> >> > >> > > > > > > > > >>>>>>>>>> checks on transactions to avoid hanging
> >> > >> > > transactions.
> >> > >> > > > I
> >> > >> > > > > > know
> >> > >> > > > > > > > > this
> >> > >> > > > > > > > > >>>>>> has
> >> > >> > > > > > > > > >>>>>>>>> been
> >> > >> > > > > > > > > >>>>>>>>>> an issue for some time, so I really hope
> >> this
> >> > >> KIP
> >> > >> > > will
> >> > >> > > > > be
> >> > >> > > > > > > > > helpful
> >> > >> > > > > > > > > >>>>>> for
> >> > >> > > > > > > > > >>>>>>>>> many
> >> > >> > > > > > > > > >>>>>>>>>> users of EOS.
> >> > >> > > > > > > > > >>>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>> The KIP includes changes that will be
> >> > >> compatible
> >> > >> > > with
> >> > >> > > > > old
> >> > >> > > > > > > > > clients
> >> > >> > > > > > > > > >>>>>> and
> >> > >> > > > > > > > > >>>>>>>>>> changes to improve performance and
> >> > correctness
> >> > >> on
> >> > >> > > new
> >> > >> > > > > > > clients.
> >> > >> > > > > > > > > >>>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>> Please take a look and leave any
> comments
> >> you
> >> > >> may
> >> > >> > > > have!
> >> > >> > > > > > > > > >>>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>> KIP:
> >> > >> > > > > > > > > >>>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>
> >> > >> > > > > > > > > >>>>>>
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > >
> >> > >> > > > > > > >
> >> > >> > > > > > >
> >> > >> > > > > >
> >> > >> > > > >
> >> > >> > > >
> >> > >> > >
> >> > >> >
> >> > >>
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> >> > >> > > > > > > > > >>>>>>>>>> JIRA:
> >> > >> > > > https://issues.apache.org/jira/browse/KAFKA-14402
> >> > >> > > > > > > > > >>>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>> Thanks!
> >> > >> > > > > > > > > >>>>>>>>>> Justine
> >> > >> > > > > > > > > >>>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>>
> >> > >> > > > > > > > > >>>>>>>>
> >> > >> > > > > > > > > >>>>>>>
> >> > >> > > > > > > > > >>>>>>
> >> > >> > > > > > > > > >>>>>
> >> > >> > > > > > > > > >>>>
> >> > >> > > > > > > > > >>>
> >> > >> > > > > > > > > >>
> >> > >> > > > > > > > > >
> >> > >> > > > > > > > >
> >> > >> > > > > > > >
> >> > >> > > > > > >
> >> > >> > > > > >
> >> > >> > > > >
> >> > >> > > >
> >> > >> > >
> >> > >> >
> >> > >>
> >> > >
> >> >
> >>
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
Hey all, I've updated the KIP to incorporate Jason's suggestions.

https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense


1. Use AddPartitionsToTxn + verify flag to check on old clients
2. Updated AddPartitionsToTxn API to support transaction batching
3. Mention IBP bump
4. Mention auth change on new AddPartitionsToTxn version.

I'm planning on opening a vote soon.
Thanks,
Justine

On Fri, Jan 6, 2023 at 3:32 PM Justine Olshan <jo...@confluent.io> wrote:

> Thanks Jason. Those changes make sense to me. I will update the KIP.
>
>
>
> On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson <ja...@confluent.io.invalid>
> wrote:
>
>> Hey Justine,
>>
>> > I was wondering about compatibility here. When we send requests
>> between brokers, we want to ensure that the receiving broker understands
>> the request (specifically the new fields). Typically this is done via
>> IBP/metadata version.
>> I'm trying to think if there is a way around it but I'm not sure there is.
>>
>> Yes. I think we would gate usage of this behind an IBP bump. Does that
>> seem
>> reasonable?
>>
>> > As for the improvements -- can you clarify how the multiple
>> transactional
>> IDs would help here? Were you thinking of a case where we wait/batch
>> multiple produce requests together? My understanding for now was 1
>> transactional ID and one validation per 1 produce request.
>>
>> Each call to `AddPartitionsToTxn` is essentially a write to the
>> transaction
>> log and must block on replication. The more we can fit into a single
>> request, the more writes we can do in parallel. The alternative is to make
>> use of more connections, but usually we prefer batching since the network
>> stack is not really optimized for high connection/request loads.
>>
>> > Finally with respect to the authorizations, I think it makes sense to
>> skip
>> topic authorizations, but I'm a bit confused by the "leader ID" field.
>> Wouldn't we just want to flag the request as from a broker (does it matter
>> which one?).
>>
>> We could also make it version-based. For the next version, we could
>> require
>> CLUSTER auth. So clients would not be able to use the API anymore, which
>> is
>> probably what we want.
>>
>> -Jason
>>
>> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
>> <jo...@confluent.io.invalid>
>> wrote:
>>
>> > As a follow up, I was just thinking about the batching a bit more.
>> > I suppose if we have one request in flight and we queue up the other
>> > produce requests in some sort of purgatory, we could send information
>> out
>> > for all of them rather than one by one. So that would be a benefit of
>> > batching partitions to add per transaction.
>> >
>> > I'll need to think a bit more on the design of this part of the KIP, and
>> > will update the KIP in the next few days.
>> >
>> > Thanks,
>> > Justine
>> >
>> > On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan <jo...@confluent.io>
>> > wrote:
>> >
>> > > Hey Jason -- thanks for the input -- I was just digging a bit deeper
>> into
>> > > the design + implementation of the validation calls here and what you
>> say
>> > > makes sense.
>> > >
>> > > I was wondering about compatibility here. When we send requests
>> > > between brokers, we want to ensure that the receiving broker
>> understands
>> > > the request (specifically the new fields). Typically this is done via
>> > > IBP/metadata version.
>> > > I'm trying to think if there is a way around it but I'm not sure there
>> > is.
>> > >
>> > > As for the improvements -- can you clarify how the multiple
>> transactional
>> > > IDs would help here? Were you thinking of a case where we wait/batch
>> > > multiple produce requests together? My understanding for now was 1
>> > > transactional ID and one validation per 1 produce request.
>> > >
>> > > Finally with respect to the authorizations, I think it makes sense to
>> > skip
>> > > topic authorizations, but I'm a bit confused by the "leader ID" field.
>> > > Wouldn't we just want to flag the request as from a broker (does it
>> > matter
>> > > which one?).
>> > >
>> > > I think I want to adopt these suggestions, just had a few questions on
>> > the
>> > > details.
>> > >
>> > > Thanks,
>> > > Justine
>> > >
>> > > On Thu, Jan 5, 2023 at 5:05 PM Jason Gustafson
>> > <ja...@confluent.io.invalid>
>> > > wrote:
>> > >
>> > >> Hi Justine,
>> > >>
>> > >> Thanks for the proposal.
>> > >>
>> > >> I was thinking about the implementation a little bit. In the current
>> > >> proposal, the behavior depends on whether we have an old or new
>> client.
>> > >> For
>> > >> old clients, we send `DescribeTransactions` and verify the result and
>> > for
>> > >> new clients, we send `AddPartitionsToTxn`. We might be able to
>> simplify
>> > >> the
>> > >> implementation if we can use the same request type. For example,
>> what if
>> > >> we
>> > >> bump the protocol version for `AddPartitionsToTxn` and add a
>> > >> `validateOnly`
>> > >> flag? For older versions, we can set `validateOnly=true` so that the
>> > >> request only returns successfully if the partition had already been
>> > added.
>> > >> For new versions, we can set `validateOnly=false` and the partition
>> will
>> > >> be
>> > >> added to the transaction. The other slightly annoying thing that this
>> > >> would
>> > >> get around is the need to collect the transaction state for all
>> > partitions
>> > >> even when we only care about a subset.
>> > >>
>> > >> Some additional improvements to consider:
>> > >>
>> > >> - We can give `AddPartitionsToTxn` better batch support for
>> inter-broker
>> > >> usage. Currently we only allow one `TransactionalId` to be specified,
>> > but
>> > >> the broker may get some benefit being able to batch across multiple
>> > >> transactions.
>> > >> - Another small improvement is skipping topic authorization checks
>> for
>> > >> `AddPartitionsToTxn` when the request is from a broker. Perhaps we
>> can
>> > add
>> > >> a field for the `LeaderId` or something like that and require CLUSTER
>> > >> permission when set.
>> > >>
>> > >> Best,
>> > >> Jason
>> > >>
>> > >>
>> > >>
>> > >> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao <ju...@confluent.io.invalid>
>> > wrote:
>> > >>
>> > >> > Hi, Justine,
>> > >> >
>> > >> > Thanks for the explanation. It makes sense to me now.
>> > >> >
>> > >> > Jun
>> > >> >
>> > >> > On Mon, Dec 19, 2022 at 1:42 PM Justine Olshan
>> > >> > <jo...@confluent.io.invalid>
>> > >> > wrote:
>> > >> >
>> > >> > > Hi Jun,
>> > >> > >
>> > >> > > My understanding of the mechanism is that when we get to the last
>> > >> epoch,
>> > >> > we
>> > >> > > increment to the fencing/last epoch and if any further requests
>> come
>> > >> in
>> > >> > for
>> > >> > > this producer ID they are fenced. Then the producer gets a new ID
>> > and
>> > >> > > restarts with epoch/sequence 0. The fenced epoch sticks around
>> for
>> > the
>> > >> > > duration of producer.id.expiration.ms and blocks any late
>> messages
>> > >> > there.
>> > >> > > The new ID will get to take advantage of the improved semantics
>> > around
>> > >> > > non-zero start sequences. So I think we are covered.
>> > >> > >
>> > >> > > The only potential issue is overloading the cache, but hopefully
>> the
>> > >> > > improvements (lowered producer.id.expiration.ms) will help with
>> > that.
>> > >> > Let
>> > >> > > me know if you still have concerns.
>> > >> > >
>> > >> > > Thanks,
>> > >> > > Justine
>> > >> > >
>> > >> > > On Mon, Dec 19, 2022 at 10:24 AM Jun Rao
>> <ju...@confluent.io.invalid>
>> > >> > wrote:
>> > >> > >
>> > >> > > > Hi, Justine,
>> > >> > > >
>> > >> > > > Thanks for the explanation.
>> > >> > > >
>> > >> > > > 70. The proposed fencing logic doesn't apply when pid changes,
>> is
>> > >> that
>> > >> > > > right? If so, I am not sure how complete we are addressing this
>> > >> issue
>> > >> > if
>> > >> > > > the pid changes more frequently.
>> > >> > > >
>> > >> > > > Thanks,
>> > >> > > >
>> > >> > > > Jun
>> > >> > > >
>> > >> > > >
>> > >> > > >
>> > >> > > > On Fri, Dec 16, 2022 at 9:16 AM Justine Olshan
>> > >> > > > <jo...@confluent.io.invalid>
>> > >> > > > wrote:
>> > >> > > >
>> > >> > > > > Hi Jun,
>> > >> > > > >
>> > >> > > > > Thanks for replying!
>> > >> > > > >
>> > >> > > > > 70.We already do the overflow mechanism, so my change would
>> just
>> > >> make
>> > >> > > it
>> > >> > > > > happen more often.
>> > >> > > > > I was also not suggesting a new field in the log, but in the
>> > >> > response,
>> > >> > > > > which would be gated by the client version. Sorry if
>> something
>> > >> there
>> > >> > is
>> > >> > > > > unclear. I think we are starting to diverge.
>> > >> > > > > The goal of this KIP is to not change to the marker format at
>> > all.
>> > >> > > > >
>> > >> > > > > 71. Yes, I guess I was going under the assumption that the
>> log
>> > >> would
>> > >> > > just
>> > >> > > > > look at its last epoch and treat it as the current epoch. I
>> > >> suppose
>> > >> > we
>> > >> > > > can
>> > >> > > > > have some special logic that if the last epoch was on a
>> marker
>> > we
>> > >> > > > actually
>> > >> > > > > expect the next epoch or something like that. We just need to
>> > >> > > distinguish
>> > >> > > > > based on whether we had a commit/abort marker.
>> > >> > > > >
>> > >> > > > > 72.
>> > >> > > > > > if the producer epoch hasn't been bumped on the
>> > >> > > > > broker, it seems that the stucked message will fail the
>> sequence
>> > >> > > > validation
>> > >> > > > > and will be ignored. If the producer epoch has been bumped,
>> we
>> > >> ignore
>> > >> > > the
>> > >> > > > > sequence check and the stuck message could be appended to the
>> > log.
>> > >> > So,
>> > >> > > is
>> > >> > > > > the latter case that we want to guard?
>> > >> > > > >
>> > >> > > > > I'm not sure I follow that "the message will fail the
>> sequence
>> > >> > > > validation".
>> > >> > > > > In some of these cases, we had an abort marker (due to an
>> error)
>> > >> and
>> > >> > > then
>> > >> > > > > the late message comes in with the correct sequence number.
>> This
>> > >> is a
>> > >> > > > case
>> > >> > > > > covered by the KIP.
>> > >> > > > > The latter case is actually not something we've considered
>> > here. I
>> > >> > > think
>> > >> > > > > generally when we bump the epoch, we are accepting that the
>> > >> sequence
>> > >> > > does
>> > >> > > > > not need to be checked anymore. My understanding is also
>> that we
>> > >> > don't
>> > >> > > > > typically bump epoch mid transaction (based on a quick look
>> at
>> > the
>> > >> > > code)
>> > >> > > > > but let me know if that is the case.
>> > >> > > > >
>> > >> > > > > Thanks,
>> > >> > > > > Justine
>> > >> > > > >
>> > >> > > > > On Thu, Dec 15, 2022 at 12:23 PM Jun Rao
>> > <jun@confluent.io.invalid
>> > >> >
>> > >> > > > wrote:
>> > >> > > > >
>> > >> > > > > > Hi, Justine,
>> > >> > > > > >
>> > >> > > > > > Thanks for the reply.
>> > >> > > > > >
>> > >> > > > > > 70. Assigning a new pid on int overflow seems a bit hacky.
>> If
>> > we
>> > >> > > need a
>> > >> > > > > txn
>> > >> > > > > > level id, it will be better to model this explicitly.
>> Adding a
>> > >> new
>> > >> > > > field
>> > >> > > > > > would require a bit more work since it requires a new txn
>> > marker
>> > >> > > format
>> > >> > > > > in
>> > >> > > > > > the log. So, we probably need to guard it with an IBP or
>> > >> metadata
>> > >> > > > version
>> > >> > > > > > and document the impact on downgrade once the new format is
>> > >> written
>> > >> > > to
>> > >> > > > > the
>> > >> > > > > > log.
>> > >> > > > > >
>> > >> > > > > > 71. Hmm, once the marker is written, the partition will
>> expect
>> > >> the
>> > >> > > next
>> > >> > > > > > append to be on the next epoch. Does that cover the case
>> you
>> > >> > > mentioned?
>> > >> > > > > >
>> > >> > > > > > 72. Also, just to be clear on the stucked message issue
>> > >> described
>> > >> > in
>> > >> > > > the
>> > >> > > > > > motivation. With EoS, we also validate the sequence id for
>> > >> > > idempotency.
>> > >> > > > > So,
>> > >> > > > > > with the current logic, if the producer epoch hasn't been
>> > >> bumped on
>> > >> > > the
>> > >> > > > > > broker, it seems that the stucked message will fail the
>> > sequence
>> > >> > > > > validation
>> > >> > > > > > and will be ignored. If the producer epoch has been
>> bumped, we
>> > >> > ignore
>> > >> > > > the
>> > >> > > > > > sequence check and the stuck message could be appended to
>> the
>> > >> log.
>> > >> > > So,
>> > >> > > > is
>> > >> > > > > > the latter case that we want to guard?
>> > >> > > > > >
>> > >> > > > > > Thanks,
>> > >> > > > > >
>> > >> > > > > > Jun
>> > >> > > > > >
>> > >> > > > > > On Wed, Dec 14, 2022 at 10:44 AM Justine Olshan
>> > >> > > > > > <jo...@confluent.io.invalid> wrote:
>> > >> > > > > >
>> > >> > > > > > > Matthias — thanks again for taking time to look a this.
>> You
>> > >> said:
>> > >> > > > > > >
>> > >> > > > > > > > My proposal was only focusing to avoid dangling
>> > >> > > > > > >
>> > >> > > > > > > transactions if records are added without registered
>> > >> partition.
>> > >> > --
>> > >> > > > > Maybe
>> > >> > > > > > >
>> > >> > > > > > > you can add a few more details to the KIP about this
>> > scenario
>> > >> for
>> > >> > > > > better
>> > >> > > > > > >
>> > >> > > > > > > documentation purpose?
>> > >> > > > > > >
>> > >> > > > > > >
>> > >> > > > > > > I'm not sure I understand what you mean here. The
>> motivation
>> > >> > > section
>> > >> > > > > > > describes two scenarios about how the record can be added
>> > >> > without a
>> > >> > > > > > > registered partition:
>> > >> > > > > > >
>> > >> > > > > > >
>> > >> > > > > > > > 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.
>> > >> > > > > > >
>> > >> > > > > > >
>> > >> > > > > > > > Another way hanging transactions can occur is that a
>> > client
>> > >> is
>> > >> > > > buggy
>> > >> > > > > > and
>> > >> > > > > > > may somehow try to write to a partition before it adds
>> the
>> > >> > > partition
>> > >> > > > to
>> > >> > > > > > the
>> > >> > > > > > > transaction.
>> > >> > > > > > >
>> > >> > > > > > >
>> > >> > > > > > >
>> > >> > > > > > > For the first example of this would it be helpful to say
>> > that
>> > >> > this
>> > >> > > > > > message
>> > >> > > > > > > comes in after the abort, but before the partition is
>> added
>> > to
>> > >> > the
>> > >> > > > next
>> > >> > > > > > > transaction so it becomes "hanging." Perhaps the next
>> > sentence
>> > >> > > > > describing
>> > >> > > > > > > the message becoming part of the next transaction (a
>> > different
>> > >> > > case)
>> > >> > > > > was
>> > >> > > > > > > not properly differentiated.
>> > >> > > > > > >
>> > >> > > > > > >
>> > >> > > > > > >
>> > >> > > > > > > Jun — thanks for reading the KIP.
>> > >> > > > > > >
>> > >> > > > > > > 70. The int typing was a concern. Currently we have a
>> > >> mechanism
>> > >> > in
>> > >> > > > > place
>> > >> > > > > > to
>> > >> > > > > > > fence the final epoch when the epoch is about to overflow
>> > and
>> > >> > > assign
>> > >> > > > a
>> > >> > > > > > new
>> > >> > > > > > > producer ID with epoch 0. Of course, this is a bit tricky
>> > >> when it
>> > >> > > > comes
>> > >> > > > > > to
>> > >> > > > > > > the response back to the client.
>> > >> > > > > > > Making this a long could be another option, but I wonder
>> are
>> > >> > there
>> > >> > > > any
>> > >> > > > > > > implications on changing this field if the epoch is
>> > persisted
>> > >> to
>> > >> > > > disk?
>> > >> > > > > > I'd
>> > >> > > > > > > need to check the usages.
>> > >> > > > > > >
>> > >> > > > > > > 71.This was something Matthias asked about as well. I was
>> > >> > > > considering a
>> > >> > > > > > > possible edge case where a produce request from a new
>> > >> transaction
>> > >> > > > > somehow
>> > >> > > > > > > gets sent right after the marker is written, but before
>> the
>> > >> > > producer
>> > >> > > > is
>> > >> > > > > > > alerted of the newly bumped epoch. In this case, we may
>> > >> include
>> > >> > > this
>> > >> > > > > > record
>> > >> > > > > > > when we don't want to. I suppose we could try to do
>> > something
>> > >> > > client
>> > >> > > > > side
>> > >> > > > > > > to bump the epoch after sending an endTxn as well in this
>> > >> > scenario
>> > >> > > —
>> > >> > > > > but
>> > >> > > > > > I
>> > >> > > > > > > wonder how it would work when the server is aborting
>> based
>> > on
>> > >> a
>> > >> > > > > > server-side
>> > >> > > > > > > error. I could also be missing something and this
>> scenario
>> > is
>> > >> > > > actually
>> > >> > > > > > not
>> > >> > > > > > > possible.
>> > >> > > > > > >
>> > >> > > > > > > Thanks again to everyone reading and commenting. Let me
>> know
>> > >> > about
>> > >> > > > any
>> > >> > > > > > > further questions or comments.
>> > >> > > > > > >
>> > >> > > > > > > Justine
>> > >> > > > > > >
>> > >> > > > > > > On Wed, Dec 14, 2022 at 9:41 AM Jun Rao
>> > >> <jun@confluent.io.invalid
>> > >> > >
>> > >> > > > > > wrote:
>> > >> > > > > > >
>> > >> > > > > > > > Hi, Justine,
>> > >> > > > > > > >
>> > >> > > > > > > > Thanks for the KIP. A couple of comments.
>> > >> > > > > > > >
>> > >> > > > > > > > 70. Currently, the producer epoch is an int. I am not
>> sure
>> > >> if
>> > >> > > it's
>> > >> > > > > > enough
>> > >> > > > > > > > to accommodate all transactions in the lifetime of a
>> > >> producer.
>> > >> > > > Should
>> > >> > > > > > we
>> > >> > > > > > > > change that to a long or add a new long field like
>> txnId?
>> > >> > > > > > > >
>> > >> > > > > > > > 71. "it will write the prepare commit message with a
>> > bumped
>> > >> > epoch
>> > >> > > > and
>> > >> > > > > > > send
>> > >> > > > > > > > WriteTxnMarkerRequests with the bumped epoch." Hmm, the
>> > >> epoch
>> > >> > is
>> > >> > > > > > > associated
>> > >> > > > > > > > with the current txn right? So, it seems weird to
>> write a
>> > >> > commit
>> > >> > > > > > message
>> > >> > > > > > > > with a bumped epoch. Should we only bump up the epoch
>> in
>> > >> > > > > EndTxnResponse
>> > >> > > > > > > and
>> > >> > > > > > > > rename the field to sth like nextProducerEpoch?
>> > >> > > > > > > >
>> > >> > > > > > > > Thanks,
>> > >> > > > > > > >
>> > >> > > > > > > > Jun
>> > >> > > > > > > >
>> > >> > > > > > > >
>> > >> > > > > > > >
>> > >> > > > > > > > On Mon, Dec 12, 2022 at 8:54 PM Matthias J. Sax <
>> > >> > > mjsax@apache.org>
>> > >> > > > > > > wrote:
>> > >> > > > > > > >
>> > >> > > > > > > > > Thanks for the background.
>> > >> > > > > > > > >
>> > >> > > > > > > > > 20/30: SGTM. My proposal was only focusing to avoid
>> > >> dangling
>> > >> > > > > > > > > transactions if records are added without registered
>> > >> > partition.
>> > >> > > > --
>> > >> > > > > > > Maybe
>> > >> > > > > > > > > you can add a few more details to the KIP about this
>> > >> scenario
>> > >> > > for
>> > >> > > > > > > better
>> > >> > > > > > > > > documentation purpose?
>> > >> > > > > > > > >
>> > >> > > > > > > > > 40: I think you hit a fair point about race
>> conditions
>> > or
>> > >> > > client
>> > >> > > > > bugs
>> > >> > > > > > > > > (incorrectly not bumping the epoch). The
>> > >> complexity/confusion
>> > >> > > for
>> > >> > > > > > using
>> > >> > > > > > > > > the bumped epoch I see, is mainly for internal
>> > debugging,
>> > >> ie,
>> > >> > > > > > > inspecting
>> > >> > > > > > > > > log segment dumps -- it seems harder to reason about
>> the
>> > >> > system
>> > >> > > > for
>> > >> > > > > > us
>> > >> > > > > > > > > humans. But if we get better guarantees, it would be
>> > >> worth to
>> > >> > > use
>> > >> > > > > the
>> > >> > > > > > > > > bumped epoch.
>> > >> > > > > > > > >
>> > >> > > > > > > > > 60: as I mentioned already, I don't know the broker
>> > >> internals
>> > >> > > to
>> > >> > > > > > > provide
>> > >> > > > > > > > > more input. So if nobody else chimes in, we should
>> just
>> > >> move
>> > >> > > > > forward
>> > >> > > > > > > > > with your proposal.
>> > >> > > > > > > > >
>> > >> > > > > > > > >
>> > >> > > > > > > > > -Matthias
>> > >> > > > > > > > >
>> > >> > > > > > > > >
>> > >> > > > > > > > > On 12/6/22 4:22 PM, Justine Olshan wrote:
>> > >> > > > > > > > > > Hi all,
>> > >> > > > > > > > > > After Artem's questions about error behavior, I've
>> > >> > > re-evaluated
>> > >> > > > > the
>> > >> > > > > > > > > > unknown producer ID exception and had some
>> discussions
>> > >> > > offline.
>> > >> > > > > > > > > >
>> > >> > > > > > > > > > I think generally it makes sense to simplify error
>> > >> handling
>> > >> > > in
>> > >> > > > > > cases
>> > >> > > > > > > > like
>> > >> > > > > > > > > > this and the UNKNOWN_PRODUCER_ID error has a pretty
>> > long
>> > >> > and
>> > >> > > > > > > > complicated
>> > >> > > > > > > > > > history. Because of this, I propose adding a new
>> error
>> > >> code
>> > >> > > > > > > > > ABORTABLE_ERROR
>> > >> > > > > > > > > > that when encountered by new clients (gated by the
>> > >> produce
>> > >> > > > > request
>> > >> > > > > > > > > version)
>> > >> > > > > > > > > > will simply abort the transaction. This allows the
>> > >> server
>> > >> > to
>> > >> > > > have
>> > >> > > > > > > some
>> > >> > > > > > > > > say
>> > >> > > > > > > > > > in whether the client aborts and makes handling
>> much
>> > >> > simpler.
>> > >> > > > In
>> > >> > > > > > the
>> > >> > > > > > > > > > future, we can also use this error in other
>> situations
>> > >> > where
>> > >> > > we
>> > >> > > > > > want
>> > >> > > > > > > to
>> > >> > > > > > > > > > abort the transactions. We can even use on other
>> apis.
>> > >> > > > > > > > > >
>> > >> > > > > > > > > > I've added this to the KIP. Let me know if there
>> are
>> > any
>> > >> > > > > questions
>> > >> > > > > > or
>> > >> > > > > > > > > > issues.
>> > >> > > > > > > > > >
>> > >> > > > > > > > > > Justine
>> > >> > > > > > > > > >
>> > >> > > > > > > > > > On Fri, Dec 2, 2022 at 10:22 AM Justine Olshan <
>> > >> > > > > > jolshan@confluent.io
>> > >> > > > > > > >
>> > >> > > > > > > > > wrote:
>> > >> > > > > > > > > >
>> > >> > > > > > > > > >> Hey Matthias,
>> > >> > > > > > > > > >>
>> > >> > > > > > > > > >>
>> > >> > > > > > > > > >> 20/30 — Maybe I also didn't express myself
>> clearly.
>> > For
>> > >> > > older
>> > >> > > > > > > clients
>> > >> > > > > > > > we
>> > >> > > > > > > > > >> don't have a way to distinguish between a previous
>> > and
>> > >> the
>> > >> > > > > current
>> > >> > > > > > > > > >> transaction since we don't have the epoch bump.
>> This
>> > >> means
>> > >> > > > that
>> > >> > > > > a
>> > >> > > > > > > late
>> > >> > > > > > > > > >> message from the previous transaction may be
>> added to
>> > >> the
>> > >> > > new
>> > >> > > > > one.
>> > >> > > > > > > > With
>> > >> > > > > > > > > >> older clients — we can't guarantee this won't
>> happen
>> > >> if we
>> > >> > > > > already
>> > >> > > > > > > > sent
>> > >> > > > > > > > > the
>> > >> > > > > > > > > >> addPartitionsToTxn call (why we make changes for
>> the
>> > >> newer
>> > >> > > > > client)
>> > >> > > > > > > but
>> > >> > > > > > > > > we
>> > >> > > > > > > > > >> can at least gate some by ensuring that the
>> partition
>> > >> has
>> > >> > > been
>> > >> > > > > > added
>> > >> > > > > > > > to
>> > >> > > > > > > > > the
>> > >> > > > > > > > > >> transaction. The rationale here is that there are
>> > >> likely
>> > >> > > LESS
>> > >> > > > > late
>> > >> > > > > > > > > arrivals
>> > >> > > > > > > > > >> as time goes on, so hopefully most late arrivals
>> will
>> > >> come
>> > >> > > in
>> > >> > > > > > BEFORE
>> > >> > > > > > > > the
>> > >> > > > > > > > > >> addPartitionsToTxn call. Those that arrive before
>> > will
>> > >> be
>> > >> > > > > properly
>> > >> > > > > > > > gated
>> > >> > > > > > > > > >> with the describeTransactions approach.
>> > >> > > > > > > > > >>
>> > >> > > > > > > > > >> If we take the approach you suggested, ANY late
>> > arrival
>> > >> > > from a
>> > >> > > > > > > > previous
>> > >> > > > > > > > > >> transaction will be added. And we don't want
>> that. I
>> > >> also
>> > >> > > > don't
>> > >> > > > > > see
>> > >> > > > > > > > any
>> > >> > > > > > > > > >> benefit in sending addPartitionsToTxn over the
>> > >> > describeTxns
>> > >> > > > > call.
>> > >> > > > > > > They
>> > >> > > > > > > > > will
>> > >> > > > > > > > > >> both be one extra RPC to the Txn coordinator.
>> > >> > > > > > > > > >>
>> > >> > > > > > > > > >>
>> > >> > > > > > > > > >> To be clear — newer clients will use
>> > addPartitionsToTxn
>> > >> > > > instead
>> > >> > > > > of
>> > >> > > > > > > the
>> > >> > > > > > > > > >> DescribeTxns.
>> > >> > > > > > > > > >>
>> > >> > > > > > > > > >>
>> > >> > > > > > > > > >> 40)
>> > >> > > > > > > > > >> My concern is that if we have some delay in the
>> > client
>> > >> to
>> > >> > > bump
>> > >> > > > > the
>> > >> > > > > > > > > epoch,
>> > >> > > > > > > > > >> it could continue to send epoch 73 and those
>> records
>> > >> would
>> > >> > > not
>> > >> > > > > be
>> > >> > > > > > > > > fenced.
>> > >> > > > > > > > > >> Perhaps this is not an issue if we don't allow the
>> > next
>> > >> > > > produce
>> > >> > > > > to
>> > >> > > > > > > go
>> > >> > > > > > > > > >> through before the EndTxn request returns. I'm
>> also
>> > >> > thinking
>> > >> > > > > about
>> > >> > > > > > > > > cases of
>> > >> > > > > > > > > >> failure. I will need to think on this a bit.
>> > >> > > > > > > > > >>
>> > >> > > > > > > > > >> I wasn't sure if it was that confusing. But if we
>> > >> think it
>> > >> > > is,
>> > >> > > > > we
>> > >> > > > > > > can
>> > >> > > > > > > > > >> investigate other ways.
>> > >> > > > > > > > > >>
>> > >> > > > > > > > > >>
>> > >> > > > > > > > > >> 60)
>> > >> > > > > > > > > >>
>> > >> > > > > > > > > >> I'm not sure these are the same purgatories since
>> one
>> > >> is a
>> > >> > > > > produce
>> > >> > > > > > > > > >> purgatory (I was planning on using a callback
>> rather
>> > >> than
>> > >> > > > > > purgatory)
>> > >> > > > > > > > and
>> > >> > > > > > > > > >> the other is simply a request to append to the
>> log.
>> > Not
>> > >> > sure
>> > >> > > > we
>> > >> > > > > > have
>> > >> > > > > > > > any
>> > >> > > > > > > > > >> structure here for ordering, but my understanding
>> is
>> > >> that
>> > >> > > the
>> > >> > > > > > broker
>> > >> > > > > > > > > could
>> > >> > > > > > > > > >> handle the write request before it hears back from
>> > the
>> > >> Txn
>> > >> > > > > > > > Coordinator.
>> > >> > > > > > > > > >>
>> > >> > > > > > > > > >> Let me know if I misunderstood something or
>> something
>> > >> was
>> > >> > > > > unclear.
>> > >> > > > > > > > > >>
>> > >> > > > > > > > > >> Justine
>> > >> > > > > > > > > >>
>> > >> > > > > > > > > >> On Thu, Dec 1, 2022 at 12:15 PM Matthias J. Sax <
>> > >> > > > > mjsax@apache.org
>> > >> > > > > > >
>> > >> > > > > > > > > wrote:
>> > >> > > > > > > > > >>
>> > >> > > > > > > > > >>> Thanks for the details Justine!
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > > >>>> 20)
>> > >> > > > > > > > > >>>>
>> > >> > > > > > > > > >>>> The client side change for 2 is removing the
>> > >> > addPartitions
>> > >> > > > to
>> > >> > > > > > > > > >>> transaction
>> > >> > > > > > > > > >>>> call. We don't need to make this from the
>> producer
>> > to
>> > >> > the
>> > >> > > > txn
>> > >> > > > > > > > > >>> coordinator,
>> > >> > > > > > > > > >>>> only server side.
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > > >>> I think I did not express myself clearly. I
>> > understand
>> > >> > that
>> > >> > > > we
>> > >> > > > > > can
>> > >> > > > > > > > (and
>> > >> > > > > > > > > >>> should) change the producer to not send the
>> > >> > `addPartitions`
>> > >> > > > > > request
>> > >> > > > > > > > any
>> > >> > > > > > > > > >>> longer. But I don't thinks it's requirement to
>> > change
>> > >> the
>> > >> > > > > broker?
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > > >>> What I am trying to say is: as a safe-guard and
>> > >> > improvement
>> > >> > > > for
>> > >> > > > > > > older
>> > >> > > > > > > > > >>> producers, the partition leader can just send the
>> > >> > > > > `addPartitions`
>> > >> > > > > > > > > >>> request to the TX-coordinator in any case -- if
>> the
>> > >> old
>> > >> > > > > producer
>> > >> > > > > > > > > >>> correctly did send the `addPartition` request to
>> the
>> > >> > > > > > TX-coordinator
>> > >> > > > > > > > > >>> already, the TX-coordinator can just "ignore" is
>> as
>> > >> > > > idempotent.
>> > >> > > > > > > > > However,
>> > >> > > > > > > > > >>> if the old producer has a bug and did forget to
>> sent
>> > >> the
>> > >> > > > > > > > `addPartition`
>> > >> > > > > > > > > >>> request, we would now ensure that the partition
>> is
>> > >> indeed
>> > >> > > > added
>> > >> > > > > > to
>> > >> > > > > > > > the
>> > >> > > > > > > > > >>> TX and thus fix a potential producer bug (even
>> if we
>> > >> > don't
>> > >> > > > get
>> > >> > > > > > the
>> > >> > > > > > > > > >>> fencing via the bump epoch). -- It seems to be a
>> > good
>> > >> > > > > > improvement?
>> > >> > > > > > > Or
>> > >> > > > > > > > > is
>> > >> > > > > > > > > >>> there a reason to not do this?
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > > >>>> 30)
>> > >> > > > > > > > > >>>>
>> > >> > > > > > > > > >>>> Transaction is ongoing = partition was added to
>> > >> > > transaction
>> > >> > > > > via
>> > >> > > > > > > > > >>>> addPartitionsToTxn. We check this with the
>> > >> > > > > DescribeTransactions
>> > >> > > > > > > > call.
>> > >> > > > > > > > > >>> Let
>> > >> > > > > > > > > >>>> me know if this wasn't sufficiently explained
>> here:
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > > >>> If we do what I propose in (20), we don't really
>> > need
>> > >> to
>> > >> > > make
>> > >> > > > > > this
>> > >> > > > > > > > > >>> `DescribeTransaction` call, as the partition
>> leader
>> > >> adds
>> > >> > > the
>> > >> > > > > > > > partition
>> > >> > > > > > > > > >>> for older clients and we get this check for free.
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > > >>>> 40)
>> > >> > > > > > > > > >>>>
>> > >> > > > > > > > > >>>> The idea here is that if any messages somehow
>> come
>> > in
>> > >> > > before
>> > >> > > > > we
>> > >> > > > > > > get
>> > >> > > > > > > > > the
>> > >> > > > > > > > > >>> new
>> > >> > > > > > > > > >>>> epoch to the producer, they will be fenced.
>> > However,
>> > >> if
>> > >> > we
>> > >> > > > > don't
>> > >> > > > > > > > think
>> > >> > > > > > > > > >>> this
>> > >> > > > > > > > > >>>> is necessary, it can be discussed
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > > >>> I agree that we should have epoch fencing. My
>> > >> question is
>> > >> > > > > > > different:
>> > >> > > > > > > > > >>> Assume we are at epoch 73, and we have an ongoing
>> > >> > > > transaction,
>> > >> > > > > > that
>> > >> > > > > > > > is
>> > >> > > > > > > > > >>> committed. It seems natural to write the "prepare
>> > >> commit"
>> > >> > > > > marker
>> > >> > > > > > > and
>> > >> > > > > > > > > the
>> > >> > > > > > > > > >>> `WriteTxMarkerRequest` both with epoch 73, too,
>> as
>> > it
>> > >> > > belongs
>> > >> > > > > to
>> > >> > > > > > > the
>> > >> > > > > > > > > >>> current transaction. Of course, we now also bump
>> the
>> > >> > epoch
>> > >> > > > and
>> > >> > > > > > > expect
>> > >> > > > > > > > > >>> the next requests to have epoch 74, and would
>> reject
>> > >> an
>> > >> > > > request
>> > >> > > > > > > with
>> > >> > > > > > > > > >>> epoch 73, as the corresponding TX for epoch 73
>> was
>> > >> > already
>> > >> > > > > > > committed.
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > > >>> It seems you propose to write the "prepare commit
>> > >> marker"
>> > >> > > and
>> > >> > > > > > > > > >>> `WriteTxMarkerRequest` with epoch 74 though, what
>> > >> would
>> > >> > > work,
>> > >> > > > > but
>> > >> > > > > > > it
>> > >> > > > > > > > > >>> seems confusing. Is there a reason why we would
>> use
>> > >> the
>> > >> > > > bumped
>> > >> > > > > > > epoch
>> > >> > > > > > > > 74
>> > >> > > > > > > > > >>> instead of the current epoch 73?
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > > >>>> 60)
>> > >> > > > > > > > > >>>>
>> > >> > > > > > > > > >>>> When we are checking if the transaction is
>> ongoing,
>> > >> we
>> > >> > > need
>> > >> > > > to
>> > >> > > > > > > make
>> > >> > > > > > > > a
>> > >> > > > > > > > > >>> round
>> > >> > > > > > > > > >>>> trip from the leader partition to the
>> transaction
>> > >> > > > coordinator.
>> > >> > > > > > In
>> > >> > > > > > > > the
>> > >> > > > > > > > > >>> time
>> > >> > > > > > > > > >>>> we are waiting for this message to come back, in
>> > >> theory
>> > >> > we
>> > >> > > > > could
>> > >> > > > > > > > have
>> > >> > > > > > > > > >>> sent
>> > >> > > > > > > > > >>>> a commit/abort call that would make the original
>> > >> result
>> > >> > of
>> > >> > > > the
>> > >> > > > > > > check
>> > >> > > > > > > > > >>> out of
>> > >> > > > > > > > > >>>> date. That is why we can check the leader state
>> > >> before
>> > >> > we
>> > >> > > > > write
>> > >> > > > > > to
>> > >> > > > > > > > the
>> > >> > > > > > > > > >>> log.
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > > >>> Thanks. Got it.
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > > >>> However, is this really an issue? We put the
>> produce
>> > >> > > request
>> > >> > > > in
>> > >> > > > > > > > > >>> purgatory, so how could we process the
>> > >> > > > `WriteTxnMarkerRequest`
>> > >> > > > > > > first?
>> > >> > > > > > > > > >>> Don't we need to put the `WriteTxnMarkerRequest`
>> > into
>> > >> > > > > purgatory,
>> > >> > > > > > > too,
>> > >> > > > > > > > > >>> for this case, and process both request in-order?
>> > >> (Again,
>> > >> > > my
>> > >> > > > > > broker
>> > >> > > > > > > > > >>> knowledge is limited and maybe we don't maintain
>> > >> request
>> > >> > > > order
>> > >> > > > > > for
>> > >> > > > > > > > this
>> > >> > > > > > > > > >>> case, what seems to be an issue IMHO, and I am
>> > >> wondering
>> > >> > if
>> > >> > > > > > > changing
>> > >> > > > > > > > > >>> request handling to preserve order for this case
>> > >> might be
>> > >> > > the
>> > >> > > > > > > cleaner
>> > >> > > > > > > > > >>> solution?)
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > > >>> -Matthias
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > > >>> On 11/30/22 3:28 PM, Artem Livshits wrote:
>> > >> > > > > > > > > >>>> Hi Justine,
>> > >> > > > > > > > > >>>>
>> > >> > > > > > > > > >>>> I think the interesting part is not in this
>> logic
>> > >> > (because
>> > >> > > > it
>> > >> > > > > > > tries
>> > >> > > > > > > > to
>> > >> > > > > > > > > >>>> figure out when UNKNOWN_PRODUCER_ID is retriable
>> > and
>> > >> if
>> > >> > > it's
>> > >> > > > > > > > > retryable,
>> > >> > > > > > > > > >>>> it's definitely not fatal), but what happens
>> when
>> > >> this
>> > >> > > logic
>> > >> > > > > > > doesn't
>> > >> > > > > > > > > >>> return
>> > >> > > > > > > > > >>>> 'true' and falls through.  In the old clients it
>> > >> seems
>> > >> > to
>> > >> > > be
>> > >> > > > > > > fatal,
>> > >> > > > > > > > if
>> > >> > > > > > > > > >>> we
>> > >> > > > > > > > > >>>> keep the behavior in the new clients, I'd
>> expect it
>> > >> > would
>> > >> > > be
>> > >> > > > > > fatal
>> > >> > > > > > > > as
>> > >> > > > > > > > > >>> well.
>> > >> > > > > > > > > >>>>
>> > >> > > > > > > > > >>>> -Artem
>> > >> > > > > > > > > >>>>
>> > >> > > > > > > > > >>>> On Tue, Nov 29, 2022 at 11:57 AM Justine Olshan
>> > >> > > > > > > > > >>>> <jo...@confluent.io.invalid> wrote:
>> > >> > > > > > > > > >>>>
>> > >> > > > > > > > > >>>>> Hi Artem and Jeff,
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>> Thanks for taking a look and sorry for the slow
>> > >> > response.
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>> You both mentioned the change to handle
>> > >> > > UNKNOWN_PRODUCER_ID
>> > >> > > > > > > errors.
>> > >> > > > > > > > > To
>> > >> > > > > > > > > >>> be
>> > >> > > > > > > > > >>>>> clear — this error code will only be sent again
>> > when
>> > >> > the
>> > >> > > > > > client's
>> > >> > > > > > > > > >>> request
>> > >> > > > > > > > > >>>>> version is high enough to ensure we handle it
>> > >> > correctly.
>> > >> > > > > > > > > >>>>> The current (Java) client handles this by the
>> > >> following
>> > >> > > > > > (somewhat
>> > >> > > > > > > > > long)
>> > >> > > > > > > > > >>>>> code snippet:
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>> // An UNKNOWN_PRODUCER_ID means that we have
>> lost
>> > >> the
>> > >> > > > > producer
>> > >> > > > > > > > state
>> > >> > > > > > > > > >>> on the
>> > >> > > > > > > > > >>>>> broker. Depending on the log start
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>> // offset, we may want to retry these, as
>> > described
>> > >> for
>> > >> > > > each
>> > >> > > > > > case
>> > >> > > > > > > > > >>> below. If
>> > >> > > > > > > > > >>>>> none of those apply, then for the
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>> // idempotent producer, we will locally bump
>> the
>> > >> epoch
>> > >> > > and
>> > >> > > > > > reset
>> > >> > > > > > > > the
>> > >> > > > > > > > > >>>>> sequence numbers of in-flight batches from
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>> // sequence 0, then retry the failed batch,
>> which
>> > >> > should
>> > >> > > > now
>> > >> > > > > > > > succeed.
>> > >> > > > > > > > > >>> For
>> > >> > > > > > > > > >>>>> the transactional producer, allow the
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>> // batch to fail. When processing the failed
>> > batch,
>> > >> we
>> > >> > > will
>> > >> > > > > > > > > transition
>> > >> > > > > > > > > >>> to
>> > >> > > > > > > > > >>>>> an abortable error and set a flag
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>> // indicating that we need to bump the epoch
>> (if
>> > >> > > supported
>> > >> > > > by
>> > >> > > > > > the
>> > >> > > > > > > > > >>> broker).
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>> if (error == Errors.*UNKNOWN_PRODUCER_ID*) {
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>       if (response.logStartOffset == -1) {
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           // We don't know the log start offset
>> > with
>> > >> > this
>> > >> > > > > > > response.
>> > >> > > > > > > > > We
>> > >> > > > > > > > > >>> should
>> > >> > > > > > > > > >>>>> just retry the request until we get it.
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           // The UNKNOWN_PRODUCER_ID error code
>> > was
>> > >> > added
>> > >> > > > > along
>> > >> > > > > > > > with
>> > >> > > > > > > > > >>> the new
>> > >> > > > > > > > > >>>>> ProduceResponse which includes the
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           // logStartOffset. So the '-1'
>> sentinel
>> > is
>> > >> > not
>> > >> > > > for
>> > >> > > > > > > > backward
>> > >> > > > > > > > > >>>>> compatibility. Instead, it is possible for
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           // a broker to not know the
>> > >> logStartOffset at
>> > >> > > > when
>> > >> > > > > it
>> > >> > > > > > > is
>> > >> > > > > > > > > >>> returning
>> > >> > > > > > > > > >>>>> the response because the partition
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           // may have moved away from the
>> broker
>> > >> from
>> > >> > the
>> > >> > > > > time
>> > >> > > > > > > the
>> > >> > > > > > > > > >>> error was
>> > >> > > > > > > > > >>>>> initially raised to the time the
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           // response was being constructed. In
>> > >> these
>> > >> > > > cases,
>> > >> > > > > we
>> > >> > > > > > > > > should
>> > >> > > > > > > > > >>> just
>> > >> > > > > > > > > >>>>> retry the request: we are guaranteed
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           // to eventually get a logStartOffset
>> > once
>> > >> > > things
>> > >> > > > > > > settle
>> > >> > > > > > > > > down.
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           return true;
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>       }
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>       if (batch.sequenceHasBeenReset()) {
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           // When the first inflight batch
>> fails
>> > >> due to
>> > >> > > the
>> > >> > > > > > > > > truncation
>> > >> > > > > > > > > >>> case,
>> > >> > > > > > > > > >>>>> then the sequences of all the other
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           // in flight batches would have been
>> > >> > restarted
>> > >> > > > from
>> > >> > > > > > the
>> > >> > > > > > > > > >>> beginning.
>> > >> > > > > > > > > >>>>> However, when those responses
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           // come back from the broker, they
>> would
>> > >> also
>> > >> > > > come
>> > >> > > > > > with
>> > >> > > > > > > > an
>> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID error. In this case, we
>> should
>> > >> not
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           // reset the sequence numbers to the
>> > >> > beginning.
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           return true;
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>       } else if
>> > >> > > > > (lastAckedOffset(batch.topicPartition).orElse(
>> > >> > > > > > > > > >>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
>> > >> > > > response.logStartOffset) {
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           // The head of the log has been
>> removed,
>> > >> > > probably
>> > >> > > > > due
>> > >> > > > > > > to
>> > >> > > > > > > > > the
>> > >> > > > > > > > > >>>>> retention time elapsing. In this case,
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           // we expect to lose the producer
>> state.
>> > >> For
>> > >> > > the
>> > >> > > > > > > > > transactional
>> > >> > > > > > > > > >>>>> producer, reset the sequences of all
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           // inflight batches to be from the
>> > >> beginning
>> > >> > > and
>> > >> > > > > > retry
>> > >> > > > > > > > > them,
>> > >> > > > > > > > > >>> so
>> > >> > > > > > > > > >>>>> that the transaction does not need to
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           // be aborted. For the idempotent
>> > >> producer,
>> > >> > > bump
>> > >> > > > > the
>> > >> > > > > > > > epoch
>> > >> > > > > > > > > to
>> > >> > > > > > > > > >>> avoid
>> > >> > > > > > > > > >>>>> reusing (sequence, epoch) pairs
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           if (isTransactional()) {
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>
>> > >> > > > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
>> > >> > > > > > > > > >>>>> this.producerIdAndEpoch);
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           } else {
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>
>> > >> > > > > >  requestEpochBumpForPartition(batch.topicPartition);
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           }
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           return true;
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>       }
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>       if (!isTransactional()) {
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           // For the idempotent producer,
>> always
>> > >> retry
>> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
>> > >> > > > > > > > > >>>>> errors. If the batch has the current
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           // producer ID and epoch, request a
>> bump
>> > >> of
>> > >> > the
>> > >> > > > > > epoch.
>> > >> > > > > > > > > >>> Otherwise
>> > >> > > > > > > > > >>>>> just retry the produce.
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>
>> > >> > > >  requestEpochBumpForPartition(batch.topicPartition);
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>           return true;
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>       }
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>> }
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>> I was considering keeping this behavior — but
>> am
>> > >> open
>> > >> > to
>> > >> > > > > > > > simplifying
>> > >> > > > > > > > > >>> it.
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>> We are leaving changes to older clients off the
>> > >> table
>> > >> > > here
>> > >> > > > > > since
>> > >> > > > > > > it
>> > >> > > > > > > > > >>> caused
>> > >> > > > > > > > > >>>>> many issues for clients in the past. Previously
>> > this
>> > >> > was
>> > >> > > a
>> > >> > > > > > fatal
>> > >> > > > > > > > > error
>> > >> > > > > > > > > >>> and
>> > >> > > > > > > > > >>>>> we didn't have the mechanisms in place to
>> detect
>> > >> when
>> > >> > > this
>> > >> > > > > was
>> > >> > > > > > a
>> > >> > > > > > > > > >>> legitimate
>> > >> > > > > > > > > >>>>> case vs some bug or gap in the protocol.
>> Ensuring
>> > >> each
>> > >> > > > > > > transaction
>> > >> > > > > > > > > has
>> > >> > > > > > > > > >>> its
>> > >> > > > > > > > > >>>>> own epoch should close this gap.
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>> And to address Jeff's second point:
>> > >> > > > > > > > > >>>>> *does the typical produce request path append
>> > >> records
>> > >> > to
>> > >> > > > > local
>> > >> > > > > > > log
>> > >> > > > > > > > > >>> along*
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>> *with the currentTxnFirstOffset information? I
>> > would
>> > >> > like
>> > >> > > > to
>> > >> > > > > > > > > >>> understand*
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>> *when the field is written to disk.*
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>> Yes, the first produce request populates this
>> > field
>> > >> and
>> > >> > > > > writes
>> > >> > > > > > > the
>> > >> > > > > > > > > >>> offset
>> > >> > > > > > > > > >>>>> as part of the record batch and also to the
>> > producer
>> > >> > > state
>> > >> > > > > > > > snapshot.
>> > >> > > > > > > > > >>> When
>> > >> > > > > > > > > >>>>> we reload the records on restart and/or
>> > >> reassignment,
>> > >> > we
>> > >> > > > > > > repopulate
>> > >> > > > > > > > > >>> this
>> > >> > > > > > > > > >>>>> field with the snapshot from disk along with
>> the
>> > >> rest
>> > >> > of
>> > >> > > > the
>> > >> > > > > > > > producer
>> > >> > > > > > > > > >>>>> state.
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>> Let me know if there are further comments
>> and/or
>> > >> > > questions.
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>> Thanks,
>> > >> > > > > > > > > >>>>> Justine
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>> On Tue, Nov 22, 2022 at 9:00 PM Jeff Kim
>> > >> > > > > > > > > <jeff.kim@confluent.io.invalid
>> > >> > > > > > > > > >>>>
>> > >> > > > > > > > > >>>>> wrote:
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>>> Hi Justine,
>> > >> > > > > > > > > >>>>>>
>> > >> > > > > > > > > >>>>>> Thanks for the KIP! I have two questions:
>> > >> > > > > > > > > >>>>>>
>> > >> > > > > > > > > >>>>>> 1) For new clients, we can once again return
>> an
>> > >> error
>> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
>> > >> > > > > > > > > >>>>>> for sequences
>> > >> > > > > > > > > >>>>>> that are non-zero when there is no producer
>> state
>> > >> > > present
>> > >> > > > on
>> > >> > > > > > the
>> > >> > > > > > > > > >>> server.
>> > >> > > > > > > > > >>>>>> This will indicate we missed the 0 sequence
>> and
>> > we
>> > >> > don't
>> > >> > > > yet
>> > >> > > > > > > want
>> > >> > > > > > > > to
>> > >> > > > > > > > > >>>>> write
>> > >> > > > > > > > > >>>>>> to the log.
>> > >> > > > > > > > > >>>>>>
>> > >> > > > > > > > > >>>>>> I would like to understand the current
>> behavior
>> > to
>> > >> > > handle
>> > >> > > > > > older
>> > >> > > > > > > > > >>> clients,
>> > >> > > > > > > > > >>>>>> and if there are any changes we are making.
>> Maybe
>> > >> I'm
>> > >> > > > > missing
>> > >> > > > > > > > > >>> something,
>> > >> > > > > > > > > >>>>>> but we would want to identify whether we
>> missed
>> > >> the 0
>> > >> > > > > sequence
>> > >> > > > > > > for
>> > >> > > > > > > > > >>> older
>> > >> > > > > > > > > >>>>>> clients, no?
>> > >> > > > > > > > > >>>>>>
>> > >> > > > > > > > > >>>>>> 2) Upon returning from the transaction
>> > >> coordinator, we
>> > >> > > can
>> > >> > > > > set
>> > >> > > > > > > the
>> > >> > > > > > > > > >>>>>> transaction
>> > >> > > > > > > > > >>>>>> as ongoing on the leader by populating
>> > >> > > > currentTxnFirstOffset
>> > >> > > > > > > > > >>>>>> through the typical produce request handling.
>> > >> > > > > > > > > >>>>>>
>> > >> > > > > > > > > >>>>>> does the typical produce request path append
>> > >> records
>> > >> > to
>> > >> > > > > local
>> > >> > > > > > > log
>> > >> > > > > > > > > >>> along
>> > >> > > > > > > > > >>>>>> with the currentTxnFirstOffset information? I
>> > would
>> > >> > like
>> > >> > > > to
>> > >> > > > > > > > > understand
>> > >> > > > > > > > > >>>>>> when the field is written to disk.
>> > >> > > > > > > > > >>>>>>
>> > >> > > > > > > > > >>>>>> Thanks,
>> > >> > > > > > > > > >>>>>> Jeff
>> > >> > > > > > > > > >>>>>>
>> > >> > > > > > > > > >>>>>>
>> > >> > > > > > > > > >>>>>> On Tue, Nov 22, 2022 at 4:44 PM Artem Livshits
>> > >> > > > > > > > > >>>>>> <al...@confluent.io.invalid> wrote:
>> > >> > > > > > > > > >>>>>>
>> > >> > > > > > > > > >>>>>>> Hi Justine,
>> > >> > > > > > > > > >>>>>>>
>> > >> > > > > > > > > >>>>>>> Thank you for the KIP.  I have one question.
>> > >> > > > > > > > > >>>>>>>
>> > >> > > > > > > > > >>>>>>> 5) For new clients, we can once again return
>> an
>> > >> error
>> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
>> > >> > > > > > > > > >>>>>>>
>> > >> > > > > > > > > >>>>>>> I believe we had problems in the past with
>> > >> returning
>> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
>> > >> > > > > > > > > >>>>>>> because it was considered fatal and required
>> > >> client
>> > >> > > > > restart.
>> > >> > > > > > > It
>> > >> > > > > > > > > >>> would
>> > >> > > > > > > > > >>>>> be
>> > >> > > > > > > > > >>>>>>> good to spell out the new client behavior
>> when
>> > it
>> > >> > > > receives
>> > >> > > > > > the
>> > >> > > > > > > > > error.
>> > >> > > > > > > > > >>>>>>>
>> > >> > > > > > > > > >>>>>>> -Artem
>> > >> > > > > > > > > >>>>>>>
>> > >> > > > > > > > > >>>>>>> On Tue, Nov 22, 2022 at 10:00 AM Justine
>> Olshan
>> > >> > > > > > > > > >>>>>>> <jo...@confluent.io.invalid> wrote:
>> > >> > > > > > > > > >>>>>>>
>> > >> > > > > > > > > >>>>>>>> Thanks for taking a look Matthias. I've
>> tried
>> > to
>> > >> > > answer
>> > >> > > > > your
>> > >> > > > > > > > > >>>>> questions
>> > >> > > > > > > > > >>>>>>>> below:
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>> 10)
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>> Right — so the hanging transaction only
>> occurs
>> > >> when
>> > >> > we
>> > >> > > > > have
>> > >> > > > > > a
>> > >> > > > > > > > late
>> > >> > > > > > > > > >>>>>>> message
>> > >> > > > > > > > > >>>>>>>> come in and the partition is never added to
>> a
>> > >> > > > transaction
>> > >> > > > > > > again.
>> > >> > > > > > > > > If
>> > >> > > > > > > > > >>>>> we
>> > >> > > > > > > > > >>>>>>>> never add the partition to a transaction, we
>> > will
>> > >> > > never
>> > >> > > > > > write
>> > >> > > > > > > a
>> > >> > > > > > > > > >>>>> marker
>> > >> > > > > > > > > >>>>>>> and
>> > >> > > > > > > > > >>>>>>>> never advance the LSO.
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>> If we do end up adding the partition to the
>> > >> > > transaction
>> > >> > > > (I
>> > >> > > > > > > > suppose
>> > >> > > > > > > > > >>>>> this
>> > >> > > > > > > > > >>>>>>> can
>> > >> > > > > > > > > >>>>>>>> happen before or after the late message
>> comes
>> > in)
>> > >> > then
>> > >> > > > we
>> > >> > > > > > will
>> > >> > > > > > > > > >>>>> include
>> > >> > > > > > > > > >>>>>>> the
>> > >> > > > > > > > > >>>>>>>> late message in the next (incorrect)
>> > transaction.
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>> So perhaps it is clearer to make the
>> > distinction
>> > >> > > between
>> > >> > > > > > > > messages
>> > >> > > > > > > > > >>>>> that
>> > >> > > > > > > > > >>>>>>>> eventually get added to the transaction (but
>> > the
>> > >> > wrong
>> > >> > > > > one)
>> > >> > > > > > or
>> > >> > > > > > > > > >>>>> messages
>> > >> > > > > > > > > >>>>>>>> that never get added and become hanging.
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>> 20)
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>> The client side change for 2 is removing the
>> > >> > > > addPartitions
>> > >> > > > > > to
>> > >> > > > > > > > > >>>>>> transaction
>> > >> > > > > > > > > >>>>>>>> call. We don't need to make this from the
>> > >> producer
>> > >> > to
>> > >> > > > the
>> > >> > > > > > txn
>> > >> > > > > > > > > >>>>>>> coordinator,
>> > >> > > > > > > > > >>>>>>>> only server side.
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>> In my opinion, the issue with the
>> > >> addPartitionsToTxn
>> > >> > > > call
>> > >> > > > > > for
>> > >> > > > > > > > > older
>> > >> > > > > > > > > >>>>>>> clients
>> > >> > > > > > > > > >>>>>>>> is that we don't have the epoch bump, so we
>> > don't
>> > >> > know
>> > >> > > > if
>> > >> > > > > > the
>> > >> > > > > > > > > >>> message
>> > >> > > > > > > > > >>>>>>>> belongs to the previous transaction or this
>> > one.
>> > >> We
>> > >> > > need
>> > >> > > > > to
>> > >> > > > > > > > check
>> > >> > > > > > > > > if
>> > >> > > > > > > > > >>>>>> the
>> > >> > > > > > > > > >>>>>>>> partition has been added to this
>> transaction.
>> > Of
>> > >> > > course,
>> > >> > > > > > this
>> > >> > > > > > > > > means
>> > >> > > > > > > > > >>>>> we
>> > >> > > > > > > > > >>>>>>>> won't completely cover the case where we
>> have a
>> > >> > really
>> > >> > > > > late
>> > >> > > > > > > > > message
>> > >> > > > > > > > > >>>>> and
>> > >> > > > > > > > > >>>>>>> we
>> > >> > > > > > > > > >>>>>>>> have added the partition to the new
>> > transaction,
>> > >> but
>> > >> > > > > that's
>> > >> > > > > > > > > >>>>>> unfortunately
>> > >> > > > > > > > > >>>>>>>> something we will need the new clients to
>> > cover.
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>> 30)
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>> Transaction is ongoing = partition was
>> added to
>> > >> > > > > transaction
>> > >> > > > > > > via
>> > >> > > > > > > > > >>>>>>>> addPartitionsToTxn. We check this with the
>> > >> > > > > > > DescribeTransactions
>> > >> > > > > > > > > >>> call.
>> > >> > > > > > > > > >>>>>> Let
>> > >> > > > > > > > > >>>>>>>> me know if this wasn't sufficiently
>> explained
>> > >> here:
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>
>> > >> > > > > > > > > >>>>>>
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > >
>> > >> > > > > > > >
>> > >> > > > > > >
>> > >> > > > > >
>> > >> > > > >
>> > >> > > >
>> > >> > >
>> > >> >
>> > >>
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>> 40)
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>> The idea here is that if any messages
>> somehow
>> > >> come
>> > >> > in
>> > >> > > > > before
>> > >> > > > > > > we
>> > >> > > > > > > > > get
>> > >> > > > > > > > > >>>>> the
>> > >> > > > > > > > > >>>>>>> new
>> > >> > > > > > > > > >>>>>>>> epoch to the producer, they will be fenced.
>> > >> However,
>> > >> > > if
>> > >> > > > we
>> > >> > > > > > > don't
>> > >> > > > > > > > > >>>>> think
>> > >> > > > > > > > > >>>>>>> this
>> > >> > > > > > > > > >>>>>>>> is necessary, it can be discussed
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>> 50)
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>> It should be synchronous because if we have
>> an
>> > >> event
>> > >> > > > (ie,
>> > >> > > > > an
>> > >> > > > > > > > > error)
>> > >> > > > > > > > > >>>>>> that
>> > >> > > > > > > > > >>>>>>>> causes us to need to abort the transaction,
>> we
>> > >> need
>> > >> > to
>> > >> > > > > know
>> > >> > > > > > > > which
>> > >> > > > > > > > > >>>>>>>> partitions to send transaction markers to.
>> We
>> > >> know
>> > >> > the
>> > >> > > > > > > > partitions
>> > >> > > > > > > > > >>>>>> because
>> > >> > > > > > > > > >>>>>>>> we added them to the coordinator via the
>> > >> > > > > addPartitionsToTxn
>> > >> > > > > > > > call.
>> > >> > > > > > > > > >>>>>>>> Previously we have had asynchronous calls in
>> > the
>> > >> > past
>> > >> > > > (ie,
>> > >> > > > > > > > writing
>> > >> > > > > > > > > >>>>> the
>> > >> > > > > > > > > >>>>>>>> commit markers when the transaction is
>> > completed)
>> > >> > but
>> > >> > > > > often
>> > >> > > > > > > this
>> > >> > > > > > > > > >>> just
>> > >> > > > > > > > > >>>>>>>> causes confusion as we need to wait for some
>> > >> > > operations
>> > >> > > > to
>> > >> > > > > > > > > complete.
>> > >> > > > > > > > > >>>>> In
>> > >> > > > > > > > > >>>>>>> the
>> > >> > > > > > > > > >>>>>>>> writing commit markers case, clients often
>> see
>> > >> > > > > > > > > >>>>> CONCURRENT_TRANSACTIONs
>> > >> > > > > > > > > >>>>>>>> error messages and that can be confusing.
>> For
>> > >> that
>> > >> > > > reason,
>> > >> > > > > > it
>> > >> > > > > > > > may
>> > >> > > > > > > > > be
>> > >> > > > > > > > > >>>>>>>> simpler to just have synchronous calls —
>> > >> especially
>> > >> > if
>> > >> > > > we
>> > >> > > > > > need
>> > >> > > > > > > > to
>> > >> > > > > > > > > >>>>> block
>> > >> > > > > > > > > >>>>>>> on
>> > >> > > > > > > > > >>>>>>>> some operation's completion anyway before we
>> > can
>> > >> > start
>> > >> > > > the
>> > >> > > > > > > next
>> > >> > > > > > > > > >>>>>>>> transaction. And yes, I meant coordinator. I
>> > will
>> > >> > fix
>> > >> > > > > that.
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>> 60)
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>> When we are checking if the transaction is
>> > >> ongoing,
>> > >> > we
>> > >> > > > > need
>> > >> > > > > > to
>> > >> > > > > > > > > make
>> > >> > > > > > > > > >>> a
>> > >> > > > > > > > > >>>>>>> round
>> > >> > > > > > > > > >>>>>>>> trip from the leader partition to the
>> > transaction
>> > >> > > > > > coordinator.
>> > >> > > > > > > > In
>> > >> > > > > > > > > >>> the
>> > >> > > > > > > > > >>>>>>> time
>> > >> > > > > > > > > >>>>>>>> we are waiting for this message to come
>> back,
>> > in
>> > >> > > theory
>> > >> > > > we
>> > >> > > > > > > could
>> > >> > > > > > > > > >>> have
>> > >> > > > > > > > > >>>>>>> sent
>> > >> > > > > > > > > >>>>>>>> a commit/abort call that would make the
>> > original
>> > >> > > result
>> > >> > > > of
>> > >> > > > > > the
>> > >> > > > > > > > > check
>> > >> > > > > > > > > >>>>>> out
>> > >> > > > > > > > > >>>>>>> of
>> > >> > > > > > > > > >>>>>>>> date. That is why we can check the leader
>> state
>> > >> > before
>> > >> > > > we
>> > >> > > > > > > write
>> > >> > > > > > > > to
>> > >> > > > > > > > > >>>>> the
>> > >> > > > > > > > > >>>>>>> log.
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>> I'm happy to update the KIP if some of these
>> > >> things
>> > >> > > were
>> > >> > > > > not
>> > >> > > > > > > > > clear.
>> > >> > > > > > > > > >>>>>>>> Thanks,
>> > >> > > > > > > > > >>>>>>>> Justine
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>> On Mon, Nov 21, 2022 at 7:11 PM Matthias J.
>> > Sax <
>> > >> > > > > > > > mjsax@apache.org
>> > >> > > > > > > > > >
>> > >> > > > > > > > > >>>>>>> wrote:
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>>> Thanks for the KIP.
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>> Couple of clarification questions (I am
>> not a
>> > >> > broker
>> > >> > > > > expert
>> > >> > > > > > > do
>> > >> > > > > > > > > >>>>> maybe
>> > >> > > > > > > > > >>>>>>>>> some question are obvious for others, but
>> not
>> > >> for
>> > >> > me
>> > >> > > > with
>> > >> > > > > > my
>> > >> > > > > > > > lack
>> > >> > > > > > > > > >>>>> of
>> > >> > > > > > > > > >>>>>>>>> broker knowledge).
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>> (10)
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>> 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.
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>> What happens if the message come in before
>> the
>> > >> next
>> > >> > > > > > > > > >>>>>> addPartitionsToTxn
>> > >> > > > > > > > > >>>>>>>>> request? It seems the broker hosting the
>> data
>> > >> > > > partitions
>> > >> > > > > > > won't
>> > >> > > > > > > > > know
>> > >> > > > > > > > > >>>>>>>>> anything about it and append it to the
>> > >> partition,
>> > >> > > too?
>> > >> > > > > What
>> > >> > > > > > > is
>> > >> > > > > > > > > the
>> > >> > > > > > > > > >>>>>>>>> difference between both cases?
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>> Also, it seems a TX would only hang, if
>> there
>> > >> is no
>> > >> > > > > > following
>> > >> > > > > > > > TX
>> > >> > > > > > > > > >>>>> that
>> > >> > > > > > > > > >>>>>>> is
>> > >> > > > > > > > > >>>>>>>>> either committer or aborted? Thus, for the
>> > case
>> > >> > > above,
>> > >> > > > > the
>> > >> > > > > > TX
>> > >> > > > > > > > > might
>> > >> > > > > > > > > >>>>>>>>> actually not hang (of course, we might get
>> an
>> > >> EOS
>> > >> > > > > violation
>> > >> > > > > > > if
>> > >> > > > > > > > > the
>> > >> > > > > > > > > >>>>>>> first
>> > >> > > > > > > > > >>>>>>>>> TX was aborted and the second committed, or
>> > the
>> > >> > other
>> > >> > > > way
>> > >> > > > > > > > > around).
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>> (20)
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>> Of course, 1 and 2 require client-side
>> > >> changes, so
>> > >> > > for
>> > >> > > > > > older
>> > >> > > > > > > > > >>>>>> clients,
>> > >> > > > > > > > > >>>>>>>>> those approaches won’t apply.
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>> For (1) I understand why a client change is
>> > >> > > necessary,
>> > >> > > > > but
>> > >> > > > > > > not
>> > >> > > > > > > > > sure
>> > >> > > > > > > > > >>>>>> why
>> > >> > > > > > > > > >>>>>>>>> we need a client change for (2). Can you
>> > >> elaborate?
>> > >> > > --
>> > >> > > > > > Later
>> > >> > > > > > > > you
>> > >> > > > > > > > > >>>>>>> explain
>> > >> > > > > > > > > >>>>>>>>> that we should send a
>> > >> DescribeTransactionRequest,
>> > >> > > but I
>> > >> > > > > am
>> > >> > > > > > > not
>> > >> > > > > > > > > sure
>> > >> > > > > > > > > >>>>>>> why?
>> > >> > > > > > > > > >>>>>>>>> Can't we not just do an implicit
>> > >> AddPartiitonToTx,
>> > >> > > too?
>> > >> > > > > If
>> > >> > > > > > > the
>> > >> > > > > > > > > old
>> > >> > > > > > > > > >>>>>>>>> producer correctly registered the partition
>> > >> > already,
>> > >> > > > the
>> > >> > > > > > > > > >>>>>> TX-coordinator
>> > >> > > > > > > > > >>>>>>>>> can just ignore it as it's an idempotent
>> > >> operation?
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>> (30)
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>> To cover older clients, we will ensure a
>> > >> > transaction
>> > >> > > > is
>> > >> > > > > > > > ongoing
>> > >> > > > > > > > > >>>>>>> before
>> > >> > > > > > > > > >>>>>>>>> we write to a transaction
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>> Not sure what you mean by this? Can you
>> > >> elaborate?
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>> (40)
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>> [the TX-coordinator] will write the
>> prepare
>> > >> commit
>> > >> > > > > message
>> > >> > > > > > > > with
>> > >> > > > > > > > > a
>> > >> > > > > > > > > >>>>>>>> bumped
>> > >> > > > > > > > > >>>>>>>>> epoch and send WriteTxnMarkerRequests with
>> the
>> > >> > bumped
>> > >> > > > > > epoch.
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>> Why do we use the bumped epoch for both? It
>> > >> seems
>> > >> > > more
>> > >> > > > > > > > intuitive
>> > >> > > > > > > > > to
>> > >> > > > > > > > > >>>>>> use
>> > >> > > > > > > > > >>>>>>>>> the current epoch, and only return the
>> bumped
>> > >> epoch
>> > >> > > to
>> > >> > > > > the
>> > >> > > > > > > > > >>>>> producer?
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>> (50) "Implicit AddPartitionToTransaction"
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>> Why does the implicitly sent request need
>> to
>> > be
>> > >> > > > > > synchronous?
>> > >> > > > > > > > The
>> > >> > > > > > > > > >>>>> KIP
>> > >> > > > > > > > > >>>>>>>>> also says
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>> in case we need to abort and need to know
>> > which
>> > >> > > > > partitions
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>> What do you mean by this?
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>> we don’t want to write to it before we
>> store
>> > in
>> > >> > the
>> > >> > > > > > > > transaction
>> > >> > > > > > > > > >>>>>>> manager
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>> Do you mean TX-coordinator instead of
>> > "manager"?
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>> (60)
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>> For older clients and ensuring that the TX
>> is
>> > >> > > ongoing,
>> > >> > > > > you
>> > >> > > > > > > > > >>>>> describe a
>> > >> > > > > > > > > >>>>>>>>> race condition. I am not sure if I can
>> follow
>> > >> here.
>> > >> > > Can
>> > >> > > > > you
>> > >> > > > > > > > > >>>>>> elaborate?
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>> -Matthias
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>> On 11/18/22 1:21 PM, Justine Olshan wrote:
>> > >> > > > > > > > > >>>>>>>>>> Hey all!
>> > >> > > > > > > > > >>>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>> I'd like to start a discussion on my
>> proposal
>> > >> to
>> > >> > add
>> > >> > > > > some
>> > >> > > > > > > > > >>>>>> server-side
>> > >> > > > > > > > > >>>>>>>>>> checks on transactions to avoid hanging
>> > >> > > transactions.
>> > >> > > > I
>> > >> > > > > > know
>> > >> > > > > > > > > this
>> > >> > > > > > > > > >>>>>> has
>> > >> > > > > > > > > >>>>>>>>> been
>> > >> > > > > > > > > >>>>>>>>>> an issue for some time, so I really hope
>> this
>> > >> KIP
>> > >> > > will
>> > >> > > > > be
>> > >> > > > > > > > > helpful
>> > >> > > > > > > > > >>>>>> for
>> > >> > > > > > > > > >>>>>>>>> many
>> > >> > > > > > > > > >>>>>>>>>> users of EOS.
>> > >> > > > > > > > > >>>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>> The KIP includes changes that will be
>> > >> compatible
>> > >> > > with
>> > >> > > > > old
>> > >> > > > > > > > > clients
>> > >> > > > > > > > > >>>>>> and
>> > >> > > > > > > > > >>>>>>>>>> changes to improve performance and
>> > correctness
>> > >> on
>> > >> > > new
>> > >> > > > > > > clients.
>> > >> > > > > > > > > >>>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>> Please take a look and leave any comments
>> you
>> > >> may
>> > >> > > > have!
>> > >> > > > > > > > > >>>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>> KIP:
>> > >> > > > > > > > > >>>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>
>> > >> > > > > > > > > >>>>>>
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > >
>> > >> > > > > > > >
>> > >> > > > > > >
>> > >> > > > > >
>> > >> > > > >
>> > >> > > >
>> > >> > >
>> > >> >
>> > >>
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
>> > >> > > > > > > > > >>>>>>>>>> JIRA:
>> > >> > > > https://issues.apache.org/jira/browse/KAFKA-14402
>> > >> > > > > > > > > >>>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>> Thanks!
>> > >> > > > > > > > > >>>>>>>>>> Justine
>> > >> > > > > > > > > >>>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>>
>> > >> > > > > > > > > >>>>>>>>
>> > >> > > > > > > > > >>>>>>>
>> > >> > > > > > > > > >>>>>>
>> > >> > > > > > > > > >>>>>
>> > >> > > > > > > > > >>>>
>> > >> > > > > > > > > >>>
>> > >> > > > > > > > > >>
>> > >> > > > > > > > > >
>> > >> > > > > > > > >
>> > >> > > > > > > >
>> > >> > > > > > >
>> > >> > > > > >
>> > >> > > > >
>> > >> > > >
>> > >> > >
>> > >> >
>> > >>
>> > >
>> >
>>
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
Thanks Jason. Those changes make sense to me. I will update the KIP.



On Fri, Jan 6, 2023 at 3:31 PM Jason Gustafson <ja...@confluent.io.invalid>
wrote:

> Hey Justine,
>
> > I was wondering about compatibility here. When we send requests
> between brokers, we want to ensure that the receiving broker understands
> the request (specifically the new fields). Typically this is done via
> IBP/metadata version.
> I'm trying to think if there is a way around it but I'm not sure there is.
>
> Yes. I think we would gate usage of this behind an IBP bump. Does that seem
> reasonable?
>
> > As for the improvements -- can you clarify how the multiple transactional
> IDs would help here? Were you thinking of a case where we wait/batch
> multiple produce requests together? My understanding for now was 1
> transactional ID and one validation per 1 produce request.
>
> Each call to `AddPartitionsToTxn` is essentially a write to the transaction
> log and must block on replication. The more we can fit into a single
> request, the more writes we can do in parallel. The alternative is to make
> use of more connections, but usually we prefer batching since the network
> stack is not really optimized for high connection/request loads.
>
> > Finally with respect to the authorizations, I think it makes sense to
> skip
> topic authorizations, but I'm a bit confused by the "leader ID" field.
> Wouldn't we just want to flag the request as from a broker (does it matter
> which one?).
>
> We could also make it version-based. For the next version, we could require
> CLUSTER auth. So clients would not be able to use the API anymore, which is
> probably what we want.
>
> -Jason
>
> On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan
> <jo...@confluent.io.invalid>
> wrote:
>
> > As a follow up, I was just thinking about the batching a bit more.
> > I suppose if we have one request in flight and we queue up the other
> > produce requests in some sort of purgatory, we could send information out
> > for all of them rather than one by one. So that would be a benefit of
> > batching partitions to add per transaction.
> >
> > I'll need to think a bit more on the design of this part of the KIP, and
> > will update the KIP in the next few days.
> >
> > Thanks,
> > Justine
> >
> > On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan <jo...@confluent.io>
> > wrote:
> >
> > > Hey Jason -- thanks for the input -- I was just digging a bit deeper
> into
> > > the design + implementation of the validation calls here and what you
> say
> > > makes sense.
> > >
> > > I was wondering about compatibility here. When we send requests
> > > between brokers, we want to ensure that the receiving broker
> understands
> > > the request (specifically the new fields). Typically this is done via
> > > IBP/metadata version.
> > > I'm trying to think if there is a way around it but I'm not sure there
> > is.
> > >
> > > As for the improvements -- can you clarify how the multiple
> transactional
> > > IDs would help here? Were you thinking of a case where we wait/batch
> > > multiple produce requests together? My understanding for now was 1
> > > transactional ID and one validation per 1 produce request.
> > >
> > > Finally with respect to the authorizations, I think it makes sense to
> > skip
> > > topic authorizations, but I'm a bit confused by the "leader ID" field.
> > > Wouldn't we just want to flag the request as from a broker (does it
> > matter
> > > which one?).
> > >
> > > I think I want to adopt these suggestions, just had a few questions on
> > the
> > > details.
> > >
> > > Thanks,
> > > Justine
> > >
> > > On Thu, Jan 5, 2023 at 5:05 PM Jason Gustafson
> > <ja...@confluent.io.invalid>
> > > wrote:
> > >
> > >> Hi Justine,
> > >>
> > >> Thanks for the proposal.
> > >>
> > >> I was thinking about the implementation a little bit. In the current
> > >> proposal, the behavior depends on whether we have an old or new
> client.
> > >> For
> > >> old clients, we send `DescribeTransactions` and verify the result and
> > for
> > >> new clients, we send `AddPartitionsToTxn`. We might be able to
> simplify
> > >> the
> > >> implementation if we can use the same request type. For example, what
> if
> > >> we
> > >> bump the protocol version for `AddPartitionsToTxn` and add a
> > >> `validateOnly`
> > >> flag? For older versions, we can set `validateOnly=true` so that the
> > >> request only returns successfully if the partition had already been
> > added.
> > >> For new versions, we can set `validateOnly=false` and the partition
> will
> > >> be
> > >> added to the transaction. The other slightly annoying thing that this
> > >> would
> > >> get around is the need to collect the transaction state for all
> > partitions
> > >> even when we only care about a subset.
> > >>
> > >> Some additional improvements to consider:
> > >>
> > >> - We can give `AddPartitionsToTxn` better batch support for
> inter-broker
> > >> usage. Currently we only allow one `TransactionalId` to be specified,
> > but
> > >> the broker may get some benefit being able to batch across multiple
> > >> transactions.
> > >> - Another small improvement is skipping topic authorization checks for
> > >> `AddPartitionsToTxn` when the request is from a broker. Perhaps we can
> > add
> > >> a field for the `LeaderId` or something like that and require CLUSTER
> > >> permission when set.
> > >>
> > >> Best,
> > >> Jason
> > >>
> > >>
> > >>
> > >> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao <ju...@confluent.io.invalid>
> > wrote:
> > >>
> > >> > Hi, Justine,
> > >> >
> > >> > Thanks for the explanation. It makes sense to me now.
> > >> >
> > >> > Jun
> > >> >
> > >> > On Mon, Dec 19, 2022 at 1:42 PM Justine Olshan
> > >> > <jo...@confluent.io.invalid>
> > >> > wrote:
> > >> >
> > >> > > Hi Jun,
> > >> > >
> > >> > > My understanding of the mechanism is that when we get to the last
> > >> epoch,
> > >> > we
> > >> > > increment to the fencing/last epoch and if any further requests
> come
> > >> in
> > >> > for
> > >> > > this producer ID they are fenced. Then the producer gets a new ID
> > and
> > >> > > restarts with epoch/sequence 0. The fenced epoch sticks around for
> > the
> > >> > > duration of producer.id.expiration.ms and blocks any late
> messages
> > >> > there.
> > >> > > The new ID will get to take advantage of the improved semantics
> > around
> > >> > > non-zero start sequences. So I think we are covered.
> > >> > >
> > >> > > The only potential issue is overloading the cache, but hopefully
> the
> > >> > > improvements (lowered producer.id.expiration.ms) will help with
> > that.
> > >> > Let
> > >> > > me know if you still have concerns.
> > >> > >
> > >> > > Thanks,
> > >> > > Justine
> > >> > >
> > >> > > On Mon, Dec 19, 2022 at 10:24 AM Jun Rao <jun@confluent.io.invalid
> >
> > >> > wrote:
> > >> > >
> > >> > > > Hi, Justine,
> > >> > > >
> > >> > > > Thanks for the explanation.
> > >> > > >
> > >> > > > 70. The proposed fencing logic doesn't apply when pid changes,
> is
> > >> that
> > >> > > > right? If so, I am not sure how complete we are addressing this
> > >> issue
> > >> > if
> > >> > > > the pid changes more frequently.
> > >> > > >
> > >> > > > Thanks,
> > >> > > >
> > >> > > > Jun
> > >> > > >
> > >> > > >
> > >> > > >
> > >> > > > On Fri, Dec 16, 2022 at 9:16 AM Justine Olshan
> > >> > > > <jo...@confluent.io.invalid>
> > >> > > > wrote:
> > >> > > >
> > >> > > > > Hi Jun,
> > >> > > > >
> > >> > > > > Thanks for replying!
> > >> > > > >
> > >> > > > > 70.We already do the overflow mechanism, so my change would
> just
> > >> make
> > >> > > it
> > >> > > > > happen more often.
> > >> > > > > I was also not suggesting a new field in the log, but in the
> > >> > response,
> > >> > > > > which would be gated by the client version. Sorry if something
> > >> there
> > >> > is
> > >> > > > > unclear. I think we are starting to diverge.
> > >> > > > > The goal of this KIP is to not change to the marker format at
> > all.
> > >> > > > >
> > >> > > > > 71. Yes, I guess I was going under the assumption that the log
> > >> would
> > >> > > just
> > >> > > > > look at its last epoch and treat it as the current epoch. I
> > >> suppose
> > >> > we
> > >> > > > can
> > >> > > > > have some special logic that if the last epoch was on a marker
> > we
> > >> > > > actually
> > >> > > > > expect the next epoch or something like that. We just need to
> > >> > > distinguish
> > >> > > > > based on whether we had a commit/abort marker.
> > >> > > > >
> > >> > > > > 72.
> > >> > > > > > if the producer epoch hasn't been bumped on the
> > >> > > > > broker, it seems that the stucked message will fail the
> sequence
> > >> > > > validation
> > >> > > > > and will be ignored. If the producer epoch has been bumped, we
> > >> ignore
> > >> > > the
> > >> > > > > sequence check and the stuck message could be appended to the
> > log.
> > >> > So,
> > >> > > is
> > >> > > > > the latter case that we want to guard?
> > >> > > > >
> > >> > > > > I'm not sure I follow that "the message will fail the sequence
> > >> > > > validation".
> > >> > > > > In some of these cases, we had an abort marker (due to an
> error)
> > >> and
> > >> > > then
> > >> > > > > the late message comes in with the correct sequence number.
> This
> > >> is a
> > >> > > > case
> > >> > > > > covered by the KIP.
> > >> > > > > The latter case is actually not something we've considered
> > here. I
> > >> > > think
> > >> > > > > generally when we bump the epoch, we are accepting that the
> > >> sequence
> > >> > > does
> > >> > > > > not need to be checked anymore. My understanding is also that
> we
> > >> > don't
> > >> > > > > typically bump epoch mid transaction (based on a quick look at
> > the
> > >> > > code)
> > >> > > > > but let me know if that is the case.
> > >> > > > >
> > >> > > > > Thanks,
> > >> > > > > Justine
> > >> > > > >
> > >> > > > > On Thu, Dec 15, 2022 at 12:23 PM Jun Rao
> > <jun@confluent.io.invalid
> > >> >
> > >> > > > wrote:
> > >> > > > >
> > >> > > > > > Hi, Justine,
> > >> > > > > >
> > >> > > > > > Thanks for the reply.
> > >> > > > > >
> > >> > > > > > 70. Assigning a new pid on int overflow seems a bit hacky.
> If
> > we
> > >> > > need a
> > >> > > > > txn
> > >> > > > > > level id, it will be better to model this explicitly.
> Adding a
> > >> new
> > >> > > > field
> > >> > > > > > would require a bit more work since it requires a new txn
> > marker
> > >> > > format
> > >> > > > > in
> > >> > > > > > the log. So, we probably need to guard it with an IBP or
> > >> metadata
> > >> > > > version
> > >> > > > > > and document the impact on downgrade once the new format is
> > >> written
> > >> > > to
> > >> > > > > the
> > >> > > > > > log.
> > >> > > > > >
> > >> > > > > > 71. Hmm, once the marker is written, the partition will
> expect
> > >> the
> > >> > > next
> > >> > > > > > append to be on the next epoch. Does that cover the case you
> > >> > > mentioned?
> > >> > > > > >
> > >> > > > > > 72. Also, just to be clear on the stucked message issue
> > >> described
> > >> > in
> > >> > > > the
> > >> > > > > > motivation. With EoS, we also validate the sequence id for
> > >> > > idempotency.
> > >> > > > > So,
> > >> > > > > > with the current logic, if the producer epoch hasn't been
> > >> bumped on
> > >> > > the
> > >> > > > > > broker, it seems that the stucked message will fail the
> > sequence
> > >> > > > > validation
> > >> > > > > > and will be ignored. If the producer epoch has been bumped,
> we
> > >> > ignore
> > >> > > > the
> > >> > > > > > sequence check and the stuck message could be appended to
> the
> > >> log.
> > >> > > So,
> > >> > > > is
> > >> > > > > > the latter case that we want to guard?
> > >> > > > > >
> > >> > > > > > Thanks,
> > >> > > > > >
> > >> > > > > > Jun
> > >> > > > > >
> > >> > > > > > On Wed, Dec 14, 2022 at 10:44 AM Justine Olshan
> > >> > > > > > <jo...@confluent.io.invalid> wrote:
> > >> > > > > >
> > >> > > > > > > Matthias — thanks again for taking time to look a this.
> You
> > >> said:
> > >> > > > > > >
> > >> > > > > > > > My proposal was only focusing to avoid dangling
> > >> > > > > > >
> > >> > > > > > > transactions if records are added without registered
> > >> partition.
> > >> > --
> > >> > > > > Maybe
> > >> > > > > > >
> > >> > > > > > > you can add a few more details to the KIP about this
> > scenario
> > >> for
> > >> > > > > better
> > >> > > > > > >
> > >> > > > > > > documentation purpose?
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > > I'm not sure I understand what you mean here. The
> motivation
> > >> > > section
> > >> > > > > > > describes two scenarios about how the record can be added
> > >> > without a
> > >> > > > > > > registered partition:
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > > > 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.
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > > > Another way hanging transactions can occur is that a
> > client
> > >> is
> > >> > > > buggy
> > >> > > > > > and
> > >> > > > > > > may somehow try to write to a partition before it adds the
> > >> > > partition
> > >> > > > to
> > >> > > > > > the
> > >> > > > > > > transaction.
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > > For the first example of this would it be helpful to say
> > that
> > >> > this
> > >> > > > > > message
> > >> > > > > > > comes in after the abort, but before the partition is
> added
> > to
> > >> > the
> > >> > > > next
> > >> > > > > > > transaction so it becomes "hanging." Perhaps the next
> > sentence
> > >> > > > > describing
> > >> > > > > > > the message becoming part of the next transaction (a
> > different
> > >> > > case)
> > >> > > > > was
> > >> > > > > > > not properly differentiated.
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > > Jun — thanks for reading the KIP.
> > >> > > > > > >
> > >> > > > > > > 70. The int typing was a concern. Currently we have a
> > >> mechanism
> > >> > in
> > >> > > > > place
> > >> > > > > > to
> > >> > > > > > > fence the final epoch when the epoch is about to overflow
> > and
> > >> > > assign
> > >> > > > a
> > >> > > > > > new
> > >> > > > > > > producer ID with epoch 0. Of course, this is a bit tricky
> > >> when it
> > >> > > > comes
> > >> > > > > > to
> > >> > > > > > > the response back to the client.
> > >> > > > > > > Making this a long could be another option, but I wonder
> are
> > >> > there
> > >> > > > any
> > >> > > > > > > implications on changing this field if the epoch is
> > persisted
> > >> to
> > >> > > > disk?
> > >> > > > > > I'd
> > >> > > > > > > need to check the usages.
> > >> > > > > > >
> > >> > > > > > > 71.This was something Matthias asked about as well. I was
> > >> > > > considering a
> > >> > > > > > > possible edge case where a produce request from a new
> > >> transaction
> > >> > > > > somehow
> > >> > > > > > > gets sent right after the marker is written, but before
> the
> > >> > > producer
> > >> > > > is
> > >> > > > > > > alerted of the newly bumped epoch. In this case, we may
> > >> include
> > >> > > this
> > >> > > > > > record
> > >> > > > > > > when we don't want to. I suppose we could try to do
> > something
> > >> > > client
> > >> > > > > side
> > >> > > > > > > to bump the epoch after sending an endTxn as well in this
> > >> > scenario
> > >> > > —
> > >> > > > > but
> > >> > > > > > I
> > >> > > > > > > wonder how it would work when the server is aborting based
> > on
> > >> a
> > >> > > > > > server-side
> > >> > > > > > > error. I could also be missing something and this scenario
> > is
> > >> > > > actually
> > >> > > > > > not
> > >> > > > > > > possible.
> > >> > > > > > >
> > >> > > > > > > Thanks again to everyone reading and commenting. Let me
> know
> > >> > about
> > >> > > > any
> > >> > > > > > > further questions or comments.
> > >> > > > > > >
> > >> > > > > > > Justine
> > >> > > > > > >
> > >> > > > > > > On Wed, Dec 14, 2022 at 9:41 AM Jun Rao
> > >> <jun@confluent.io.invalid
> > >> > >
> > >> > > > > > wrote:
> > >> > > > > > >
> > >> > > > > > > > Hi, Justine,
> > >> > > > > > > >
> > >> > > > > > > > Thanks for the KIP. A couple of comments.
> > >> > > > > > > >
> > >> > > > > > > > 70. Currently, the producer epoch is an int. I am not
> sure
> > >> if
> > >> > > it's
> > >> > > > > > enough
> > >> > > > > > > > to accommodate all transactions in the lifetime of a
> > >> producer.
> > >> > > > Should
> > >> > > > > > we
> > >> > > > > > > > change that to a long or add a new long field like
> txnId?
> > >> > > > > > > >
> > >> > > > > > > > 71. "it will write the prepare commit message with a
> > bumped
> > >> > epoch
> > >> > > > and
> > >> > > > > > > send
> > >> > > > > > > > WriteTxnMarkerRequests with the bumped epoch." Hmm, the
> > >> epoch
> > >> > is
> > >> > > > > > > associated
> > >> > > > > > > > with the current txn right? So, it seems weird to write
> a
> > >> > commit
> > >> > > > > > message
> > >> > > > > > > > with a bumped epoch. Should we only bump up the epoch in
> > >> > > > > EndTxnResponse
> > >> > > > > > > and
> > >> > > > > > > > rename the field to sth like nextProducerEpoch?
> > >> > > > > > > >
> > >> > > > > > > > Thanks,
> > >> > > > > > > >
> > >> > > > > > > > Jun
> > >> > > > > > > >
> > >> > > > > > > >
> > >> > > > > > > >
> > >> > > > > > > > On Mon, Dec 12, 2022 at 8:54 PM Matthias J. Sax <
> > >> > > mjsax@apache.org>
> > >> > > > > > > wrote:
> > >> > > > > > > >
> > >> > > > > > > > > Thanks for the background.
> > >> > > > > > > > >
> > >> > > > > > > > > 20/30: SGTM. My proposal was only focusing to avoid
> > >> dangling
> > >> > > > > > > > > transactions if records are added without registered
> > >> > partition.
> > >> > > > --
> > >> > > > > > > Maybe
> > >> > > > > > > > > you can add a few more details to the KIP about this
> > >> scenario
> > >> > > for
> > >> > > > > > > better
> > >> > > > > > > > > documentation purpose?
> > >> > > > > > > > >
> > >> > > > > > > > > 40: I think you hit a fair point about race conditions
> > or
> > >> > > client
> > >> > > > > bugs
> > >> > > > > > > > > (incorrectly not bumping the epoch). The
> > >> complexity/confusion
> > >> > > for
> > >> > > > > > using
> > >> > > > > > > > > the bumped epoch I see, is mainly for internal
> > debugging,
> > >> ie,
> > >> > > > > > > inspecting
> > >> > > > > > > > > log segment dumps -- it seems harder to reason about
> the
> > >> > system
> > >> > > > for
> > >> > > > > > us
> > >> > > > > > > > > humans. But if we get better guarantees, it would be
> > >> worth to
> > >> > > use
> > >> > > > > the
> > >> > > > > > > > > bumped epoch.
> > >> > > > > > > > >
> > >> > > > > > > > > 60: as I mentioned already, I don't know the broker
> > >> internals
> > >> > > to
> > >> > > > > > > provide
> > >> > > > > > > > > more input. So if nobody else chimes in, we should
> just
> > >> move
> > >> > > > > forward
> > >> > > > > > > > > with your proposal.
> > >> > > > > > > > >
> > >> > > > > > > > >
> > >> > > > > > > > > -Matthias
> > >> > > > > > > > >
> > >> > > > > > > > >
> > >> > > > > > > > > On 12/6/22 4:22 PM, Justine Olshan wrote:
> > >> > > > > > > > > > Hi all,
> > >> > > > > > > > > > After Artem's questions about error behavior, I've
> > >> > > re-evaluated
> > >> > > > > the
> > >> > > > > > > > > > unknown producer ID exception and had some
> discussions
> > >> > > offline.
> > >> > > > > > > > > >
> > >> > > > > > > > > > I think generally it makes sense to simplify error
> > >> handling
> > >> > > in
> > >> > > > > > cases
> > >> > > > > > > > like
> > >> > > > > > > > > > this and the UNKNOWN_PRODUCER_ID error has a pretty
> > long
> > >> > and
> > >> > > > > > > > complicated
> > >> > > > > > > > > > history. Because of this, I propose adding a new
> error
> > >> code
> > >> > > > > > > > > ABORTABLE_ERROR
> > >> > > > > > > > > > that when encountered by new clients (gated by the
> > >> produce
> > >> > > > > request
> > >> > > > > > > > > version)
> > >> > > > > > > > > > will simply abort the transaction. This allows the
> > >> server
> > >> > to
> > >> > > > have
> > >> > > > > > > some
> > >> > > > > > > > > say
> > >> > > > > > > > > > in whether the client aborts and makes handling much
> > >> > simpler.
> > >> > > > In
> > >> > > > > > the
> > >> > > > > > > > > > future, we can also use this error in other
> situations
> > >> > where
> > >> > > we
> > >> > > > > > want
> > >> > > > > > > to
> > >> > > > > > > > > > abort the transactions. We can even use on other
> apis.
> > >> > > > > > > > > >
> > >> > > > > > > > > > I've added this to the KIP. Let me know if there are
> > any
> > >> > > > > questions
> > >> > > > > > or
> > >> > > > > > > > > > issues.
> > >> > > > > > > > > >
> > >> > > > > > > > > > Justine
> > >> > > > > > > > > >
> > >> > > > > > > > > > On Fri, Dec 2, 2022 at 10:22 AM Justine Olshan <
> > >> > > > > > jolshan@confluent.io
> > >> > > > > > > >
> > >> > > > > > > > > wrote:
> > >> > > > > > > > > >
> > >> > > > > > > > > >> Hey Matthias,
> > >> > > > > > > > > >>
> > >> > > > > > > > > >>
> > >> > > > > > > > > >> 20/30 — Maybe I also didn't express myself clearly.
> > For
> > >> > > older
> > >> > > > > > > clients
> > >> > > > > > > > we
> > >> > > > > > > > > >> don't have a way to distinguish between a previous
> > and
> > >> the
> > >> > > > > current
> > >> > > > > > > > > >> transaction since we don't have the epoch bump.
> This
> > >> means
> > >> > > > that
> > >> > > > > a
> > >> > > > > > > late
> > >> > > > > > > > > >> message from the previous transaction may be added
> to
> > >> the
> > >> > > new
> > >> > > > > one.
> > >> > > > > > > > With
> > >> > > > > > > > > >> older clients — we can't guarantee this won't
> happen
> > >> if we
> > >> > > > > already
> > >> > > > > > > > sent
> > >> > > > > > > > > the
> > >> > > > > > > > > >> addPartitionsToTxn call (why we make changes for
> the
> > >> newer
> > >> > > > > client)
> > >> > > > > > > but
> > >> > > > > > > > > we
> > >> > > > > > > > > >> can at least gate some by ensuring that the
> partition
> > >> has
> > >> > > been
> > >> > > > > > added
> > >> > > > > > > > to
> > >> > > > > > > > > the
> > >> > > > > > > > > >> transaction. The rationale here is that there are
> > >> likely
> > >> > > LESS
> > >> > > > > late
> > >> > > > > > > > > arrivals
> > >> > > > > > > > > >> as time goes on, so hopefully most late arrivals
> will
> > >> come
> > >> > > in
> > >> > > > > > BEFORE
> > >> > > > > > > > the
> > >> > > > > > > > > >> addPartitionsToTxn call. Those that arrive before
> > will
> > >> be
> > >> > > > > properly
> > >> > > > > > > > gated
> > >> > > > > > > > > >> with the describeTransactions approach.
> > >> > > > > > > > > >>
> > >> > > > > > > > > >> If we take the approach you suggested, ANY late
> > arrival
> > >> > > from a
> > >> > > > > > > > previous
> > >> > > > > > > > > >> transaction will be added. And we don't want that.
> I
> > >> also
> > >> > > > don't
> > >> > > > > > see
> > >> > > > > > > > any
> > >> > > > > > > > > >> benefit in sending addPartitionsToTxn over the
> > >> > describeTxns
> > >> > > > > call.
> > >> > > > > > > They
> > >> > > > > > > > > will
> > >> > > > > > > > > >> both be one extra RPC to the Txn coordinator.
> > >> > > > > > > > > >>
> > >> > > > > > > > > >>
> > >> > > > > > > > > >> To be clear — newer clients will use
> > addPartitionsToTxn
> > >> > > > instead
> > >> > > > > of
> > >> > > > > > > the
> > >> > > > > > > > > >> DescribeTxns.
> > >> > > > > > > > > >>
> > >> > > > > > > > > >>
> > >> > > > > > > > > >> 40)
> > >> > > > > > > > > >> My concern is that if we have some delay in the
> > client
> > >> to
> > >> > > bump
> > >> > > > > the
> > >> > > > > > > > > epoch,
> > >> > > > > > > > > >> it could continue to send epoch 73 and those
> records
> > >> would
> > >> > > not
> > >> > > > > be
> > >> > > > > > > > > fenced.
> > >> > > > > > > > > >> Perhaps this is not an issue if we don't allow the
> > next
> > >> > > > produce
> > >> > > > > to
> > >> > > > > > > go
> > >> > > > > > > > > >> through before the EndTxn request returns. I'm also
> > >> > thinking
> > >> > > > > about
> > >> > > > > > > > > cases of
> > >> > > > > > > > > >> failure. I will need to think on this a bit.
> > >> > > > > > > > > >>
> > >> > > > > > > > > >> I wasn't sure if it was that confusing. But if we
> > >> think it
> > >> > > is,
> > >> > > > > we
> > >> > > > > > > can
> > >> > > > > > > > > >> investigate other ways.
> > >> > > > > > > > > >>
> > >> > > > > > > > > >>
> > >> > > > > > > > > >> 60)
> > >> > > > > > > > > >>
> > >> > > > > > > > > >> I'm not sure these are the same purgatories since
> one
> > >> is a
> > >> > > > > produce
> > >> > > > > > > > > >> purgatory (I was planning on using a callback
> rather
> > >> than
> > >> > > > > > purgatory)
> > >> > > > > > > > and
> > >> > > > > > > > > >> the other is simply a request to append to the log.
> > Not
> > >> > sure
> > >> > > > we
> > >> > > > > > have
> > >> > > > > > > > any
> > >> > > > > > > > > >> structure here for ordering, but my understanding
> is
> > >> that
> > >> > > the
> > >> > > > > > broker
> > >> > > > > > > > > could
> > >> > > > > > > > > >> handle the write request before it hears back from
> > the
> > >> Txn
> > >> > > > > > > > Coordinator.
> > >> > > > > > > > > >>
> > >> > > > > > > > > >> Let me know if I misunderstood something or
> something
> > >> was
> > >> > > > > unclear.
> > >> > > > > > > > > >>
> > >> > > > > > > > > >> Justine
> > >> > > > > > > > > >>
> > >> > > > > > > > > >> On Thu, Dec 1, 2022 at 12:15 PM Matthias J. Sax <
> > >> > > > > mjsax@apache.org
> > >> > > > > > >
> > >> > > > > > > > > wrote:
> > >> > > > > > > > > >>
> > >> > > > > > > > > >>> Thanks for the details Justine!
> > >> > > > > > > > > >>>
> > >> > > > > > > > > >>>> 20)
> > >> > > > > > > > > >>>>
> > >> > > > > > > > > >>>> The client side change for 2 is removing the
> > >> > addPartitions
> > >> > > > to
> > >> > > > > > > > > >>> transaction
> > >> > > > > > > > > >>>> call. We don't need to make this from the
> producer
> > to
> > >> > the
> > >> > > > txn
> > >> > > > > > > > > >>> coordinator,
> > >> > > > > > > > > >>>> only server side.
> > >> > > > > > > > > >>>
> > >> > > > > > > > > >>> I think I did not express myself clearly. I
> > understand
> > >> > that
> > >> > > > we
> > >> > > > > > can
> > >> > > > > > > > (and
> > >> > > > > > > > > >>> should) change the producer to not send the
> > >> > `addPartitions`
> > >> > > > > > request
> > >> > > > > > > > any
> > >> > > > > > > > > >>> longer. But I don't thinks it's requirement to
> > change
> > >> the
> > >> > > > > broker?
> > >> > > > > > > > > >>>
> > >> > > > > > > > > >>> What I am trying to say is: as a safe-guard and
> > >> > improvement
> > >> > > > for
> > >> > > > > > > older
> > >> > > > > > > > > >>> producers, the partition leader can just send the
> > >> > > > > `addPartitions`
> > >> > > > > > > > > >>> request to the TX-coordinator in any case -- if
> the
> > >> old
> > >> > > > > producer
> > >> > > > > > > > > >>> correctly did send the `addPartition` request to
> the
> > >> > > > > > TX-coordinator
> > >> > > > > > > > > >>> already, the TX-coordinator can just "ignore" is
> as
> > >> > > > idempotent.
> > >> > > > > > > > > However,
> > >> > > > > > > > > >>> if the old producer has a bug and did forget to
> sent
> > >> the
> > >> > > > > > > > `addPartition`
> > >> > > > > > > > > >>> request, we would now ensure that the partition is
> > >> indeed
> > >> > > > added
> > >> > > > > > to
> > >> > > > > > > > the
> > >> > > > > > > > > >>> TX and thus fix a potential producer bug (even if
> we
> > >> > don't
> > >> > > > get
> > >> > > > > > the
> > >> > > > > > > > > >>> fencing via the bump epoch). -- It seems to be a
> > good
> > >> > > > > > improvement?
> > >> > > > > > > Or
> > >> > > > > > > > > is
> > >> > > > > > > > > >>> there a reason to not do this?
> > >> > > > > > > > > >>>
> > >> > > > > > > > > >>>
> > >> > > > > > > > > >>>
> > >> > > > > > > > > >>>> 30)
> > >> > > > > > > > > >>>>
> > >> > > > > > > > > >>>> Transaction is ongoing = partition was added to
> > >> > > transaction
> > >> > > > > via
> > >> > > > > > > > > >>>> addPartitionsToTxn. We check this with the
> > >> > > > > DescribeTransactions
> > >> > > > > > > > call.
> > >> > > > > > > > > >>> Let
> > >> > > > > > > > > >>>> me know if this wasn't sufficiently explained
> here:
> > >> > > > > > > > > >>>
> > >> > > > > > > > > >>> If we do what I propose in (20), we don't really
> > need
> > >> to
> > >> > > make
> > >> > > > > > this
> > >> > > > > > > > > >>> `DescribeTransaction` call, as the partition
> leader
> > >> adds
> > >> > > the
> > >> > > > > > > > partition
> > >> > > > > > > > > >>> for older clients and we get this check for free.
> > >> > > > > > > > > >>>
> > >> > > > > > > > > >>>
> > >> > > > > > > > > >>>> 40)
> > >> > > > > > > > > >>>>
> > >> > > > > > > > > >>>> The idea here is that if any messages somehow
> come
> > in
> > >> > > before
> > >> > > > > we
> > >> > > > > > > get
> > >> > > > > > > > > the
> > >> > > > > > > > > >>> new
> > >> > > > > > > > > >>>> epoch to the producer, they will be fenced.
> > However,
> > >> if
> > >> > we
> > >> > > > > don't
> > >> > > > > > > > think
> > >> > > > > > > > > >>> this
> > >> > > > > > > > > >>>> is necessary, it can be discussed
> > >> > > > > > > > > >>>
> > >> > > > > > > > > >>> I agree that we should have epoch fencing. My
> > >> question is
> > >> > > > > > > different:
> > >> > > > > > > > > >>> Assume we are at epoch 73, and we have an ongoing
> > >> > > > transaction,
> > >> > > > > > that
> > >> > > > > > > > is
> > >> > > > > > > > > >>> committed. It seems natural to write the "prepare
> > >> commit"
> > >> > > > > marker
> > >> > > > > > > and
> > >> > > > > > > > > the
> > >> > > > > > > > > >>> `WriteTxMarkerRequest` both with epoch 73, too, as
> > it
> > >> > > belongs
> > >> > > > > to
> > >> > > > > > > the
> > >> > > > > > > > > >>> current transaction. Of course, we now also bump
> the
> > >> > epoch
> > >> > > > and
> > >> > > > > > > expect
> > >> > > > > > > > > >>> the next requests to have epoch 74, and would
> reject
> > >> an
> > >> > > > request
> > >> > > > > > > with
> > >> > > > > > > > > >>> epoch 73, as the corresponding TX for epoch 73 was
> > >> > already
> > >> > > > > > > committed.
> > >> > > > > > > > > >>>
> > >> > > > > > > > > >>> It seems you propose to write the "prepare commit
> > >> marker"
> > >> > > and
> > >> > > > > > > > > >>> `WriteTxMarkerRequest` with epoch 74 though, what
> > >> would
> > >> > > work,
> > >> > > > > but
> > >> > > > > > > it
> > >> > > > > > > > > >>> seems confusing. Is there a reason why we would
> use
> > >> the
> > >> > > > bumped
> > >> > > > > > > epoch
> > >> > > > > > > > 74
> > >> > > > > > > > > >>> instead of the current epoch 73?
> > >> > > > > > > > > >>>
> > >> > > > > > > > > >>>
> > >> > > > > > > > > >>>> 60)
> > >> > > > > > > > > >>>>
> > >> > > > > > > > > >>>> When we are checking if the transaction is
> ongoing,
> > >> we
> > >> > > need
> > >> > > > to
> > >> > > > > > > make
> > >> > > > > > > > a
> > >> > > > > > > > > >>> round
> > >> > > > > > > > > >>>> trip from the leader partition to the transaction
> > >> > > > coordinator.
> > >> > > > > > In
> > >> > > > > > > > the
> > >> > > > > > > > > >>> time
> > >> > > > > > > > > >>>> we are waiting for this message to come back, in
> > >> theory
> > >> > we
> > >> > > > > could
> > >> > > > > > > > have
> > >> > > > > > > > > >>> sent
> > >> > > > > > > > > >>>> a commit/abort call that would make the original
> > >> result
> > >> > of
> > >> > > > the
> > >> > > > > > > check
> > >> > > > > > > > > >>> out of
> > >> > > > > > > > > >>>> date. That is why we can check the leader state
> > >> before
> > >> > we
> > >> > > > > write
> > >> > > > > > to
> > >> > > > > > > > the
> > >> > > > > > > > > >>> log.
> > >> > > > > > > > > >>>
> > >> > > > > > > > > >>> Thanks. Got it.
> > >> > > > > > > > > >>>
> > >> > > > > > > > > >>> However, is this really an issue? We put the
> produce
> > >> > > request
> > >> > > > in
> > >> > > > > > > > > >>> purgatory, so how could we process the
> > >> > > > `WriteTxnMarkerRequest`
> > >> > > > > > > first?
> > >> > > > > > > > > >>> Don't we need to put the `WriteTxnMarkerRequest`
> > into
> > >> > > > > purgatory,
> > >> > > > > > > too,
> > >> > > > > > > > > >>> for this case, and process both request in-order?
> > >> (Again,
> > >> > > my
> > >> > > > > > broker
> > >> > > > > > > > > >>> knowledge is limited and maybe we don't maintain
> > >> request
> > >> > > > order
> > >> > > > > > for
> > >> > > > > > > > this
> > >> > > > > > > > > >>> case, what seems to be an issue IMHO, and I am
> > >> wondering
> > >> > if
> > >> > > > > > > changing
> > >> > > > > > > > > >>> request handling to preserve order for this case
> > >> might be
> > >> > > the
> > >> > > > > > > cleaner
> > >> > > > > > > > > >>> solution?)
> > >> > > > > > > > > >>>
> > >> > > > > > > > > >>>
> > >> > > > > > > > > >>>
> > >> > > > > > > > > >>> -Matthias
> > >> > > > > > > > > >>>
> > >> > > > > > > > > >>>
> > >> > > > > > > > > >>>
> > >> > > > > > > > > >>>
> > >> > > > > > > > > >>> On 11/30/22 3:28 PM, Artem Livshits wrote:
> > >> > > > > > > > > >>>> Hi Justine,
> > >> > > > > > > > > >>>>
> > >> > > > > > > > > >>>> I think the interesting part is not in this logic
> > >> > (because
> > >> > > > it
> > >> > > > > > > tries
> > >> > > > > > > > to
> > >> > > > > > > > > >>>> figure out when UNKNOWN_PRODUCER_ID is retriable
> > and
> > >> if
> > >> > > it's
> > >> > > > > > > > > retryable,
> > >> > > > > > > > > >>>> it's definitely not fatal), but what happens when
> > >> this
> > >> > > logic
> > >> > > > > > > doesn't
> > >> > > > > > > > > >>> return
> > >> > > > > > > > > >>>> 'true' and falls through.  In the old clients it
> > >> seems
> > >> > to
> > >> > > be
> > >> > > > > > > fatal,
> > >> > > > > > > > if
> > >> > > > > > > > > >>> we
> > >> > > > > > > > > >>>> keep the behavior in the new clients, I'd expect
> it
> > >> > would
> > >> > > be
> > >> > > > > > fatal
> > >> > > > > > > > as
> > >> > > > > > > > > >>> well.
> > >> > > > > > > > > >>>>
> > >> > > > > > > > > >>>> -Artem
> > >> > > > > > > > > >>>>
> > >> > > > > > > > > >>>> On Tue, Nov 29, 2022 at 11:57 AM Justine Olshan
> > >> > > > > > > > > >>>> <jo...@confluent.io.invalid> wrote:
> > >> > > > > > > > > >>>>
> > >> > > > > > > > > >>>>> Hi Artem and Jeff,
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>> Thanks for taking a look and sorry for the slow
> > >> > response.
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>> You both mentioned the change to handle
> > >> > > UNKNOWN_PRODUCER_ID
> > >> > > > > > > errors.
> > >> > > > > > > > > To
> > >> > > > > > > > > >>> be
> > >> > > > > > > > > >>>>> clear — this error code will only be sent again
> > when
> > >> > the
> > >> > > > > > client's
> > >> > > > > > > > > >>> request
> > >> > > > > > > > > >>>>> version is high enough to ensure we handle it
> > >> > correctly.
> > >> > > > > > > > > >>>>> The current (Java) client handles this by the
> > >> following
> > >> > > > > > (somewhat
> > >> > > > > > > > > long)
> > >> > > > > > > > > >>>>> code snippet:
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>> // An UNKNOWN_PRODUCER_ID means that we have
> lost
> > >> the
> > >> > > > > producer
> > >> > > > > > > > state
> > >> > > > > > > > > >>> on the
> > >> > > > > > > > > >>>>> broker. Depending on the log start
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>> // offset, we may want to retry these, as
> > described
> > >> for
> > >> > > > each
> > >> > > > > > case
> > >> > > > > > > > > >>> below. If
> > >> > > > > > > > > >>>>> none of those apply, then for the
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>> // idempotent producer, we will locally bump the
> > >> epoch
> > >> > > and
> > >> > > > > > reset
> > >> > > > > > > > the
> > >> > > > > > > > > >>>>> sequence numbers of in-flight batches from
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>> // sequence 0, then retry the failed batch,
> which
> > >> > should
> > >> > > > now
> > >> > > > > > > > succeed.
> > >> > > > > > > > > >>> For
> > >> > > > > > > > > >>>>> the transactional producer, allow the
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>> // batch to fail. When processing the failed
> > batch,
> > >> we
> > >> > > will
> > >> > > > > > > > > transition
> > >> > > > > > > > > >>> to
> > >> > > > > > > > > >>>>> an abortable error and set a flag
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>> // indicating that we need to bump the epoch (if
> > >> > > supported
> > >> > > > by
> > >> > > > > > the
> > >> > > > > > > > > >>> broker).
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>> if (error == Errors.*UNKNOWN_PRODUCER_ID*) {
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>       if (response.logStartOffset == -1) {
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           // We don't know the log start offset
> > with
> > >> > this
> > >> > > > > > > response.
> > >> > > > > > > > > We
> > >> > > > > > > > > >>> should
> > >> > > > > > > > > >>>>> just retry the request until we get it.
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           // The UNKNOWN_PRODUCER_ID error code
> > was
> > >> > added
> > >> > > > > along
> > >> > > > > > > > with
> > >> > > > > > > > > >>> the new
> > >> > > > > > > > > >>>>> ProduceResponse which includes the
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           // logStartOffset. So the '-1'
> sentinel
> > is
> > >> > not
> > >> > > > for
> > >> > > > > > > > backward
> > >> > > > > > > > > >>>>> compatibility. Instead, it is possible for
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           // a broker to not know the
> > >> logStartOffset at
> > >> > > > when
> > >> > > > > it
> > >> > > > > > > is
> > >> > > > > > > > > >>> returning
> > >> > > > > > > > > >>>>> the response because the partition
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           // may have moved away from the broker
> > >> from
> > >> > the
> > >> > > > > time
> > >> > > > > > > the
> > >> > > > > > > > > >>> error was
> > >> > > > > > > > > >>>>> initially raised to the time the
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           // response was being constructed. In
> > >> these
> > >> > > > cases,
> > >> > > > > we
> > >> > > > > > > > > should
> > >> > > > > > > > > >>> just
> > >> > > > > > > > > >>>>> retry the request: we are guaranteed
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           // to eventually get a logStartOffset
> > once
> > >> > > things
> > >> > > > > > > settle
> > >> > > > > > > > > down.
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           return true;
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>       }
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>       if (batch.sequenceHasBeenReset()) {
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           // When the first inflight batch fails
> > >> due to
> > >> > > the
> > >> > > > > > > > > truncation
> > >> > > > > > > > > >>> case,
> > >> > > > > > > > > >>>>> then the sequences of all the other
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           // in flight batches would have been
> > >> > restarted
> > >> > > > from
> > >> > > > > > the
> > >> > > > > > > > > >>> beginning.
> > >> > > > > > > > > >>>>> However, when those responses
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           // come back from the broker, they
> would
> > >> also
> > >> > > > come
> > >> > > > > > with
> > >> > > > > > > > an
> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID error. In this case, we
> should
> > >> not
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           // reset the sequence numbers to the
> > >> > beginning.
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           return true;
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>       } else if
> > >> > > > > (lastAckedOffset(batch.topicPartition).orElse(
> > >> > > > > > > > > >>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > >> > > > response.logStartOffset) {
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           // The head of the log has been
> removed,
> > >> > > probably
> > >> > > > > due
> > >> > > > > > > to
> > >> > > > > > > > > the
> > >> > > > > > > > > >>>>> retention time elapsing. In this case,
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           // we expect to lose the producer
> state.
> > >> For
> > >> > > the
> > >> > > > > > > > > transactional
> > >> > > > > > > > > >>>>> producer, reset the sequences of all
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           // inflight batches to be from the
> > >> beginning
> > >> > > and
> > >> > > > > > retry
> > >> > > > > > > > > them,
> > >> > > > > > > > > >>> so
> > >> > > > > > > > > >>>>> that the transaction does not need to
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           // be aborted. For the idempotent
> > >> producer,
> > >> > > bump
> > >> > > > > the
> > >> > > > > > > > epoch
> > >> > > > > > > > > to
> > >> > > > > > > > > >>> avoid
> > >> > > > > > > > > >>>>> reusing (sequence, epoch) pairs
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           if (isTransactional()) {
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>
> > >> > > > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > >> > > > > > > > > >>>>> this.producerIdAndEpoch);
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           } else {
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>
> > >> > > > > >  requestEpochBumpForPartition(batch.topicPartition);
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           }
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           return true;
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>       }
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>       if (!isTransactional()) {
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           // For the idempotent producer, always
> > >> retry
> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > >> > > > > > > > > >>>>> errors. If the batch has the current
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           // producer ID and epoch, request a
> bump
> > >> of
> > >> > the
> > >> > > > > > epoch.
> > >> > > > > > > > > >>> Otherwise
> > >> > > > > > > > > >>>>> just retry the produce.
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>
> > >> > > >  requestEpochBumpForPartition(batch.topicPartition);
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>           return true;
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>       }
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>> }
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>> I was considering keeping this behavior — but am
> > >> open
> > >> > to
> > >> > > > > > > > simplifying
> > >> > > > > > > > > >>> it.
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>> We are leaving changes to older clients off the
> > >> table
> > >> > > here
> > >> > > > > > since
> > >> > > > > > > it
> > >> > > > > > > > > >>> caused
> > >> > > > > > > > > >>>>> many issues for clients in the past. Previously
> > this
> > >> > was
> > >> > > a
> > >> > > > > > fatal
> > >> > > > > > > > > error
> > >> > > > > > > > > >>> and
> > >> > > > > > > > > >>>>> we didn't have the mechanisms in place to detect
> > >> when
> > >> > > this
> > >> > > > > was
> > >> > > > > > a
> > >> > > > > > > > > >>> legitimate
> > >> > > > > > > > > >>>>> case vs some bug or gap in the protocol.
> Ensuring
> > >> each
> > >> > > > > > > transaction
> > >> > > > > > > > > has
> > >> > > > > > > > > >>> its
> > >> > > > > > > > > >>>>> own epoch should close this gap.
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>> And to address Jeff's second point:
> > >> > > > > > > > > >>>>> *does the typical produce request path append
> > >> records
> > >> > to
> > >> > > > > local
> > >> > > > > > > log
> > >> > > > > > > > > >>> along*
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>> *with the currentTxnFirstOffset information? I
> > would
> > >> > like
> > >> > > > to
> > >> > > > > > > > > >>> understand*
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>> *when the field is written to disk.*
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>> Yes, the first produce request populates this
> > field
> > >> and
> > >> > > > > writes
> > >> > > > > > > the
> > >> > > > > > > > > >>> offset
> > >> > > > > > > > > >>>>> as part of the record batch and also to the
> > producer
> > >> > > state
> > >> > > > > > > > snapshot.
> > >> > > > > > > > > >>> When
> > >> > > > > > > > > >>>>> we reload the records on restart and/or
> > >> reassignment,
> > >> > we
> > >> > > > > > > repopulate
> > >> > > > > > > > > >>> this
> > >> > > > > > > > > >>>>> field with the snapshot from disk along with the
> > >> rest
> > >> > of
> > >> > > > the
> > >> > > > > > > > producer
> > >> > > > > > > > > >>>>> state.
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>> Let me know if there are further comments and/or
> > >> > > questions.
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>> Thanks,
> > >> > > > > > > > > >>>>> Justine
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>> On Tue, Nov 22, 2022 at 9:00 PM Jeff Kim
> > >> > > > > > > > > <jeff.kim@confluent.io.invalid
> > >> > > > > > > > > >>>>
> > >> > > > > > > > > >>>>> wrote:
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>>> Hi Justine,
> > >> > > > > > > > > >>>>>>
> > >> > > > > > > > > >>>>>> Thanks for the KIP! I have two questions:
> > >> > > > > > > > > >>>>>>
> > >> > > > > > > > > >>>>>> 1) For new clients, we can once again return an
> > >> error
> > >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > >> > > > > > > > > >>>>>> for sequences
> > >> > > > > > > > > >>>>>> that are non-zero when there is no producer
> state
> > >> > > present
> > >> > > > on
> > >> > > > > > the
> > >> > > > > > > > > >>> server.
> > >> > > > > > > > > >>>>>> This will indicate we missed the 0 sequence and
> > we
> > >> > don't
> > >> > > > yet
> > >> > > > > > > want
> > >> > > > > > > > to
> > >> > > > > > > > > >>>>> write
> > >> > > > > > > > > >>>>>> to the log.
> > >> > > > > > > > > >>>>>>
> > >> > > > > > > > > >>>>>> I would like to understand the current behavior
> > to
> > >> > > handle
> > >> > > > > > older
> > >> > > > > > > > > >>> clients,
> > >> > > > > > > > > >>>>>> and if there are any changes we are making.
> Maybe
> > >> I'm
> > >> > > > > missing
> > >> > > > > > > > > >>> something,
> > >> > > > > > > > > >>>>>> but we would want to identify whether we missed
> > >> the 0
> > >> > > > > sequence
> > >> > > > > > > for
> > >> > > > > > > > > >>> older
> > >> > > > > > > > > >>>>>> clients, no?
> > >> > > > > > > > > >>>>>>
> > >> > > > > > > > > >>>>>> 2) Upon returning from the transaction
> > >> coordinator, we
> > >> > > can
> > >> > > > > set
> > >> > > > > > > the
> > >> > > > > > > > > >>>>>> transaction
> > >> > > > > > > > > >>>>>> as ongoing on the leader by populating
> > >> > > > currentTxnFirstOffset
> > >> > > > > > > > > >>>>>> through the typical produce request handling.
> > >> > > > > > > > > >>>>>>
> > >> > > > > > > > > >>>>>> does the typical produce request path append
> > >> records
> > >> > to
> > >> > > > > local
> > >> > > > > > > log
> > >> > > > > > > > > >>> along
> > >> > > > > > > > > >>>>>> with the currentTxnFirstOffset information? I
> > would
> > >> > like
> > >> > > > to
> > >> > > > > > > > > understand
> > >> > > > > > > > > >>>>>> when the field is written to disk.
> > >> > > > > > > > > >>>>>>
> > >> > > > > > > > > >>>>>> Thanks,
> > >> > > > > > > > > >>>>>> Jeff
> > >> > > > > > > > > >>>>>>
> > >> > > > > > > > > >>>>>>
> > >> > > > > > > > > >>>>>> On Tue, Nov 22, 2022 at 4:44 PM Artem Livshits
> > >> > > > > > > > > >>>>>> <al...@confluent.io.invalid> wrote:
> > >> > > > > > > > > >>>>>>
> > >> > > > > > > > > >>>>>>> Hi Justine,
> > >> > > > > > > > > >>>>>>>
> > >> > > > > > > > > >>>>>>> Thank you for the KIP.  I have one question.
> > >> > > > > > > > > >>>>>>>
> > >> > > > > > > > > >>>>>>> 5) For new clients, we can once again return
> an
> > >> error
> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > >> > > > > > > > > >>>>>>>
> > >> > > > > > > > > >>>>>>> I believe we had problems in the past with
> > >> returning
> > >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > >> > > > > > > > > >>>>>>> because it was considered fatal and required
> > >> client
> > >> > > > > restart.
> > >> > > > > > > It
> > >> > > > > > > > > >>> would
> > >> > > > > > > > > >>>>> be
> > >> > > > > > > > > >>>>>>> good to spell out the new client behavior when
> > it
> > >> > > > receives
> > >> > > > > > the
> > >> > > > > > > > > error.
> > >> > > > > > > > > >>>>>>>
> > >> > > > > > > > > >>>>>>> -Artem
> > >> > > > > > > > > >>>>>>>
> > >> > > > > > > > > >>>>>>> On Tue, Nov 22, 2022 at 10:00 AM Justine
> Olshan
> > >> > > > > > > > > >>>>>>> <jo...@confluent.io.invalid> wrote:
> > >> > > > > > > > > >>>>>>>
> > >> > > > > > > > > >>>>>>>> Thanks for taking a look Matthias. I've tried
> > to
> > >> > > answer
> > >> > > > > your
> > >> > > > > > > > > >>>>> questions
> > >> > > > > > > > > >>>>>>>> below:
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>> 10)
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>> Right — so the hanging transaction only
> occurs
> > >> when
> > >> > we
> > >> > > > > have
> > >> > > > > > a
> > >> > > > > > > > late
> > >> > > > > > > > > >>>>>>> message
> > >> > > > > > > > > >>>>>>>> come in and the partition is never added to a
> > >> > > > transaction
> > >> > > > > > > again.
> > >> > > > > > > > > If
> > >> > > > > > > > > >>>>> we
> > >> > > > > > > > > >>>>>>>> never add the partition to a transaction, we
> > will
> > >> > > never
> > >> > > > > > write
> > >> > > > > > > a
> > >> > > > > > > > > >>>>> marker
> > >> > > > > > > > > >>>>>>> and
> > >> > > > > > > > > >>>>>>>> never advance the LSO.
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>> If we do end up adding the partition to the
> > >> > > transaction
> > >> > > > (I
> > >> > > > > > > > suppose
> > >> > > > > > > > > >>>>> this
> > >> > > > > > > > > >>>>>>> can
> > >> > > > > > > > > >>>>>>>> happen before or after the late message comes
> > in)
> > >> > then
> > >> > > > we
> > >> > > > > > will
> > >> > > > > > > > > >>>>> include
> > >> > > > > > > > > >>>>>>> the
> > >> > > > > > > > > >>>>>>>> late message in the next (incorrect)
> > transaction.
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>> So perhaps it is clearer to make the
> > distinction
> > >> > > between
> > >> > > > > > > > messages
> > >> > > > > > > > > >>>>> that
> > >> > > > > > > > > >>>>>>>> eventually get added to the transaction (but
> > the
> > >> > wrong
> > >> > > > > one)
> > >> > > > > > or
> > >> > > > > > > > > >>>>> messages
> > >> > > > > > > > > >>>>>>>> that never get added and become hanging.
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>> 20)
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>> The client side change for 2 is removing the
> > >> > > > addPartitions
> > >> > > > > > to
> > >> > > > > > > > > >>>>>> transaction
> > >> > > > > > > > > >>>>>>>> call. We don't need to make this from the
> > >> producer
> > >> > to
> > >> > > > the
> > >> > > > > > txn
> > >> > > > > > > > > >>>>>>> coordinator,
> > >> > > > > > > > > >>>>>>>> only server side.
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>> In my opinion, the issue with the
> > >> addPartitionsToTxn
> > >> > > > call
> > >> > > > > > for
> > >> > > > > > > > > older
> > >> > > > > > > > > >>>>>>> clients
> > >> > > > > > > > > >>>>>>>> is that we don't have the epoch bump, so we
> > don't
> > >> > know
> > >> > > > if
> > >> > > > > > the
> > >> > > > > > > > > >>> message
> > >> > > > > > > > > >>>>>>>> belongs to the previous transaction or this
> > one.
> > >> We
> > >> > > need
> > >> > > > > to
> > >> > > > > > > > check
> > >> > > > > > > > > if
> > >> > > > > > > > > >>>>>> the
> > >> > > > > > > > > >>>>>>>> partition has been added to this transaction.
> > Of
> > >> > > course,
> > >> > > > > > this
> > >> > > > > > > > > means
> > >> > > > > > > > > >>>>> we
> > >> > > > > > > > > >>>>>>>> won't completely cover the case where we
> have a
> > >> > really
> > >> > > > > late
> > >> > > > > > > > > message
> > >> > > > > > > > > >>>>> and
> > >> > > > > > > > > >>>>>>> we
> > >> > > > > > > > > >>>>>>>> have added the partition to the new
> > transaction,
> > >> but
> > >> > > > > that's
> > >> > > > > > > > > >>>>>> unfortunately
> > >> > > > > > > > > >>>>>>>> something we will need the new clients to
> > cover.
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>> 30)
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>> Transaction is ongoing = partition was added
> to
> > >> > > > > transaction
> > >> > > > > > > via
> > >> > > > > > > > > >>>>>>>> addPartitionsToTxn. We check this with the
> > >> > > > > > > DescribeTransactions
> > >> > > > > > > > > >>> call.
> > >> > > > > > > > > >>>>>> Let
> > >> > > > > > > > > >>>>>>>> me know if this wasn't sufficiently explained
> > >> here:
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>
> > >> > > > > > > > > >>>>>>
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>
> > >> > > > > > > > >
> > >> > > > > > > >
> > >> > > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>> 40)
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>> The idea here is that if any messages somehow
> > >> come
> > >> > in
> > >> > > > > before
> > >> > > > > > > we
> > >> > > > > > > > > get
> > >> > > > > > > > > >>>>> the
> > >> > > > > > > > > >>>>>>> new
> > >> > > > > > > > > >>>>>>>> epoch to the producer, they will be fenced.
> > >> However,
> > >> > > if
> > >> > > > we
> > >> > > > > > > don't
> > >> > > > > > > > > >>>>> think
> > >> > > > > > > > > >>>>>>> this
> > >> > > > > > > > > >>>>>>>> is necessary, it can be discussed
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>> 50)
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>> It should be synchronous because if we have
> an
> > >> event
> > >> > > > (ie,
> > >> > > > > an
> > >> > > > > > > > > error)
> > >> > > > > > > > > >>>>>> that
> > >> > > > > > > > > >>>>>>>> causes us to need to abort the transaction,
> we
> > >> need
> > >> > to
> > >> > > > > know
> > >> > > > > > > > which
> > >> > > > > > > > > >>>>>>>> partitions to send transaction markers to. We
> > >> know
> > >> > the
> > >> > > > > > > > partitions
> > >> > > > > > > > > >>>>>> because
> > >> > > > > > > > > >>>>>>>> we added them to the coordinator via the
> > >> > > > > addPartitionsToTxn
> > >> > > > > > > > call.
> > >> > > > > > > > > >>>>>>>> Previously we have had asynchronous calls in
> > the
> > >> > past
> > >> > > > (ie,
> > >> > > > > > > > writing
> > >> > > > > > > > > >>>>> the
> > >> > > > > > > > > >>>>>>>> commit markers when the transaction is
> > completed)
> > >> > but
> > >> > > > > often
> > >> > > > > > > this
> > >> > > > > > > > > >>> just
> > >> > > > > > > > > >>>>>>>> causes confusion as we need to wait for some
> > >> > > operations
> > >> > > > to
> > >> > > > > > > > > complete.
> > >> > > > > > > > > >>>>> In
> > >> > > > > > > > > >>>>>>> the
> > >> > > > > > > > > >>>>>>>> writing commit markers case, clients often
> see
> > >> > > > > > > > > >>>>> CONCURRENT_TRANSACTIONs
> > >> > > > > > > > > >>>>>>>> error messages and that can be confusing. For
> > >> that
> > >> > > > reason,
> > >> > > > > > it
> > >> > > > > > > > may
> > >> > > > > > > > > be
> > >> > > > > > > > > >>>>>>>> simpler to just have synchronous calls —
> > >> especially
> > >> > if
> > >> > > > we
> > >> > > > > > need
> > >> > > > > > > > to
> > >> > > > > > > > > >>>>> block
> > >> > > > > > > > > >>>>>>> on
> > >> > > > > > > > > >>>>>>>> some operation's completion anyway before we
> > can
> > >> > start
> > >> > > > the
> > >> > > > > > > next
> > >> > > > > > > > > >>>>>>>> transaction. And yes, I meant coordinator. I
> > will
> > >> > fix
> > >> > > > > that.
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>> 60)
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>> When we are checking if the transaction is
> > >> ongoing,
> > >> > we
> > >> > > > > need
> > >> > > > > > to
> > >> > > > > > > > > make
> > >> > > > > > > > > >>> a
> > >> > > > > > > > > >>>>>>> round
> > >> > > > > > > > > >>>>>>>> trip from the leader partition to the
> > transaction
> > >> > > > > > coordinator.
> > >> > > > > > > > In
> > >> > > > > > > > > >>> the
> > >> > > > > > > > > >>>>>>> time
> > >> > > > > > > > > >>>>>>>> we are waiting for this message to come back,
> > in
> > >> > > theory
> > >> > > > we
> > >> > > > > > > could
> > >> > > > > > > > > >>> have
> > >> > > > > > > > > >>>>>>> sent
> > >> > > > > > > > > >>>>>>>> a commit/abort call that would make the
> > original
> > >> > > result
> > >> > > > of
> > >> > > > > > the
> > >> > > > > > > > > check
> > >> > > > > > > > > >>>>>> out
> > >> > > > > > > > > >>>>>>> of
> > >> > > > > > > > > >>>>>>>> date. That is why we can check the leader
> state
> > >> > before
> > >> > > > we
> > >> > > > > > > write
> > >> > > > > > > > to
> > >> > > > > > > > > >>>>> the
> > >> > > > > > > > > >>>>>>> log.
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>> I'm happy to update the KIP if some of these
> > >> things
> > >> > > were
> > >> > > > > not
> > >> > > > > > > > > clear.
> > >> > > > > > > > > >>>>>>>> Thanks,
> > >> > > > > > > > > >>>>>>>> Justine
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>> On Mon, Nov 21, 2022 at 7:11 PM Matthias J.
> > Sax <
> > >> > > > > > > > mjsax@apache.org
> > >> > > > > > > > > >
> > >> > > > > > > > > >>>>>>> wrote:
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>>> Thanks for the KIP.
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>> Couple of clarification questions (I am not
> a
> > >> > broker
> > >> > > > > expert
> > >> > > > > > > do
> > >> > > > > > > > > >>>>> maybe
> > >> > > > > > > > > >>>>>>>>> some question are obvious for others, but
> not
> > >> for
> > >> > me
> > >> > > > with
> > >> > > > > > my
> > >> > > > > > > > lack
> > >> > > > > > > > > >>>>> of
> > >> > > > > > > > > >>>>>>>>> broker knowledge).
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>> (10)
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>> 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.
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>> What happens if the message come in before
> the
> > >> next
> > >> > > > > > > > > >>>>>> addPartitionsToTxn
> > >> > > > > > > > > >>>>>>>>> request? It seems the broker hosting the
> data
> > >> > > > partitions
> > >> > > > > > > won't
> > >> > > > > > > > > know
> > >> > > > > > > > > >>>>>>>>> anything about it and append it to the
> > >> partition,
> > >> > > too?
> > >> > > > > What
> > >> > > > > > > is
> > >> > > > > > > > > the
> > >> > > > > > > > > >>>>>>>>> difference between both cases?
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>> Also, it seems a TX would only hang, if
> there
> > >> is no
> > >> > > > > > following
> > >> > > > > > > > TX
> > >> > > > > > > > > >>>>> that
> > >> > > > > > > > > >>>>>>> is
> > >> > > > > > > > > >>>>>>>>> either committer or aborted? Thus, for the
> > case
> > >> > > above,
> > >> > > > > the
> > >> > > > > > TX
> > >> > > > > > > > > might
> > >> > > > > > > > > >>>>>>>>> actually not hang (of course, we might get
> an
> > >> EOS
> > >> > > > > violation
> > >> > > > > > > if
> > >> > > > > > > > > the
> > >> > > > > > > > > >>>>>>> first
> > >> > > > > > > > > >>>>>>>>> TX was aborted and the second committed, or
> > the
> > >> > other
> > >> > > > way
> > >> > > > > > > > > around).
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>> (20)
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>> Of course, 1 and 2 require client-side
> > >> changes, so
> > >> > > for
> > >> > > > > > older
> > >> > > > > > > > > >>>>>> clients,
> > >> > > > > > > > > >>>>>>>>> those approaches won’t apply.
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>> For (1) I understand why a client change is
> > >> > > necessary,
> > >> > > > > but
> > >> > > > > > > not
> > >> > > > > > > > > sure
> > >> > > > > > > > > >>>>>> why
> > >> > > > > > > > > >>>>>>>>> we need a client change for (2). Can you
> > >> elaborate?
> > >> > > --
> > >> > > > > > Later
> > >> > > > > > > > you
> > >> > > > > > > > > >>>>>>> explain
> > >> > > > > > > > > >>>>>>>>> that we should send a
> > >> DescribeTransactionRequest,
> > >> > > but I
> > >> > > > > am
> > >> > > > > > > not
> > >> > > > > > > > > sure
> > >> > > > > > > > > >>>>>>> why?
> > >> > > > > > > > > >>>>>>>>> Can't we not just do an implicit
> > >> AddPartiitonToTx,
> > >> > > too?
> > >> > > > > If
> > >> > > > > > > the
> > >> > > > > > > > > old
> > >> > > > > > > > > >>>>>>>>> producer correctly registered the partition
> > >> > already,
> > >> > > > the
> > >> > > > > > > > > >>>>>> TX-coordinator
> > >> > > > > > > > > >>>>>>>>> can just ignore it as it's an idempotent
> > >> operation?
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>> (30)
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>> To cover older clients, we will ensure a
> > >> > transaction
> > >> > > > is
> > >> > > > > > > > ongoing
> > >> > > > > > > > > >>>>>>> before
> > >> > > > > > > > > >>>>>>>>> we write to a transaction
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>> Not sure what you mean by this? Can you
> > >> elaborate?
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>> (40)
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>> [the TX-coordinator] will write the prepare
> > >> commit
> > >> > > > > message
> > >> > > > > > > > with
> > >> > > > > > > > > a
> > >> > > > > > > > > >>>>>>>> bumped
> > >> > > > > > > > > >>>>>>>>> epoch and send WriteTxnMarkerRequests with
> the
> > >> > bumped
> > >> > > > > > epoch.
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>> Why do we use the bumped epoch for both? It
> > >> seems
> > >> > > more
> > >> > > > > > > > intuitive
> > >> > > > > > > > > to
> > >> > > > > > > > > >>>>>> use
> > >> > > > > > > > > >>>>>>>>> the current epoch, and only return the
> bumped
> > >> epoch
> > >> > > to
> > >> > > > > the
> > >> > > > > > > > > >>>>> producer?
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>> (50) "Implicit AddPartitionToTransaction"
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>> Why does the implicitly sent request need to
> > be
> > >> > > > > > synchronous?
> > >> > > > > > > > The
> > >> > > > > > > > > >>>>> KIP
> > >> > > > > > > > > >>>>>>>>> also says
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>> in case we need to abort and need to know
> > which
> > >> > > > > partitions
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>> What do you mean by this?
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>> we don’t want to write to it before we
> store
> > in
> > >> > the
> > >> > > > > > > > transaction
> > >> > > > > > > > > >>>>>>> manager
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>> Do you mean TX-coordinator instead of
> > "manager"?
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>> (60)
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>> For older clients and ensuring that the TX
> is
> > >> > > ongoing,
> > >> > > > > you
> > >> > > > > > > > > >>>>> describe a
> > >> > > > > > > > > >>>>>>>>> race condition. I am not sure if I can
> follow
> > >> here.
> > >> > > Can
> > >> > > > > you
> > >> > > > > > > > > >>>>>> elaborate?
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>> -Matthias
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>> On 11/18/22 1:21 PM, Justine Olshan wrote:
> > >> > > > > > > > > >>>>>>>>>> Hey all!
> > >> > > > > > > > > >>>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>> I'd like to start a discussion on my
> proposal
> > >> to
> > >> > add
> > >> > > > > some
> > >> > > > > > > > > >>>>>> server-side
> > >> > > > > > > > > >>>>>>>>>> checks on transactions to avoid hanging
> > >> > > transactions.
> > >> > > > I
> > >> > > > > > know
> > >> > > > > > > > > this
> > >> > > > > > > > > >>>>>> has
> > >> > > > > > > > > >>>>>>>>> been
> > >> > > > > > > > > >>>>>>>>>> an issue for some time, so I really hope
> this
> > >> KIP
> > >> > > will
> > >> > > > > be
> > >> > > > > > > > > helpful
> > >> > > > > > > > > >>>>>> for
> > >> > > > > > > > > >>>>>>>>> many
> > >> > > > > > > > > >>>>>>>>>> users of EOS.
> > >> > > > > > > > > >>>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>> The KIP includes changes that will be
> > >> compatible
> > >> > > with
> > >> > > > > old
> > >> > > > > > > > > clients
> > >> > > > > > > > > >>>>>> and
> > >> > > > > > > > > >>>>>>>>>> changes to improve performance and
> > correctness
> > >> on
> > >> > > new
> > >> > > > > > > clients.
> > >> > > > > > > > > >>>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>> Please take a look and leave any comments
> you
> > >> may
> > >> > > > have!
> > >> > > > > > > > > >>>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>> KIP:
> > >> > > > > > > > > >>>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>
> > >> > > > > > > > > >>>>>>
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>
> > >> > > > > > > > >
> > >> > > > > > > >
> > >> > > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > >> > > > > > > > > >>>>>>>>>> JIRA:
> > >> > > > https://issues.apache.org/jira/browse/KAFKA-14402
> > >> > > > > > > > > >>>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>> Thanks!
> > >> > > > > > > > > >>>>>>>>>> Justine
> > >> > > > > > > > > >>>>>>>>>>
> > >> > > > > > > > > >>>>>>>>>
> > >> > > > > > > > > >>>>>>>>
> > >> > > > > > > > > >>>>>>>
> > >> > > > > > > > > >>>>>>
> > >> > > > > > > > > >>>>>
> > >> > > > > > > > > >>>>
> > >> > > > > > > > > >>>
> > >> > > > > > > > > >>
> > >> > > > > > > > > >
> > >> > > > > > > > >
> > >> > > > > > > >
> > >> > > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Jason Gustafson <ja...@confluent.io.INVALID>.
Hey Justine,

> I was wondering about compatibility here. When we send requests
between brokers, we want to ensure that the receiving broker understands
the request (specifically the new fields). Typically this is done via
IBP/metadata version.
I'm trying to think if there is a way around it but I'm not sure there is.

Yes. I think we would gate usage of this behind an IBP bump. Does that seem
reasonable?

> As for the improvements -- can you clarify how the multiple transactional
IDs would help here? Were you thinking of a case where we wait/batch
multiple produce requests together? My understanding for now was 1
transactional ID and one validation per 1 produce request.

Each call to `AddPartitionsToTxn` is essentially a write to the transaction
log and must block on replication. The more we can fit into a single
request, the more writes we can do in parallel. The alternative is to make
use of more connections, but usually we prefer batching since the network
stack is not really optimized for high connection/request loads.

> Finally with respect to the authorizations, I think it makes sense to skip
topic authorizations, but I'm a bit confused by the "leader ID" field.
Wouldn't we just want to flag the request as from a broker (does it matter
which one?).

We could also make it version-based. For the next version, we could require
CLUSTER auth. So clients would not be able to use the API anymore, which is
probably what we want.

-Jason

On Fri, Jan 6, 2023 at 10:43 AM Justine Olshan <jo...@confluent.io.invalid>
wrote:

> As a follow up, I was just thinking about the batching a bit more.
> I suppose if we have one request in flight and we queue up the other
> produce requests in some sort of purgatory, we could send information out
> for all of them rather than one by one. So that would be a benefit of
> batching partitions to add per transaction.
>
> I'll need to think a bit more on the design of this part of the KIP, and
> will update the KIP in the next few days.
>
> Thanks,
> Justine
>
> On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan <jo...@confluent.io>
> wrote:
>
> > Hey Jason -- thanks for the input -- I was just digging a bit deeper into
> > the design + implementation of the validation calls here and what you say
> > makes sense.
> >
> > I was wondering about compatibility here. When we send requests
> > between brokers, we want to ensure that the receiving broker understands
> > the request (specifically the new fields). Typically this is done via
> > IBP/metadata version.
> > I'm trying to think if there is a way around it but I'm not sure there
> is.
> >
> > As for the improvements -- can you clarify how the multiple transactional
> > IDs would help here? Were you thinking of a case where we wait/batch
> > multiple produce requests together? My understanding for now was 1
> > transactional ID and one validation per 1 produce request.
> >
> > Finally with respect to the authorizations, I think it makes sense to
> skip
> > topic authorizations, but I'm a bit confused by the "leader ID" field.
> > Wouldn't we just want to flag the request as from a broker (does it
> matter
> > which one?).
> >
> > I think I want to adopt these suggestions, just had a few questions on
> the
> > details.
> >
> > Thanks,
> > Justine
> >
> > On Thu, Jan 5, 2023 at 5:05 PM Jason Gustafson
> <ja...@confluent.io.invalid>
> > wrote:
> >
> >> Hi Justine,
> >>
> >> Thanks for the proposal.
> >>
> >> I was thinking about the implementation a little bit. In the current
> >> proposal, the behavior depends on whether we have an old or new client.
> >> For
> >> old clients, we send `DescribeTransactions` and verify the result and
> for
> >> new clients, we send `AddPartitionsToTxn`. We might be able to simplify
> >> the
> >> implementation if we can use the same request type. For example, what if
> >> we
> >> bump the protocol version for `AddPartitionsToTxn` and add a
> >> `validateOnly`
> >> flag? For older versions, we can set `validateOnly=true` so that the
> >> request only returns successfully if the partition had already been
> added.
> >> For new versions, we can set `validateOnly=false` and the partition will
> >> be
> >> added to the transaction. The other slightly annoying thing that this
> >> would
> >> get around is the need to collect the transaction state for all
> partitions
> >> even when we only care about a subset.
> >>
> >> Some additional improvements to consider:
> >>
> >> - We can give `AddPartitionsToTxn` better batch support for inter-broker
> >> usage. Currently we only allow one `TransactionalId` to be specified,
> but
> >> the broker may get some benefit being able to batch across multiple
> >> transactions.
> >> - Another small improvement is skipping topic authorization checks for
> >> `AddPartitionsToTxn` when the request is from a broker. Perhaps we can
> add
> >> a field for the `LeaderId` or something like that and require CLUSTER
> >> permission when set.
> >>
> >> Best,
> >> Jason
> >>
> >>
> >>
> >> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao <ju...@confluent.io.invalid>
> wrote:
> >>
> >> > Hi, Justine,
> >> >
> >> > Thanks for the explanation. It makes sense to me now.
> >> >
> >> > Jun
> >> >
> >> > On Mon, Dec 19, 2022 at 1:42 PM Justine Olshan
> >> > <jo...@confluent.io.invalid>
> >> > wrote:
> >> >
> >> > > Hi Jun,
> >> > >
> >> > > My understanding of the mechanism is that when we get to the last
> >> epoch,
> >> > we
> >> > > increment to the fencing/last epoch and if any further requests come
> >> in
> >> > for
> >> > > this producer ID they are fenced. Then the producer gets a new ID
> and
> >> > > restarts with epoch/sequence 0. The fenced epoch sticks around for
> the
> >> > > duration of producer.id.expiration.ms and blocks any late messages
> >> > there.
> >> > > The new ID will get to take advantage of the improved semantics
> around
> >> > > non-zero start sequences. So I think we are covered.
> >> > >
> >> > > The only potential issue is overloading the cache, but hopefully the
> >> > > improvements (lowered producer.id.expiration.ms) will help with
> that.
> >> > Let
> >> > > me know if you still have concerns.
> >> > >
> >> > > Thanks,
> >> > > Justine
> >> > >
> >> > > On Mon, Dec 19, 2022 at 10:24 AM Jun Rao <ju...@confluent.io.invalid>
> >> > wrote:
> >> > >
> >> > > > Hi, Justine,
> >> > > >
> >> > > > Thanks for the explanation.
> >> > > >
> >> > > > 70. The proposed fencing logic doesn't apply when pid changes, is
> >> that
> >> > > > right? If so, I am not sure how complete we are addressing this
> >> issue
> >> > if
> >> > > > the pid changes more frequently.
> >> > > >
> >> > > > Thanks,
> >> > > >
> >> > > > Jun
> >> > > >
> >> > > >
> >> > > >
> >> > > > On Fri, Dec 16, 2022 at 9:16 AM Justine Olshan
> >> > > > <jo...@confluent.io.invalid>
> >> > > > wrote:
> >> > > >
> >> > > > > Hi Jun,
> >> > > > >
> >> > > > > Thanks for replying!
> >> > > > >
> >> > > > > 70.We already do the overflow mechanism, so my change would just
> >> make
> >> > > it
> >> > > > > happen more often.
> >> > > > > I was also not suggesting a new field in the log, but in the
> >> > response,
> >> > > > > which would be gated by the client version. Sorry if something
> >> there
> >> > is
> >> > > > > unclear. I think we are starting to diverge.
> >> > > > > The goal of this KIP is to not change to the marker format at
> all.
> >> > > > >
> >> > > > > 71. Yes, I guess I was going under the assumption that the log
> >> would
> >> > > just
> >> > > > > look at its last epoch and treat it as the current epoch. I
> >> suppose
> >> > we
> >> > > > can
> >> > > > > have some special logic that if the last epoch was on a marker
> we
> >> > > > actually
> >> > > > > expect the next epoch or something like that. We just need to
> >> > > distinguish
> >> > > > > based on whether we had a commit/abort marker.
> >> > > > >
> >> > > > > 72.
> >> > > > > > if the producer epoch hasn't been bumped on the
> >> > > > > broker, it seems that the stucked message will fail the sequence
> >> > > > validation
> >> > > > > and will be ignored. If the producer epoch has been bumped, we
> >> ignore
> >> > > the
> >> > > > > sequence check and the stuck message could be appended to the
> log.
> >> > So,
> >> > > is
> >> > > > > the latter case that we want to guard?
> >> > > > >
> >> > > > > I'm not sure I follow that "the message will fail the sequence
> >> > > > validation".
> >> > > > > In some of these cases, we had an abort marker (due to an error)
> >> and
> >> > > then
> >> > > > > the late message comes in with the correct sequence number. This
> >> is a
> >> > > > case
> >> > > > > covered by the KIP.
> >> > > > > The latter case is actually not something we've considered
> here. I
> >> > > think
> >> > > > > generally when we bump the epoch, we are accepting that the
> >> sequence
> >> > > does
> >> > > > > not need to be checked anymore. My understanding is also that we
> >> > don't
> >> > > > > typically bump epoch mid transaction (based on a quick look at
> the
> >> > > code)
> >> > > > > but let me know if that is the case.
> >> > > > >
> >> > > > > Thanks,
> >> > > > > Justine
> >> > > > >
> >> > > > > On Thu, Dec 15, 2022 at 12:23 PM Jun Rao
> <jun@confluent.io.invalid
> >> >
> >> > > > wrote:
> >> > > > >
> >> > > > > > Hi, Justine,
> >> > > > > >
> >> > > > > > Thanks for the reply.
> >> > > > > >
> >> > > > > > 70. Assigning a new pid on int overflow seems a bit hacky. If
> we
> >> > > need a
> >> > > > > txn
> >> > > > > > level id, it will be better to model this explicitly. Adding a
> >> new
> >> > > > field
> >> > > > > > would require a bit more work since it requires a new txn
> marker
> >> > > format
> >> > > > > in
> >> > > > > > the log. So, we probably need to guard it with an IBP or
> >> metadata
> >> > > > version
> >> > > > > > and document the impact on downgrade once the new format is
> >> written
> >> > > to
> >> > > > > the
> >> > > > > > log.
> >> > > > > >
> >> > > > > > 71. Hmm, once the marker is written, the partition will expect
> >> the
> >> > > next
> >> > > > > > append to be on the next epoch. Does that cover the case you
> >> > > mentioned?
> >> > > > > >
> >> > > > > > 72. Also, just to be clear on the stucked message issue
> >> described
> >> > in
> >> > > > the
> >> > > > > > motivation. With EoS, we also validate the sequence id for
> >> > > idempotency.
> >> > > > > So,
> >> > > > > > with the current logic, if the producer epoch hasn't been
> >> bumped on
> >> > > the
> >> > > > > > broker, it seems that the stucked message will fail the
> sequence
> >> > > > > validation
> >> > > > > > and will be ignored. If the producer epoch has been bumped, we
> >> > ignore
> >> > > > the
> >> > > > > > sequence check and the stuck message could be appended to the
> >> log.
> >> > > So,
> >> > > > is
> >> > > > > > the latter case that we want to guard?
> >> > > > > >
> >> > > > > > Thanks,
> >> > > > > >
> >> > > > > > Jun
> >> > > > > >
> >> > > > > > On Wed, Dec 14, 2022 at 10:44 AM Justine Olshan
> >> > > > > > <jo...@confluent.io.invalid> wrote:
> >> > > > > >
> >> > > > > > > Matthias — thanks again for taking time to look a this. You
> >> said:
> >> > > > > > >
> >> > > > > > > > My proposal was only focusing to avoid dangling
> >> > > > > > >
> >> > > > > > > transactions if records are added without registered
> >> partition.
> >> > --
> >> > > > > Maybe
> >> > > > > > >
> >> > > > > > > you can add a few more details to the KIP about this
> scenario
> >> for
> >> > > > > better
> >> > > > > > >
> >> > > > > > > documentation purpose?
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > I'm not sure I understand what you mean here. The motivation
> >> > > section
> >> > > > > > > describes two scenarios about how the record can be added
> >> > without a
> >> > > > > > > registered partition:
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > > 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.
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > > Another way hanging transactions can occur is that a
> client
> >> is
> >> > > > buggy
> >> > > > > > and
> >> > > > > > > may somehow try to write to a partition before it adds the
> >> > > partition
> >> > > > to
> >> > > > > > the
> >> > > > > > > transaction.
> >> > > > > > >
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > For the first example of this would it be helpful to say
> that
> >> > this
> >> > > > > > message
> >> > > > > > > comes in after the abort, but before the partition is added
> to
> >> > the
> >> > > > next
> >> > > > > > > transaction so it becomes "hanging." Perhaps the next
> sentence
> >> > > > > describing
> >> > > > > > > the message becoming part of the next transaction (a
> different
> >> > > case)
> >> > > > > was
> >> > > > > > > not properly differentiated.
> >> > > > > > >
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > Jun — thanks for reading the KIP.
> >> > > > > > >
> >> > > > > > > 70. The int typing was a concern. Currently we have a
> >> mechanism
> >> > in
> >> > > > > place
> >> > > > > > to
> >> > > > > > > fence the final epoch when the epoch is about to overflow
> and
> >> > > assign
> >> > > > a
> >> > > > > > new
> >> > > > > > > producer ID with epoch 0. Of course, this is a bit tricky
> >> when it
> >> > > > comes
> >> > > > > > to
> >> > > > > > > the response back to the client.
> >> > > > > > > Making this a long could be another option, but I wonder are
> >> > there
> >> > > > any
> >> > > > > > > implications on changing this field if the epoch is
> persisted
> >> to
> >> > > > disk?
> >> > > > > > I'd
> >> > > > > > > need to check the usages.
> >> > > > > > >
> >> > > > > > > 71.This was something Matthias asked about as well. I was
> >> > > > considering a
> >> > > > > > > possible edge case where a produce request from a new
> >> transaction
> >> > > > > somehow
> >> > > > > > > gets sent right after the marker is written, but before the
> >> > > producer
> >> > > > is
> >> > > > > > > alerted of the newly bumped epoch. In this case, we may
> >> include
> >> > > this
> >> > > > > > record
> >> > > > > > > when we don't want to. I suppose we could try to do
> something
> >> > > client
> >> > > > > side
> >> > > > > > > to bump the epoch after sending an endTxn as well in this
> >> > scenario
> >> > > —
> >> > > > > but
> >> > > > > > I
> >> > > > > > > wonder how it would work when the server is aborting based
> on
> >> a
> >> > > > > > server-side
> >> > > > > > > error. I could also be missing something and this scenario
> is
> >> > > > actually
> >> > > > > > not
> >> > > > > > > possible.
> >> > > > > > >
> >> > > > > > > Thanks again to everyone reading and commenting. Let me know
> >> > about
> >> > > > any
> >> > > > > > > further questions or comments.
> >> > > > > > >
> >> > > > > > > Justine
> >> > > > > > >
> >> > > > > > > On Wed, Dec 14, 2022 at 9:41 AM Jun Rao
> >> <jun@confluent.io.invalid
> >> > >
> >> > > > > > wrote:
> >> > > > > > >
> >> > > > > > > > Hi, Justine,
> >> > > > > > > >
> >> > > > > > > > Thanks for the KIP. A couple of comments.
> >> > > > > > > >
> >> > > > > > > > 70. Currently, the producer epoch is an int. I am not sure
> >> if
> >> > > it's
> >> > > > > > enough
> >> > > > > > > > to accommodate all transactions in the lifetime of a
> >> producer.
> >> > > > Should
> >> > > > > > we
> >> > > > > > > > change that to a long or add a new long field like txnId?
> >> > > > > > > >
> >> > > > > > > > 71. "it will write the prepare commit message with a
> bumped
> >> > epoch
> >> > > > and
> >> > > > > > > send
> >> > > > > > > > WriteTxnMarkerRequests with the bumped epoch." Hmm, the
> >> epoch
> >> > is
> >> > > > > > > associated
> >> > > > > > > > with the current txn right? So, it seems weird to write a
> >> > commit
> >> > > > > > message
> >> > > > > > > > with a bumped epoch. Should we only bump up the epoch in
> >> > > > > EndTxnResponse
> >> > > > > > > and
> >> > > > > > > > rename the field to sth like nextProducerEpoch?
> >> > > > > > > >
> >> > > > > > > > Thanks,
> >> > > > > > > >
> >> > > > > > > > Jun
> >> > > > > > > >
> >> > > > > > > >
> >> > > > > > > >
> >> > > > > > > > On Mon, Dec 12, 2022 at 8:54 PM Matthias J. Sax <
> >> > > mjsax@apache.org>
> >> > > > > > > wrote:
> >> > > > > > > >
> >> > > > > > > > > Thanks for the background.
> >> > > > > > > > >
> >> > > > > > > > > 20/30: SGTM. My proposal was only focusing to avoid
> >> dangling
> >> > > > > > > > > transactions if records are added without registered
> >> > partition.
> >> > > > --
> >> > > > > > > Maybe
> >> > > > > > > > > you can add a few more details to the KIP about this
> >> scenario
> >> > > for
> >> > > > > > > better
> >> > > > > > > > > documentation purpose?
> >> > > > > > > > >
> >> > > > > > > > > 40: I think you hit a fair point about race conditions
> or
> >> > > client
> >> > > > > bugs
> >> > > > > > > > > (incorrectly not bumping the epoch). The
> >> complexity/confusion
> >> > > for
> >> > > > > > using
> >> > > > > > > > > the bumped epoch I see, is mainly for internal
> debugging,
> >> ie,
> >> > > > > > > inspecting
> >> > > > > > > > > log segment dumps -- it seems harder to reason about the
> >> > system
> >> > > > for
> >> > > > > > us
> >> > > > > > > > > humans. But if we get better guarantees, it would be
> >> worth to
> >> > > use
> >> > > > > the
> >> > > > > > > > > bumped epoch.
> >> > > > > > > > >
> >> > > > > > > > > 60: as I mentioned already, I don't know the broker
> >> internals
> >> > > to
> >> > > > > > > provide
> >> > > > > > > > > more input. So if nobody else chimes in, we should just
> >> move
> >> > > > > forward
> >> > > > > > > > > with your proposal.
> >> > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > > > -Matthias
> >> > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > > > On 12/6/22 4:22 PM, Justine Olshan wrote:
> >> > > > > > > > > > Hi all,
> >> > > > > > > > > > After Artem's questions about error behavior, I've
> >> > > re-evaluated
> >> > > > > the
> >> > > > > > > > > > unknown producer ID exception and had some discussions
> >> > > offline.
> >> > > > > > > > > >
> >> > > > > > > > > > I think generally it makes sense to simplify error
> >> handling
> >> > > in
> >> > > > > > cases
> >> > > > > > > > like
> >> > > > > > > > > > this and the UNKNOWN_PRODUCER_ID error has a pretty
> long
> >> > and
> >> > > > > > > > complicated
> >> > > > > > > > > > history. Because of this, I propose adding a new error
> >> code
> >> > > > > > > > > ABORTABLE_ERROR
> >> > > > > > > > > > that when encountered by new clients (gated by the
> >> produce
> >> > > > > request
> >> > > > > > > > > version)
> >> > > > > > > > > > will simply abort the transaction. This allows the
> >> server
> >> > to
> >> > > > have
> >> > > > > > > some
> >> > > > > > > > > say
> >> > > > > > > > > > in whether the client aborts and makes handling much
> >> > simpler.
> >> > > > In
> >> > > > > > the
> >> > > > > > > > > > future, we can also use this error in other situations
> >> > where
> >> > > we
> >> > > > > > want
> >> > > > > > > to
> >> > > > > > > > > > abort the transactions. We can even use on other apis.
> >> > > > > > > > > >
> >> > > > > > > > > > I've added this to the KIP. Let me know if there are
> any
> >> > > > > questions
> >> > > > > > or
> >> > > > > > > > > > issues.
> >> > > > > > > > > >
> >> > > > > > > > > > Justine
> >> > > > > > > > > >
> >> > > > > > > > > > On Fri, Dec 2, 2022 at 10:22 AM Justine Olshan <
> >> > > > > > jolshan@confluent.io
> >> > > > > > > >
> >> > > > > > > > > wrote:
> >> > > > > > > > > >
> >> > > > > > > > > >> Hey Matthias,
> >> > > > > > > > > >>
> >> > > > > > > > > >>
> >> > > > > > > > > >> 20/30 — Maybe I also didn't express myself clearly.
> For
> >> > > older
> >> > > > > > > clients
> >> > > > > > > > we
> >> > > > > > > > > >> don't have a way to distinguish between a previous
> and
> >> the
> >> > > > > current
> >> > > > > > > > > >> transaction since we don't have the epoch bump. This
> >> means
> >> > > > that
> >> > > > > a
> >> > > > > > > late
> >> > > > > > > > > >> message from the previous transaction may be added to
> >> the
> >> > > new
> >> > > > > one.
> >> > > > > > > > With
> >> > > > > > > > > >> older clients — we can't guarantee this won't happen
> >> if we
> >> > > > > already
> >> > > > > > > > sent
> >> > > > > > > > > the
> >> > > > > > > > > >> addPartitionsToTxn call (why we make changes for the
> >> newer
> >> > > > > client)
> >> > > > > > > but
> >> > > > > > > > > we
> >> > > > > > > > > >> can at least gate some by ensuring that the partition
> >> has
> >> > > been
> >> > > > > > added
> >> > > > > > > > to
> >> > > > > > > > > the
> >> > > > > > > > > >> transaction. The rationale here is that there are
> >> likely
> >> > > LESS
> >> > > > > late
> >> > > > > > > > > arrivals
> >> > > > > > > > > >> as time goes on, so hopefully most late arrivals will
> >> come
> >> > > in
> >> > > > > > BEFORE
> >> > > > > > > > the
> >> > > > > > > > > >> addPartitionsToTxn call. Those that arrive before
> will
> >> be
> >> > > > > properly
> >> > > > > > > > gated
> >> > > > > > > > > >> with the describeTransactions approach.
> >> > > > > > > > > >>
> >> > > > > > > > > >> If we take the approach you suggested, ANY late
> arrival
> >> > > from a
> >> > > > > > > > previous
> >> > > > > > > > > >> transaction will be added. And we don't want that. I
> >> also
> >> > > > don't
> >> > > > > > see
> >> > > > > > > > any
> >> > > > > > > > > >> benefit in sending addPartitionsToTxn over the
> >> > describeTxns
> >> > > > > call.
> >> > > > > > > They
> >> > > > > > > > > will
> >> > > > > > > > > >> both be one extra RPC to the Txn coordinator.
> >> > > > > > > > > >>
> >> > > > > > > > > >>
> >> > > > > > > > > >> To be clear — newer clients will use
> addPartitionsToTxn
> >> > > > instead
> >> > > > > of
> >> > > > > > > the
> >> > > > > > > > > >> DescribeTxns.
> >> > > > > > > > > >>
> >> > > > > > > > > >>
> >> > > > > > > > > >> 40)
> >> > > > > > > > > >> My concern is that if we have some delay in the
> client
> >> to
> >> > > bump
> >> > > > > the
> >> > > > > > > > > epoch,
> >> > > > > > > > > >> it could continue to send epoch 73 and those records
> >> would
> >> > > not
> >> > > > > be
> >> > > > > > > > > fenced.
> >> > > > > > > > > >> Perhaps this is not an issue if we don't allow the
> next
> >> > > > produce
> >> > > > > to
> >> > > > > > > go
> >> > > > > > > > > >> through before the EndTxn request returns. I'm also
> >> > thinking
> >> > > > > about
> >> > > > > > > > > cases of
> >> > > > > > > > > >> failure. I will need to think on this a bit.
> >> > > > > > > > > >>
> >> > > > > > > > > >> I wasn't sure if it was that confusing. But if we
> >> think it
> >> > > is,
> >> > > > > we
> >> > > > > > > can
> >> > > > > > > > > >> investigate other ways.
> >> > > > > > > > > >>
> >> > > > > > > > > >>
> >> > > > > > > > > >> 60)
> >> > > > > > > > > >>
> >> > > > > > > > > >> I'm not sure these are the same purgatories since one
> >> is a
> >> > > > > produce
> >> > > > > > > > > >> purgatory (I was planning on using a callback rather
> >> than
> >> > > > > > purgatory)
> >> > > > > > > > and
> >> > > > > > > > > >> the other is simply a request to append to the log.
> Not
> >> > sure
> >> > > > we
> >> > > > > > have
> >> > > > > > > > any
> >> > > > > > > > > >> structure here for ordering, but my understanding is
> >> that
> >> > > the
> >> > > > > > broker
> >> > > > > > > > > could
> >> > > > > > > > > >> handle the write request before it hears back from
> the
> >> Txn
> >> > > > > > > > Coordinator.
> >> > > > > > > > > >>
> >> > > > > > > > > >> Let me know if I misunderstood something or something
> >> was
> >> > > > > unclear.
> >> > > > > > > > > >>
> >> > > > > > > > > >> Justine
> >> > > > > > > > > >>
> >> > > > > > > > > >> On Thu, Dec 1, 2022 at 12:15 PM Matthias J. Sax <
> >> > > > > mjsax@apache.org
> >> > > > > > >
> >> > > > > > > > > wrote:
> >> > > > > > > > > >>
> >> > > > > > > > > >>> Thanks for the details Justine!
> >> > > > > > > > > >>>
> >> > > > > > > > > >>>> 20)
> >> > > > > > > > > >>>>
> >> > > > > > > > > >>>> The client side change for 2 is removing the
> >> > addPartitions
> >> > > > to
> >> > > > > > > > > >>> transaction
> >> > > > > > > > > >>>> call. We don't need to make this from the producer
> to
> >> > the
> >> > > > txn
> >> > > > > > > > > >>> coordinator,
> >> > > > > > > > > >>>> only server side.
> >> > > > > > > > > >>>
> >> > > > > > > > > >>> I think I did not express myself clearly. I
> understand
> >> > that
> >> > > > we
> >> > > > > > can
> >> > > > > > > > (and
> >> > > > > > > > > >>> should) change the producer to not send the
> >> > `addPartitions`
> >> > > > > > request
> >> > > > > > > > any
> >> > > > > > > > > >>> longer. But I don't thinks it's requirement to
> change
> >> the
> >> > > > > broker?
> >> > > > > > > > > >>>
> >> > > > > > > > > >>> What I am trying to say is: as a safe-guard and
> >> > improvement
> >> > > > for
> >> > > > > > > older
> >> > > > > > > > > >>> producers, the partition leader can just send the
> >> > > > > `addPartitions`
> >> > > > > > > > > >>> request to the TX-coordinator in any case -- if the
> >> old
> >> > > > > producer
> >> > > > > > > > > >>> correctly did send the `addPartition` request to the
> >> > > > > > TX-coordinator
> >> > > > > > > > > >>> already, the TX-coordinator can just "ignore" is as
> >> > > > idempotent.
> >> > > > > > > > > However,
> >> > > > > > > > > >>> if the old producer has a bug and did forget to sent
> >> the
> >> > > > > > > > `addPartition`
> >> > > > > > > > > >>> request, we would now ensure that the partition is
> >> indeed
> >> > > > added
> >> > > > > > to
> >> > > > > > > > the
> >> > > > > > > > > >>> TX and thus fix a potential producer bug (even if we
> >> > don't
> >> > > > get
> >> > > > > > the
> >> > > > > > > > > >>> fencing via the bump epoch). -- It seems to be a
> good
> >> > > > > > improvement?
> >> > > > > > > Or
> >> > > > > > > > > is
> >> > > > > > > > > >>> there a reason to not do this?
> >> > > > > > > > > >>>
> >> > > > > > > > > >>>
> >> > > > > > > > > >>>
> >> > > > > > > > > >>>> 30)
> >> > > > > > > > > >>>>
> >> > > > > > > > > >>>> Transaction is ongoing = partition was added to
> >> > > transaction
> >> > > > > via
> >> > > > > > > > > >>>> addPartitionsToTxn. We check this with the
> >> > > > > DescribeTransactions
> >> > > > > > > > call.
> >> > > > > > > > > >>> Let
> >> > > > > > > > > >>>> me know if this wasn't sufficiently explained here:
> >> > > > > > > > > >>>
> >> > > > > > > > > >>> If we do what I propose in (20), we don't really
> need
> >> to
> >> > > make
> >> > > > > > this
> >> > > > > > > > > >>> `DescribeTransaction` call, as the partition leader
> >> adds
> >> > > the
> >> > > > > > > > partition
> >> > > > > > > > > >>> for older clients and we get this check for free.
> >> > > > > > > > > >>>
> >> > > > > > > > > >>>
> >> > > > > > > > > >>>> 40)
> >> > > > > > > > > >>>>
> >> > > > > > > > > >>>> The idea here is that if any messages somehow come
> in
> >> > > before
> >> > > > > we
> >> > > > > > > get
> >> > > > > > > > > the
> >> > > > > > > > > >>> new
> >> > > > > > > > > >>>> epoch to the producer, they will be fenced.
> However,
> >> if
> >> > we
> >> > > > > don't
> >> > > > > > > > think
> >> > > > > > > > > >>> this
> >> > > > > > > > > >>>> is necessary, it can be discussed
> >> > > > > > > > > >>>
> >> > > > > > > > > >>> I agree that we should have epoch fencing. My
> >> question is
> >> > > > > > > different:
> >> > > > > > > > > >>> Assume we are at epoch 73, and we have an ongoing
> >> > > > transaction,
> >> > > > > > that
> >> > > > > > > > is
> >> > > > > > > > > >>> committed. It seems natural to write the "prepare
> >> commit"
> >> > > > > marker
> >> > > > > > > and
> >> > > > > > > > > the
> >> > > > > > > > > >>> `WriteTxMarkerRequest` both with epoch 73, too, as
> it
> >> > > belongs
> >> > > > > to
> >> > > > > > > the
> >> > > > > > > > > >>> current transaction. Of course, we now also bump the
> >> > epoch
> >> > > > and
> >> > > > > > > expect
> >> > > > > > > > > >>> the next requests to have epoch 74, and would reject
> >> an
> >> > > > request
> >> > > > > > > with
> >> > > > > > > > > >>> epoch 73, as the corresponding TX for epoch 73 was
> >> > already
> >> > > > > > > committed.
> >> > > > > > > > > >>>
> >> > > > > > > > > >>> It seems you propose to write the "prepare commit
> >> marker"
> >> > > and
> >> > > > > > > > > >>> `WriteTxMarkerRequest` with epoch 74 though, what
> >> would
> >> > > work,
> >> > > > > but
> >> > > > > > > it
> >> > > > > > > > > >>> seems confusing. Is there a reason why we would use
> >> the
> >> > > > bumped
> >> > > > > > > epoch
> >> > > > > > > > 74
> >> > > > > > > > > >>> instead of the current epoch 73?
> >> > > > > > > > > >>>
> >> > > > > > > > > >>>
> >> > > > > > > > > >>>> 60)
> >> > > > > > > > > >>>>
> >> > > > > > > > > >>>> When we are checking if the transaction is ongoing,
> >> we
> >> > > need
> >> > > > to
> >> > > > > > > make
> >> > > > > > > > a
> >> > > > > > > > > >>> round
> >> > > > > > > > > >>>> trip from the leader partition to the transaction
> >> > > > coordinator.
> >> > > > > > In
> >> > > > > > > > the
> >> > > > > > > > > >>> time
> >> > > > > > > > > >>>> we are waiting for this message to come back, in
> >> theory
> >> > we
> >> > > > > could
> >> > > > > > > > have
> >> > > > > > > > > >>> sent
> >> > > > > > > > > >>>> a commit/abort call that would make the original
> >> result
> >> > of
> >> > > > the
> >> > > > > > > check
> >> > > > > > > > > >>> out of
> >> > > > > > > > > >>>> date. That is why we can check the leader state
> >> before
> >> > we
> >> > > > > write
> >> > > > > > to
> >> > > > > > > > the
> >> > > > > > > > > >>> log.
> >> > > > > > > > > >>>
> >> > > > > > > > > >>> Thanks. Got it.
> >> > > > > > > > > >>>
> >> > > > > > > > > >>> However, is this really an issue? We put the produce
> >> > > request
> >> > > > in
> >> > > > > > > > > >>> purgatory, so how could we process the
> >> > > > `WriteTxnMarkerRequest`
> >> > > > > > > first?
> >> > > > > > > > > >>> Don't we need to put the `WriteTxnMarkerRequest`
> into
> >> > > > > purgatory,
> >> > > > > > > too,
> >> > > > > > > > > >>> for this case, and process both request in-order?
> >> (Again,
> >> > > my
> >> > > > > > broker
> >> > > > > > > > > >>> knowledge is limited and maybe we don't maintain
> >> request
> >> > > > order
> >> > > > > > for
> >> > > > > > > > this
> >> > > > > > > > > >>> case, what seems to be an issue IMHO, and I am
> >> wondering
> >> > if
> >> > > > > > > changing
> >> > > > > > > > > >>> request handling to preserve order for this case
> >> might be
> >> > > the
> >> > > > > > > cleaner
> >> > > > > > > > > >>> solution?)
> >> > > > > > > > > >>>
> >> > > > > > > > > >>>
> >> > > > > > > > > >>>
> >> > > > > > > > > >>> -Matthias
> >> > > > > > > > > >>>
> >> > > > > > > > > >>>
> >> > > > > > > > > >>>
> >> > > > > > > > > >>>
> >> > > > > > > > > >>> On 11/30/22 3:28 PM, Artem Livshits wrote:
> >> > > > > > > > > >>>> Hi Justine,
> >> > > > > > > > > >>>>
> >> > > > > > > > > >>>> I think the interesting part is not in this logic
> >> > (because
> >> > > > it
> >> > > > > > > tries
> >> > > > > > > > to
> >> > > > > > > > > >>>> figure out when UNKNOWN_PRODUCER_ID is retriable
> and
> >> if
> >> > > it's
> >> > > > > > > > > retryable,
> >> > > > > > > > > >>>> it's definitely not fatal), but what happens when
> >> this
> >> > > logic
> >> > > > > > > doesn't
> >> > > > > > > > > >>> return
> >> > > > > > > > > >>>> 'true' and falls through.  In the old clients it
> >> seems
> >> > to
> >> > > be
> >> > > > > > > fatal,
> >> > > > > > > > if
> >> > > > > > > > > >>> we
> >> > > > > > > > > >>>> keep the behavior in the new clients, I'd expect it
> >> > would
> >> > > be
> >> > > > > > fatal
> >> > > > > > > > as
> >> > > > > > > > > >>> well.
> >> > > > > > > > > >>>>
> >> > > > > > > > > >>>> -Artem
> >> > > > > > > > > >>>>
> >> > > > > > > > > >>>> On Tue, Nov 29, 2022 at 11:57 AM Justine Olshan
> >> > > > > > > > > >>>> <jo...@confluent.io.invalid> wrote:
> >> > > > > > > > > >>>>
> >> > > > > > > > > >>>>> Hi Artem and Jeff,
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>> Thanks for taking a look and sorry for the slow
> >> > response.
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>> You both mentioned the change to handle
> >> > > UNKNOWN_PRODUCER_ID
> >> > > > > > > errors.
> >> > > > > > > > > To
> >> > > > > > > > > >>> be
> >> > > > > > > > > >>>>> clear — this error code will only be sent again
> when
> >> > the
> >> > > > > > client's
> >> > > > > > > > > >>> request
> >> > > > > > > > > >>>>> version is high enough to ensure we handle it
> >> > correctly.
> >> > > > > > > > > >>>>> The current (Java) client handles this by the
> >> following
> >> > > > > > (somewhat
> >> > > > > > > > > long)
> >> > > > > > > > > >>>>> code snippet:
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>> // An UNKNOWN_PRODUCER_ID means that we have lost
> >> the
> >> > > > > producer
> >> > > > > > > > state
> >> > > > > > > > > >>> on the
> >> > > > > > > > > >>>>> broker. Depending on the log start
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>> // offset, we may want to retry these, as
> described
> >> for
> >> > > > each
> >> > > > > > case
> >> > > > > > > > > >>> below. If
> >> > > > > > > > > >>>>> none of those apply, then for the
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>> // idempotent producer, we will locally bump the
> >> epoch
> >> > > and
> >> > > > > > reset
> >> > > > > > > > the
> >> > > > > > > > > >>>>> sequence numbers of in-flight batches from
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>> // sequence 0, then retry the failed batch, which
> >> > should
> >> > > > now
> >> > > > > > > > succeed.
> >> > > > > > > > > >>> For
> >> > > > > > > > > >>>>> the transactional producer, allow the
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>> // batch to fail. When processing the failed
> batch,
> >> we
> >> > > will
> >> > > > > > > > > transition
> >> > > > > > > > > >>> to
> >> > > > > > > > > >>>>> an abortable error and set a flag
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>> // indicating that we need to bump the epoch (if
> >> > > supported
> >> > > > by
> >> > > > > > the
> >> > > > > > > > > >>> broker).
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>> if (error == Errors.*UNKNOWN_PRODUCER_ID*) {
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>       if (response.logStartOffset == -1) {
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           // We don't know the log start offset
> with
> >> > this
> >> > > > > > > response.
> >> > > > > > > > > We
> >> > > > > > > > > >>> should
> >> > > > > > > > > >>>>> just retry the request until we get it.
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           // The UNKNOWN_PRODUCER_ID error code
> was
> >> > added
> >> > > > > along
> >> > > > > > > > with
> >> > > > > > > > > >>> the new
> >> > > > > > > > > >>>>> ProduceResponse which includes the
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           // logStartOffset. So the '-1' sentinel
> is
> >> > not
> >> > > > for
> >> > > > > > > > backward
> >> > > > > > > > > >>>>> compatibility. Instead, it is possible for
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           // a broker to not know the
> >> logStartOffset at
> >> > > > when
> >> > > > > it
> >> > > > > > > is
> >> > > > > > > > > >>> returning
> >> > > > > > > > > >>>>> the response because the partition
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           // may have moved away from the broker
> >> from
> >> > the
> >> > > > > time
> >> > > > > > > the
> >> > > > > > > > > >>> error was
> >> > > > > > > > > >>>>> initially raised to the time the
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           // response was being constructed. In
> >> these
> >> > > > cases,
> >> > > > > we
> >> > > > > > > > > should
> >> > > > > > > > > >>> just
> >> > > > > > > > > >>>>> retry the request: we are guaranteed
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           // to eventually get a logStartOffset
> once
> >> > > things
> >> > > > > > > settle
> >> > > > > > > > > down.
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           return true;
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>       }
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>       if (batch.sequenceHasBeenReset()) {
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           // When the first inflight batch fails
> >> due to
> >> > > the
> >> > > > > > > > > truncation
> >> > > > > > > > > >>> case,
> >> > > > > > > > > >>>>> then the sequences of all the other
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           // in flight batches would have been
> >> > restarted
> >> > > > from
> >> > > > > > the
> >> > > > > > > > > >>> beginning.
> >> > > > > > > > > >>>>> However, when those responses
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           // come back from the broker, they would
> >> also
> >> > > > come
> >> > > > > > with
> >> > > > > > > > an
> >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID error. In this case, we should
> >> not
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           // reset the sequence numbers to the
> >> > beginning.
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           return true;
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>       } else if
> >> > > > > (lastAckedOffset(batch.topicPartition).orElse(
> >> > > > > > > > > >>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> >> > > > response.logStartOffset) {
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           // The head of the log has been removed,
> >> > > probably
> >> > > > > due
> >> > > > > > > to
> >> > > > > > > > > the
> >> > > > > > > > > >>>>> retention time elapsing. In this case,
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           // we expect to lose the producer state.
> >> For
> >> > > the
> >> > > > > > > > > transactional
> >> > > > > > > > > >>>>> producer, reset the sequences of all
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           // inflight batches to be from the
> >> beginning
> >> > > and
> >> > > > > > retry
> >> > > > > > > > > them,
> >> > > > > > > > > >>> so
> >> > > > > > > > > >>>>> that the transaction does not need to
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           // be aborted. For the idempotent
> >> producer,
> >> > > bump
> >> > > > > the
> >> > > > > > > > epoch
> >> > > > > > > > > to
> >> > > > > > > > > >>> avoid
> >> > > > > > > > > >>>>> reusing (sequence, epoch) pairs
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           if (isTransactional()) {
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>
> >> > > > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> >> > > > > > > > > >>>>> this.producerIdAndEpoch);
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           } else {
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>
> >> > > > > >  requestEpochBumpForPartition(batch.topicPartition);
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           }
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           return true;
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>       }
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>       if (!isTransactional()) {
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           // For the idempotent producer, always
> >> retry
> >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> >> > > > > > > > > >>>>> errors. If the batch has the current
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           // producer ID and epoch, request a bump
> >> of
> >> > the
> >> > > > > > epoch.
> >> > > > > > > > > >>> Otherwise
> >> > > > > > > > > >>>>> just retry the produce.
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>
> >> > > >  requestEpochBumpForPartition(batch.topicPartition);
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>           return true;
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>       }
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>> }
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>> I was considering keeping this behavior — but am
> >> open
> >> > to
> >> > > > > > > > simplifying
> >> > > > > > > > > >>> it.
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>> We are leaving changes to older clients off the
> >> table
> >> > > here
> >> > > > > > since
> >> > > > > > > it
> >> > > > > > > > > >>> caused
> >> > > > > > > > > >>>>> many issues for clients in the past. Previously
> this
> >> > was
> >> > > a
> >> > > > > > fatal
> >> > > > > > > > > error
> >> > > > > > > > > >>> and
> >> > > > > > > > > >>>>> we didn't have the mechanisms in place to detect
> >> when
> >> > > this
> >> > > > > was
> >> > > > > > a
> >> > > > > > > > > >>> legitimate
> >> > > > > > > > > >>>>> case vs some bug or gap in the protocol. Ensuring
> >> each
> >> > > > > > > transaction
> >> > > > > > > > > has
> >> > > > > > > > > >>> its
> >> > > > > > > > > >>>>> own epoch should close this gap.
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>> And to address Jeff's second point:
> >> > > > > > > > > >>>>> *does the typical produce request path append
> >> records
> >> > to
> >> > > > > local
> >> > > > > > > log
> >> > > > > > > > > >>> along*
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>> *with the currentTxnFirstOffset information? I
> would
> >> > like
> >> > > > to
> >> > > > > > > > > >>> understand*
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>> *when the field is written to disk.*
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>> Yes, the first produce request populates this
> field
> >> and
> >> > > > > writes
> >> > > > > > > the
> >> > > > > > > > > >>> offset
> >> > > > > > > > > >>>>> as part of the record batch and also to the
> producer
> >> > > state
> >> > > > > > > > snapshot.
> >> > > > > > > > > >>> When
> >> > > > > > > > > >>>>> we reload the records on restart and/or
> >> reassignment,
> >> > we
> >> > > > > > > repopulate
> >> > > > > > > > > >>> this
> >> > > > > > > > > >>>>> field with the snapshot from disk along with the
> >> rest
> >> > of
> >> > > > the
> >> > > > > > > > producer
> >> > > > > > > > > >>>>> state.
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>> Let me know if there are further comments and/or
> >> > > questions.
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>> Thanks,
> >> > > > > > > > > >>>>> Justine
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>> On Tue, Nov 22, 2022 at 9:00 PM Jeff Kim
> >> > > > > > > > > <jeff.kim@confluent.io.invalid
> >> > > > > > > > > >>>>
> >> > > > > > > > > >>>>> wrote:
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>>> Hi Justine,
> >> > > > > > > > > >>>>>>
> >> > > > > > > > > >>>>>> Thanks for the KIP! I have two questions:
> >> > > > > > > > > >>>>>>
> >> > > > > > > > > >>>>>> 1) For new clients, we can once again return an
> >> error
> >> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> >> > > > > > > > > >>>>>> for sequences
> >> > > > > > > > > >>>>>> that are non-zero when there is no producer state
> >> > > present
> >> > > > on
> >> > > > > > the
> >> > > > > > > > > >>> server.
> >> > > > > > > > > >>>>>> This will indicate we missed the 0 sequence and
> we
> >> > don't
> >> > > > yet
> >> > > > > > > want
> >> > > > > > > > to
> >> > > > > > > > > >>>>> write
> >> > > > > > > > > >>>>>> to the log.
> >> > > > > > > > > >>>>>>
> >> > > > > > > > > >>>>>> I would like to understand the current behavior
> to
> >> > > handle
> >> > > > > > older
> >> > > > > > > > > >>> clients,
> >> > > > > > > > > >>>>>> and if there are any changes we are making. Maybe
> >> I'm
> >> > > > > missing
> >> > > > > > > > > >>> something,
> >> > > > > > > > > >>>>>> but we would want to identify whether we missed
> >> the 0
> >> > > > > sequence
> >> > > > > > > for
> >> > > > > > > > > >>> older
> >> > > > > > > > > >>>>>> clients, no?
> >> > > > > > > > > >>>>>>
> >> > > > > > > > > >>>>>> 2) Upon returning from the transaction
> >> coordinator, we
> >> > > can
> >> > > > > set
> >> > > > > > > the
> >> > > > > > > > > >>>>>> transaction
> >> > > > > > > > > >>>>>> as ongoing on the leader by populating
> >> > > > currentTxnFirstOffset
> >> > > > > > > > > >>>>>> through the typical produce request handling.
> >> > > > > > > > > >>>>>>
> >> > > > > > > > > >>>>>> does the typical produce request path append
> >> records
> >> > to
> >> > > > > local
> >> > > > > > > log
> >> > > > > > > > > >>> along
> >> > > > > > > > > >>>>>> with the currentTxnFirstOffset information? I
> would
> >> > like
> >> > > > to
> >> > > > > > > > > understand
> >> > > > > > > > > >>>>>> when the field is written to disk.
> >> > > > > > > > > >>>>>>
> >> > > > > > > > > >>>>>> Thanks,
> >> > > > > > > > > >>>>>> Jeff
> >> > > > > > > > > >>>>>>
> >> > > > > > > > > >>>>>>
> >> > > > > > > > > >>>>>> On Tue, Nov 22, 2022 at 4:44 PM Artem Livshits
> >> > > > > > > > > >>>>>> <al...@confluent.io.invalid> wrote:
> >> > > > > > > > > >>>>>>
> >> > > > > > > > > >>>>>>> Hi Justine,
> >> > > > > > > > > >>>>>>>
> >> > > > > > > > > >>>>>>> Thank you for the KIP.  I have one question.
> >> > > > > > > > > >>>>>>>
> >> > > > > > > > > >>>>>>> 5) For new clients, we can once again return an
> >> error
> >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> >> > > > > > > > > >>>>>>>
> >> > > > > > > > > >>>>>>> I believe we had problems in the past with
> >> returning
> >> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> >> > > > > > > > > >>>>>>> because it was considered fatal and required
> >> client
> >> > > > > restart.
> >> > > > > > > It
> >> > > > > > > > > >>> would
> >> > > > > > > > > >>>>> be
> >> > > > > > > > > >>>>>>> good to spell out the new client behavior when
> it
> >> > > > receives
> >> > > > > > the
> >> > > > > > > > > error.
> >> > > > > > > > > >>>>>>>
> >> > > > > > > > > >>>>>>> -Artem
> >> > > > > > > > > >>>>>>>
> >> > > > > > > > > >>>>>>> On Tue, Nov 22, 2022 at 10:00 AM Justine Olshan
> >> > > > > > > > > >>>>>>> <jo...@confluent.io.invalid> wrote:
> >> > > > > > > > > >>>>>>>
> >> > > > > > > > > >>>>>>>> Thanks for taking a look Matthias. I've tried
> to
> >> > > answer
> >> > > > > your
> >> > > > > > > > > >>>>> questions
> >> > > > > > > > > >>>>>>>> below:
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>> 10)
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>> Right — so the hanging transaction only occurs
> >> when
> >> > we
> >> > > > > have
> >> > > > > > a
> >> > > > > > > > late
> >> > > > > > > > > >>>>>>> message
> >> > > > > > > > > >>>>>>>> come in and the partition is never added to a
> >> > > > transaction
> >> > > > > > > again.
> >> > > > > > > > > If
> >> > > > > > > > > >>>>> we
> >> > > > > > > > > >>>>>>>> never add the partition to a transaction, we
> will
> >> > > never
> >> > > > > > write
> >> > > > > > > a
> >> > > > > > > > > >>>>> marker
> >> > > > > > > > > >>>>>>> and
> >> > > > > > > > > >>>>>>>> never advance the LSO.
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>> If we do end up adding the partition to the
> >> > > transaction
> >> > > > (I
> >> > > > > > > > suppose
> >> > > > > > > > > >>>>> this
> >> > > > > > > > > >>>>>>> can
> >> > > > > > > > > >>>>>>>> happen before or after the late message comes
> in)
> >> > then
> >> > > > we
> >> > > > > > will
> >> > > > > > > > > >>>>> include
> >> > > > > > > > > >>>>>>> the
> >> > > > > > > > > >>>>>>>> late message in the next (incorrect)
> transaction.
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>> So perhaps it is clearer to make the
> distinction
> >> > > between
> >> > > > > > > > messages
> >> > > > > > > > > >>>>> that
> >> > > > > > > > > >>>>>>>> eventually get added to the transaction (but
> the
> >> > wrong
> >> > > > > one)
> >> > > > > > or
> >> > > > > > > > > >>>>> messages
> >> > > > > > > > > >>>>>>>> that never get added and become hanging.
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>> 20)
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>> The client side change for 2 is removing the
> >> > > > addPartitions
> >> > > > > > to
> >> > > > > > > > > >>>>>> transaction
> >> > > > > > > > > >>>>>>>> call. We don't need to make this from the
> >> producer
> >> > to
> >> > > > the
> >> > > > > > txn
> >> > > > > > > > > >>>>>>> coordinator,
> >> > > > > > > > > >>>>>>>> only server side.
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>> In my opinion, the issue with the
> >> addPartitionsToTxn
> >> > > > call
> >> > > > > > for
> >> > > > > > > > > older
> >> > > > > > > > > >>>>>>> clients
> >> > > > > > > > > >>>>>>>> is that we don't have the epoch bump, so we
> don't
> >> > know
> >> > > > if
> >> > > > > > the
> >> > > > > > > > > >>> message
> >> > > > > > > > > >>>>>>>> belongs to the previous transaction or this
> one.
> >> We
> >> > > need
> >> > > > > to
> >> > > > > > > > check
> >> > > > > > > > > if
> >> > > > > > > > > >>>>>> the
> >> > > > > > > > > >>>>>>>> partition has been added to this transaction.
> Of
> >> > > course,
> >> > > > > > this
> >> > > > > > > > > means
> >> > > > > > > > > >>>>> we
> >> > > > > > > > > >>>>>>>> won't completely cover the case where we have a
> >> > really
> >> > > > > late
> >> > > > > > > > > message
> >> > > > > > > > > >>>>> and
> >> > > > > > > > > >>>>>>> we
> >> > > > > > > > > >>>>>>>> have added the partition to the new
> transaction,
> >> but
> >> > > > > that's
> >> > > > > > > > > >>>>>> unfortunately
> >> > > > > > > > > >>>>>>>> something we will need the new clients to
> cover.
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>> 30)
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>> Transaction is ongoing = partition was added to
> >> > > > > transaction
> >> > > > > > > via
> >> > > > > > > > > >>>>>>>> addPartitionsToTxn. We check this with the
> >> > > > > > > DescribeTransactions
> >> > > > > > > > > >>> call.
> >> > > > > > > > > >>>>>> Let
> >> > > > > > > > > >>>>>>>> me know if this wasn't sufficiently explained
> >> here:
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>
> >> > > > > > > > > >>>>>>
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>> 40)
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>> The idea here is that if any messages somehow
> >> come
> >> > in
> >> > > > > before
> >> > > > > > > we
> >> > > > > > > > > get
> >> > > > > > > > > >>>>> the
> >> > > > > > > > > >>>>>>> new
> >> > > > > > > > > >>>>>>>> epoch to the producer, they will be fenced.
> >> However,
> >> > > if
> >> > > > we
> >> > > > > > > don't
> >> > > > > > > > > >>>>> think
> >> > > > > > > > > >>>>>>> this
> >> > > > > > > > > >>>>>>>> is necessary, it can be discussed
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>> 50)
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>> It should be synchronous because if we have an
> >> event
> >> > > > (ie,
> >> > > > > an
> >> > > > > > > > > error)
> >> > > > > > > > > >>>>>> that
> >> > > > > > > > > >>>>>>>> causes us to need to abort the transaction, we
> >> need
> >> > to
> >> > > > > know
> >> > > > > > > > which
> >> > > > > > > > > >>>>>>>> partitions to send transaction markers to. We
> >> know
> >> > the
> >> > > > > > > > partitions
> >> > > > > > > > > >>>>>> because
> >> > > > > > > > > >>>>>>>> we added them to the coordinator via the
> >> > > > > addPartitionsToTxn
> >> > > > > > > > call.
> >> > > > > > > > > >>>>>>>> Previously we have had asynchronous calls in
> the
> >> > past
> >> > > > (ie,
> >> > > > > > > > writing
> >> > > > > > > > > >>>>> the
> >> > > > > > > > > >>>>>>>> commit markers when the transaction is
> completed)
> >> > but
> >> > > > > often
> >> > > > > > > this
> >> > > > > > > > > >>> just
> >> > > > > > > > > >>>>>>>> causes confusion as we need to wait for some
> >> > > operations
> >> > > > to
> >> > > > > > > > > complete.
> >> > > > > > > > > >>>>> In
> >> > > > > > > > > >>>>>>> the
> >> > > > > > > > > >>>>>>>> writing commit markers case, clients often see
> >> > > > > > > > > >>>>> CONCURRENT_TRANSACTIONs
> >> > > > > > > > > >>>>>>>> error messages and that can be confusing. For
> >> that
> >> > > > reason,
> >> > > > > > it
> >> > > > > > > > may
> >> > > > > > > > > be
> >> > > > > > > > > >>>>>>>> simpler to just have synchronous calls —
> >> especially
> >> > if
> >> > > > we
> >> > > > > > need
> >> > > > > > > > to
> >> > > > > > > > > >>>>> block
> >> > > > > > > > > >>>>>>> on
> >> > > > > > > > > >>>>>>>> some operation's completion anyway before we
> can
> >> > start
> >> > > > the
> >> > > > > > > next
> >> > > > > > > > > >>>>>>>> transaction. And yes, I meant coordinator. I
> will
> >> > fix
> >> > > > > that.
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>> 60)
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>> When we are checking if the transaction is
> >> ongoing,
> >> > we
> >> > > > > need
> >> > > > > > to
> >> > > > > > > > > make
> >> > > > > > > > > >>> a
> >> > > > > > > > > >>>>>>> round
> >> > > > > > > > > >>>>>>>> trip from the leader partition to the
> transaction
> >> > > > > > coordinator.
> >> > > > > > > > In
> >> > > > > > > > > >>> the
> >> > > > > > > > > >>>>>>> time
> >> > > > > > > > > >>>>>>>> we are waiting for this message to come back,
> in
> >> > > theory
> >> > > > we
> >> > > > > > > could
> >> > > > > > > > > >>> have
> >> > > > > > > > > >>>>>>> sent
> >> > > > > > > > > >>>>>>>> a commit/abort call that would make the
> original
> >> > > result
> >> > > > of
> >> > > > > > the
> >> > > > > > > > > check
> >> > > > > > > > > >>>>>> out
> >> > > > > > > > > >>>>>>> of
> >> > > > > > > > > >>>>>>>> date. That is why we can check the leader state
> >> > before
> >> > > > we
> >> > > > > > > write
> >> > > > > > > > to
> >> > > > > > > > > >>>>> the
> >> > > > > > > > > >>>>>>> log.
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>> I'm happy to update the KIP if some of these
> >> things
> >> > > were
> >> > > > > not
> >> > > > > > > > > clear.
> >> > > > > > > > > >>>>>>>> Thanks,
> >> > > > > > > > > >>>>>>>> Justine
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>> On Mon, Nov 21, 2022 at 7:11 PM Matthias J.
> Sax <
> >> > > > > > > > mjsax@apache.org
> >> > > > > > > > > >
> >> > > > > > > > > >>>>>>> wrote:
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>>> Thanks for the KIP.
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>> Couple of clarification questions (I am not a
> >> > broker
> >> > > > > expert
> >> > > > > > > do
> >> > > > > > > > > >>>>> maybe
> >> > > > > > > > > >>>>>>>>> some question are obvious for others, but not
> >> for
> >> > me
> >> > > > with
> >> > > > > > my
> >> > > > > > > > lack
> >> > > > > > > > > >>>>> of
> >> > > > > > > > > >>>>>>>>> broker knowledge).
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>> (10)
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>>> 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.
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>> What happens if the message come in before the
> >> next
> >> > > > > > > > > >>>>>> addPartitionsToTxn
> >> > > > > > > > > >>>>>>>>> request? It seems the broker hosting the data
> >> > > > partitions
> >> > > > > > > won't
> >> > > > > > > > > know
> >> > > > > > > > > >>>>>>>>> anything about it and append it to the
> >> partition,
> >> > > too?
> >> > > > > What
> >> > > > > > > is
> >> > > > > > > > > the
> >> > > > > > > > > >>>>>>>>> difference between both cases?
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>> Also, it seems a TX would only hang, if there
> >> is no
> >> > > > > > following
> >> > > > > > > > TX
> >> > > > > > > > > >>>>> that
> >> > > > > > > > > >>>>>>> is
> >> > > > > > > > > >>>>>>>>> either committer or aborted? Thus, for the
> case
> >> > > above,
> >> > > > > the
> >> > > > > > TX
> >> > > > > > > > > might
> >> > > > > > > > > >>>>>>>>> actually not hang (of course, we might get an
> >> EOS
> >> > > > > violation
> >> > > > > > > if
> >> > > > > > > > > the
> >> > > > > > > > > >>>>>>> first
> >> > > > > > > > > >>>>>>>>> TX was aborted and the second committed, or
> the
> >> > other
> >> > > > way
> >> > > > > > > > > around).
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>> (20)
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>>> Of course, 1 and 2 require client-side
> >> changes, so
> >> > > for
> >> > > > > > older
> >> > > > > > > > > >>>>>> clients,
> >> > > > > > > > > >>>>>>>>> those approaches won’t apply.
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>> For (1) I understand why a client change is
> >> > > necessary,
> >> > > > > but
> >> > > > > > > not
> >> > > > > > > > > sure
> >> > > > > > > > > >>>>>> why
> >> > > > > > > > > >>>>>>>>> we need a client change for (2). Can you
> >> elaborate?
> >> > > --
> >> > > > > > Later
> >> > > > > > > > you
> >> > > > > > > > > >>>>>>> explain
> >> > > > > > > > > >>>>>>>>> that we should send a
> >> DescribeTransactionRequest,
> >> > > but I
> >> > > > > am
> >> > > > > > > not
> >> > > > > > > > > sure
> >> > > > > > > > > >>>>>>> why?
> >> > > > > > > > > >>>>>>>>> Can't we not just do an implicit
> >> AddPartiitonToTx,
> >> > > too?
> >> > > > > If
> >> > > > > > > the
> >> > > > > > > > > old
> >> > > > > > > > > >>>>>>>>> producer correctly registered the partition
> >> > already,
> >> > > > the
> >> > > > > > > > > >>>>>> TX-coordinator
> >> > > > > > > > > >>>>>>>>> can just ignore it as it's an idempotent
> >> operation?
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>> (30)
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>>> To cover older clients, we will ensure a
> >> > transaction
> >> > > > is
> >> > > > > > > > ongoing
> >> > > > > > > > > >>>>>>> before
> >> > > > > > > > > >>>>>>>>> we write to a transaction
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>> Not sure what you mean by this? Can you
> >> elaborate?
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>> (40)
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>>> [the TX-coordinator] will write the prepare
> >> commit
> >> > > > > message
> >> > > > > > > > with
> >> > > > > > > > > a
> >> > > > > > > > > >>>>>>>> bumped
> >> > > > > > > > > >>>>>>>>> epoch and send WriteTxnMarkerRequests with the
> >> > bumped
> >> > > > > > epoch.
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>> Why do we use the bumped epoch for both? It
> >> seems
> >> > > more
> >> > > > > > > > intuitive
> >> > > > > > > > > to
> >> > > > > > > > > >>>>>> use
> >> > > > > > > > > >>>>>>>>> the current epoch, and only return the bumped
> >> epoch
> >> > > to
> >> > > > > the
> >> > > > > > > > > >>>>> producer?
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>> (50) "Implicit AddPartitionToTransaction"
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>> Why does the implicitly sent request need to
> be
> >> > > > > > synchronous?
> >> > > > > > > > The
> >> > > > > > > > > >>>>> KIP
> >> > > > > > > > > >>>>>>>>> also says
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>>> in case we need to abort and need to know
> which
> >> > > > > partitions
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>> What do you mean by this?
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>>> we don’t want to write to it before we store
> in
> >> > the
> >> > > > > > > > transaction
> >> > > > > > > > > >>>>>>> manager
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>> Do you mean TX-coordinator instead of
> "manager"?
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>> (60)
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>> For older clients and ensuring that the TX is
> >> > > ongoing,
> >> > > > > you
> >> > > > > > > > > >>>>> describe a
> >> > > > > > > > > >>>>>>>>> race condition. I am not sure if I can follow
> >> here.
> >> > > Can
> >> > > > > you
> >> > > > > > > > > >>>>>> elaborate?
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>> -Matthias
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>> On 11/18/22 1:21 PM, Justine Olshan wrote:
> >> > > > > > > > > >>>>>>>>>> Hey all!
> >> > > > > > > > > >>>>>>>>>>
> >> > > > > > > > > >>>>>>>>>> I'd like to start a discussion on my proposal
> >> to
> >> > add
> >> > > > > some
> >> > > > > > > > > >>>>>> server-side
> >> > > > > > > > > >>>>>>>>>> checks on transactions to avoid hanging
> >> > > transactions.
> >> > > > I
> >> > > > > > know
> >> > > > > > > > > this
> >> > > > > > > > > >>>>>> has
> >> > > > > > > > > >>>>>>>>> been
> >> > > > > > > > > >>>>>>>>>> an issue for some time, so I really hope this
> >> KIP
> >> > > will
> >> > > > > be
> >> > > > > > > > > helpful
> >> > > > > > > > > >>>>>> for
> >> > > > > > > > > >>>>>>>>> many
> >> > > > > > > > > >>>>>>>>>> users of EOS.
> >> > > > > > > > > >>>>>>>>>>
> >> > > > > > > > > >>>>>>>>>> The KIP includes changes that will be
> >> compatible
> >> > > with
> >> > > > > old
> >> > > > > > > > > clients
> >> > > > > > > > > >>>>>> and
> >> > > > > > > > > >>>>>>>>>> changes to improve performance and
> correctness
> >> on
> >> > > new
> >> > > > > > > clients.
> >> > > > > > > > > >>>>>>>>>>
> >> > > > > > > > > >>>>>>>>>> Please take a look and leave any comments you
> >> may
> >> > > > have!
> >> > > > > > > > > >>>>>>>>>>
> >> > > > > > > > > >>>>>>>>>> KIP:
> >> > > > > > > > > >>>>>>>>>>
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>
> >> > > > > > > > > >>>>>>
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> >> > > > > > > > > >>>>>>>>>> JIRA:
> >> > > > https://issues.apache.org/jira/browse/KAFKA-14402
> >> > > > > > > > > >>>>>>>>>>
> >> > > > > > > > > >>>>>>>>>> Thanks!
> >> > > > > > > > > >>>>>>>>>> Justine
> >> > > > > > > > > >>>>>>>>>>
> >> > > > > > > > > >>>>>>>>>
> >> > > > > > > > > >>>>>>>>
> >> > > > > > > > > >>>>>>>
> >> > > > > > > > > >>>>>>
> >> > > > > > > > > >>>>>
> >> > > > > > > > > >>>>
> >> > > > > > > > > >>>
> >> > > > > > > > > >>
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
As a follow up, I was just thinking about the batching a bit more.
I suppose if we have one request in flight and we queue up the other
produce requests in some sort of purgatory, we could send information out
for all of them rather than one by one. So that would be a benefit of
batching partitions to add per transaction.

I'll need to think a bit more on the design of this part of the KIP, and
will update the KIP in the next few days.

Thanks,
Justine

On Fri, Jan 6, 2023 at 10:22 AM Justine Olshan <jo...@confluent.io> wrote:

> Hey Jason -- thanks for the input -- I was just digging a bit deeper into
> the design + implementation of the validation calls here and what you say
> makes sense.
>
> I was wondering about compatibility here. When we send requests
> between brokers, we want to ensure that the receiving broker understands
> the request (specifically the new fields). Typically this is done via
> IBP/metadata version.
> I'm trying to think if there is a way around it but I'm not sure there is.
>
> As for the improvements -- can you clarify how the multiple transactional
> IDs would help here? Were you thinking of a case where we wait/batch
> multiple produce requests together? My understanding for now was 1
> transactional ID and one validation per 1 produce request.
>
> Finally with respect to the authorizations, I think it makes sense to skip
> topic authorizations, but I'm a bit confused by the "leader ID" field.
> Wouldn't we just want to flag the request as from a broker (does it matter
> which one?).
>
> I think I want to adopt these suggestions, just had a few questions on the
> details.
>
> Thanks,
> Justine
>
> On Thu, Jan 5, 2023 at 5:05 PM Jason Gustafson <ja...@confluent.io.invalid>
> wrote:
>
>> Hi Justine,
>>
>> Thanks for the proposal.
>>
>> I was thinking about the implementation a little bit. In the current
>> proposal, the behavior depends on whether we have an old or new client.
>> For
>> old clients, we send `DescribeTransactions` and verify the result and for
>> new clients, we send `AddPartitionsToTxn`. We might be able to simplify
>> the
>> implementation if we can use the same request type. For example, what if
>> we
>> bump the protocol version for `AddPartitionsToTxn` and add a
>> `validateOnly`
>> flag? For older versions, we can set `validateOnly=true` so that the
>> request only returns successfully if the partition had already been added.
>> For new versions, we can set `validateOnly=false` and the partition will
>> be
>> added to the transaction. The other slightly annoying thing that this
>> would
>> get around is the need to collect the transaction state for all partitions
>> even when we only care about a subset.
>>
>> Some additional improvements to consider:
>>
>> - We can give `AddPartitionsToTxn` better batch support for inter-broker
>> usage. Currently we only allow one `TransactionalId` to be specified, but
>> the broker may get some benefit being able to batch across multiple
>> transactions.
>> - Another small improvement is skipping topic authorization checks for
>> `AddPartitionsToTxn` when the request is from a broker. Perhaps we can add
>> a field for the `LeaderId` or something like that and require CLUSTER
>> permission when set.
>>
>> Best,
>> Jason
>>
>>
>>
>> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao <ju...@confluent.io.invalid> wrote:
>>
>> > Hi, Justine,
>> >
>> > Thanks for the explanation. It makes sense to me now.
>> >
>> > Jun
>> >
>> > On Mon, Dec 19, 2022 at 1:42 PM Justine Olshan
>> > <jo...@confluent.io.invalid>
>> > wrote:
>> >
>> > > Hi Jun,
>> > >
>> > > My understanding of the mechanism is that when we get to the last
>> epoch,
>> > we
>> > > increment to the fencing/last epoch and if any further requests come
>> in
>> > for
>> > > this producer ID they are fenced. Then the producer gets a new ID and
>> > > restarts with epoch/sequence 0. The fenced epoch sticks around for the
>> > > duration of producer.id.expiration.ms and blocks any late messages
>> > there.
>> > > The new ID will get to take advantage of the improved semantics around
>> > > non-zero start sequences. So I think we are covered.
>> > >
>> > > The only potential issue is overloading the cache, but hopefully the
>> > > improvements (lowered producer.id.expiration.ms) will help with that.
>> > Let
>> > > me know if you still have concerns.
>> > >
>> > > Thanks,
>> > > Justine
>> > >
>> > > On Mon, Dec 19, 2022 at 10:24 AM Jun Rao <ju...@confluent.io.invalid>
>> > wrote:
>> > >
>> > > > Hi, Justine,
>> > > >
>> > > > Thanks for the explanation.
>> > > >
>> > > > 70. The proposed fencing logic doesn't apply when pid changes, is
>> that
>> > > > right? If so, I am not sure how complete we are addressing this
>> issue
>> > if
>> > > > the pid changes more frequently.
>> > > >
>> > > > Thanks,
>> > > >
>> > > > Jun
>> > > >
>> > > >
>> > > >
>> > > > On Fri, Dec 16, 2022 at 9:16 AM Justine Olshan
>> > > > <jo...@confluent.io.invalid>
>> > > > wrote:
>> > > >
>> > > > > Hi Jun,
>> > > > >
>> > > > > Thanks for replying!
>> > > > >
>> > > > > 70.We already do the overflow mechanism, so my change would just
>> make
>> > > it
>> > > > > happen more often.
>> > > > > I was also not suggesting a new field in the log, but in the
>> > response,
>> > > > > which would be gated by the client version. Sorry if something
>> there
>> > is
>> > > > > unclear. I think we are starting to diverge.
>> > > > > The goal of this KIP is to not change to the marker format at all.
>> > > > >
>> > > > > 71. Yes, I guess I was going under the assumption that the log
>> would
>> > > just
>> > > > > look at its last epoch and treat it as the current epoch. I
>> suppose
>> > we
>> > > > can
>> > > > > have some special logic that if the last epoch was on a marker we
>> > > > actually
>> > > > > expect the next epoch or something like that. We just need to
>> > > distinguish
>> > > > > based on whether we had a commit/abort marker.
>> > > > >
>> > > > > 72.
>> > > > > > if the producer epoch hasn't been bumped on the
>> > > > > broker, it seems that the stucked message will fail the sequence
>> > > > validation
>> > > > > and will be ignored. If the producer epoch has been bumped, we
>> ignore
>> > > the
>> > > > > sequence check and the stuck message could be appended to the log.
>> > So,
>> > > is
>> > > > > the latter case that we want to guard?
>> > > > >
>> > > > > I'm not sure I follow that "the message will fail the sequence
>> > > > validation".
>> > > > > In some of these cases, we had an abort marker (due to an error)
>> and
>> > > then
>> > > > > the late message comes in with the correct sequence number. This
>> is a
>> > > > case
>> > > > > covered by the KIP.
>> > > > > The latter case is actually not something we've considered here. I
>> > > think
>> > > > > generally when we bump the epoch, we are accepting that the
>> sequence
>> > > does
>> > > > > not need to be checked anymore. My understanding is also that we
>> > don't
>> > > > > typically bump epoch mid transaction (based on a quick look at the
>> > > code)
>> > > > > but let me know if that is the case.
>> > > > >
>> > > > > Thanks,
>> > > > > Justine
>> > > > >
>> > > > > On Thu, Dec 15, 2022 at 12:23 PM Jun Rao <jun@confluent.io.invalid
>> >
>> > > > wrote:
>> > > > >
>> > > > > > Hi, Justine,
>> > > > > >
>> > > > > > Thanks for the reply.
>> > > > > >
>> > > > > > 70. Assigning a new pid on int overflow seems a bit hacky. If we
>> > > need a
>> > > > > txn
>> > > > > > level id, it will be better to model this explicitly. Adding a
>> new
>> > > > field
>> > > > > > would require a bit more work since it requires a new txn marker
>> > > format
>> > > > > in
>> > > > > > the log. So, we probably need to guard it with an IBP or
>> metadata
>> > > > version
>> > > > > > and document the impact on downgrade once the new format is
>> written
>> > > to
>> > > > > the
>> > > > > > log.
>> > > > > >
>> > > > > > 71. Hmm, once the marker is written, the partition will expect
>> the
>> > > next
>> > > > > > append to be on the next epoch. Does that cover the case you
>> > > mentioned?
>> > > > > >
>> > > > > > 72. Also, just to be clear on the stucked message issue
>> described
>> > in
>> > > > the
>> > > > > > motivation. With EoS, we also validate the sequence id for
>> > > idempotency.
>> > > > > So,
>> > > > > > with the current logic, if the producer epoch hasn't been
>> bumped on
>> > > the
>> > > > > > broker, it seems that the stucked message will fail the sequence
>> > > > > validation
>> > > > > > and will be ignored. If the producer epoch has been bumped, we
>> > ignore
>> > > > the
>> > > > > > sequence check and the stuck message could be appended to the
>> log.
>> > > So,
>> > > > is
>> > > > > > the latter case that we want to guard?
>> > > > > >
>> > > > > > Thanks,
>> > > > > >
>> > > > > > Jun
>> > > > > >
>> > > > > > On Wed, Dec 14, 2022 at 10:44 AM Justine Olshan
>> > > > > > <jo...@confluent.io.invalid> wrote:
>> > > > > >
>> > > > > > > Matthias — thanks again for taking time to look a this. You
>> said:
>> > > > > > >
>> > > > > > > > My proposal was only focusing to avoid dangling
>> > > > > > >
>> > > > > > > transactions if records are added without registered
>> partition.
>> > --
>> > > > > Maybe
>> > > > > > >
>> > > > > > > you can add a few more details to the KIP about this scenario
>> for
>> > > > > better
>> > > > > > >
>> > > > > > > documentation purpose?
>> > > > > > >
>> > > > > > >
>> > > > > > > I'm not sure I understand what you mean here. The motivation
>> > > section
>> > > > > > > describes two scenarios about how the record can be added
>> > without a
>> > > > > > > registered partition:
>> > > > > > >
>> > > > > > >
>> > > > > > > > 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.
>> > > > > > >
>> > > > > > >
>> > > > > > > > Another way hanging transactions can occur is that a client
>> is
>> > > > buggy
>> > > > > > and
>> > > > > > > may somehow try to write to a partition before it adds the
>> > > partition
>> > > > to
>> > > > > > the
>> > > > > > > transaction.
>> > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > > > For the first example of this would it be helpful to say that
>> > this
>> > > > > > message
>> > > > > > > comes in after the abort, but before the partition is added to
>> > the
>> > > > next
>> > > > > > > transaction so it becomes "hanging." Perhaps the next sentence
>> > > > > describing
>> > > > > > > the message becoming part of the next transaction (a different
>> > > case)
>> > > > > was
>> > > > > > > not properly differentiated.
>> > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > > > Jun — thanks for reading the KIP.
>> > > > > > >
>> > > > > > > 70. The int typing was a concern. Currently we have a
>> mechanism
>> > in
>> > > > > place
>> > > > > > to
>> > > > > > > fence the final epoch when the epoch is about to overflow and
>> > > assign
>> > > > a
>> > > > > > new
>> > > > > > > producer ID with epoch 0. Of course, this is a bit tricky
>> when it
>> > > > comes
>> > > > > > to
>> > > > > > > the response back to the client.
>> > > > > > > Making this a long could be another option, but I wonder are
>> > there
>> > > > any
>> > > > > > > implications on changing this field if the epoch is persisted
>> to
>> > > > disk?
>> > > > > > I'd
>> > > > > > > need to check the usages.
>> > > > > > >
>> > > > > > > 71.This was something Matthias asked about as well. I was
>> > > > considering a
>> > > > > > > possible edge case where a produce request from a new
>> transaction
>> > > > > somehow
>> > > > > > > gets sent right after the marker is written, but before the
>> > > producer
>> > > > is
>> > > > > > > alerted of the newly bumped epoch. In this case, we may
>> include
>> > > this
>> > > > > > record
>> > > > > > > when we don't want to. I suppose we could try to do something
>> > > client
>> > > > > side
>> > > > > > > to bump the epoch after sending an endTxn as well in this
>> > scenario
>> > > —
>> > > > > but
>> > > > > > I
>> > > > > > > wonder how it would work when the server is aborting based on
>> a
>> > > > > > server-side
>> > > > > > > error. I could also be missing something and this scenario is
>> > > > actually
>> > > > > > not
>> > > > > > > possible.
>> > > > > > >
>> > > > > > > Thanks again to everyone reading and commenting. Let me know
>> > about
>> > > > any
>> > > > > > > further questions or comments.
>> > > > > > >
>> > > > > > > Justine
>> > > > > > >
>> > > > > > > On Wed, Dec 14, 2022 at 9:41 AM Jun Rao
>> <jun@confluent.io.invalid
>> > >
>> > > > > > wrote:
>> > > > > > >
>> > > > > > > > Hi, Justine,
>> > > > > > > >
>> > > > > > > > Thanks for the KIP. A couple of comments.
>> > > > > > > >
>> > > > > > > > 70. Currently, the producer epoch is an int. I am not sure
>> if
>> > > it's
>> > > > > > enough
>> > > > > > > > to accommodate all transactions in the lifetime of a
>> producer.
>> > > > Should
>> > > > > > we
>> > > > > > > > change that to a long or add a new long field like txnId?
>> > > > > > > >
>> > > > > > > > 71. "it will write the prepare commit message with a bumped
>> > epoch
>> > > > and
>> > > > > > > send
>> > > > > > > > WriteTxnMarkerRequests with the bumped epoch." Hmm, the
>> epoch
>> > is
>> > > > > > > associated
>> > > > > > > > with the current txn right? So, it seems weird to write a
>> > commit
>> > > > > > message
>> > > > > > > > with a bumped epoch. Should we only bump up the epoch in
>> > > > > EndTxnResponse
>> > > > > > > and
>> > > > > > > > rename the field to sth like nextProducerEpoch?
>> > > > > > > >
>> > > > > > > > Thanks,
>> > > > > > > >
>> > > > > > > > Jun
>> > > > > > > >
>> > > > > > > >
>> > > > > > > >
>> > > > > > > > On Mon, Dec 12, 2022 at 8:54 PM Matthias J. Sax <
>> > > mjsax@apache.org>
>> > > > > > > wrote:
>> > > > > > > >
>> > > > > > > > > Thanks for the background.
>> > > > > > > > >
>> > > > > > > > > 20/30: SGTM. My proposal was only focusing to avoid
>> dangling
>> > > > > > > > > transactions if records are added without registered
>> > partition.
>> > > > --
>> > > > > > > Maybe
>> > > > > > > > > you can add a few more details to the KIP about this
>> scenario
>> > > for
>> > > > > > > better
>> > > > > > > > > documentation purpose?
>> > > > > > > > >
>> > > > > > > > > 40: I think you hit a fair point about race conditions or
>> > > client
>> > > > > bugs
>> > > > > > > > > (incorrectly not bumping the epoch). The
>> complexity/confusion
>> > > for
>> > > > > > using
>> > > > > > > > > the bumped epoch I see, is mainly for internal debugging,
>> ie,
>> > > > > > > inspecting
>> > > > > > > > > log segment dumps -- it seems harder to reason about the
>> > system
>> > > > for
>> > > > > > us
>> > > > > > > > > humans. But if we get better guarantees, it would be
>> worth to
>> > > use
>> > > > > the
>> > > > > > > > > bumped epoch.
>> > > > > > > > >
>> > > > > > > > > 60: as I mentioned already, I don't know the broker
>> internals
>> > > to
>> > > > > > > provide
>> > > > > > > > > more input. So if nobody else chimes in, we should just
>> move
>> > > > > forward
>> > > > > > > > > with your proposal.
>> > > > > > > > >
>> > > > > > > > >
>> > > > > > > > > -Matthias
>> > > > > > > > >
>> > > > > > > > >
>> > > > > > > > > On 12/6/22 4:22 PM, Justine Olshan wrote:
>> > > > > > > > > > Hi all,
>> > > > > > > > > > After Artem's questions about error behavior, I've
>> > > re-evaluated
>> > > > > the
>> > > > > > > > > > unknown producer ID exception and had some discussions
>> > > offline.
>> > > > > > > > > >
>> > > > > > > > > > I think generally it makes sense to simplify error
>> handling
>> > > in
>> > > > > > cases
>> > > > > > > > like
>> > > > > > > > > > this and the UNKNOWN_PRODUCER_ID error has a pretty long
>> > and
>> > > > > > > > complicated
>> > > > > > > > > > history. Because of this, I propose adding a new error
>> code
>> > > > > > > > > ABORTABLE_ERROR
>> > > > > > > > > > that when encountered by new clients (gated by the
>> produce
>> > > > > request
>> > > > > > > > > version)
>> > > > > > > > > > will simply abort the transaction. This allows the
>> server
>> > to
>> > > > have
>> > > > > > > some
>> > > > > > > > > say
>> > > > > > > > > > in whether the client aborts and makes handling much
>> > simpler.
>> > > > In
>> > > > > > the
>> > > > > > > > > > future, we can also use this error in other situations
>> > where
>> > > we
>> > > > > > want
>> > > > > > > to
>> > > > > > > > > > abort the transactions. We can even use on other apis.
>> > > > > > > > > >
>> > > > > > > > > > I've added this to the KIP. Let me know if there are any
>> > > > > questions
>> > > > > > or
>> > > > > > > > > > issues.
>> > > > > > > > > >
>> > > > > > > > > > Justine
>> > > > > > > > > >
>> > > > > > > > > > On Fri, Dec 2, 2022 at 10:22 AM Justine Olshan <
>> > > > > > jolshan@confluent.io
>> > > > > > > >
>> > > > > > > > > wrote:
>> > > > > > > > > >
>> > > > > > > > > >> Hey Matthias,
>> > > > > > > > > >>
>> > > > > > > > > >>
>> > > > > > > > > >> 20/30 — Maybe I also didn't express myself clearly. For
>> > > older
>> > > > > > > clients
>> > > > > > > > we
>> > > > > > > > > >> don't have a way to distinguish between a previous and
>> the
>> > > > > current
>> > > > > > > > > >> transaction since we don't have the epoch bump. This
>> means
>> > > > that
>> > > > > a
>> > > > > > > late
>> > > > > > > > > >> message from the previous transaction may be added to
>> the
>> > > new
>> > > > > one.
>> > > > > > > > With
>> > > > > > > > > >> older clients — we can't guarantee this won't happen
>> if we
>> > > > > already
>> > > > > > > > sent
>> > > > > > > > > the
>> > > > > > > > > >> addPartitionsToTxn call (why we make changes for the
>> newer
>> > > > > client)
>> > > > > > > but
>> > > > > > > > > we
>> > > > > > > > > >> can at least gate some by ensuring that the partition
>> has
>> > > been
>> > > > > > added
>> > > > > > > > to
>> > > > > > > > > the
>> > > > > > > > > >> transaction. The rationale here is that there are
>> likely
>> > > LESS
>> > > > > late
>> > > > > > > > > arrivals
>> > > > > > > > > >> as time goes on, so hopefully most late arrivals will
>> come
>> > > in
>> > > > > > BEFORE
>> > > > > > > > the
>> > > > > > > > > >> addPartitionsToTxn call. Those that arrive before will
>> be
>> > > > > properly
>> > > > > > > > gated
>> > > > > > > > > >> with the describeTransactions approach.
>> > > > > > > > > >>
>> > > > > > > > > >> If we take the approach you suggested, ANY late arrival
>> > > from a
>> > > > > > > > previous
>> > > > > > > > > >> transaction will be added. And we don't want that. I
>> also
>> > > > don't
>> > > > > > see
>> > > > > > > > any
>> > > > > > > > > >> benefit in sending addPartitionsToTxn over the
>> > describeTxns
>> > > > > call.
>> > > > > > > They
>> > > > > > > > > will
>> > > > > > > > > >> both be one extra RPC to the Txn coordinator.
>> > > > > > > > > >>
>> > > > > > > > > >>
>> > > > > > > > > >> To be clear — newer clients will use addPartitionsToTxn
>> > > > instead
>> > > > > of
>> > > > > > > the
>> > > > > > > > > >> DescribeTxns.
>> > > > > > > > > >>
>> > > > > > > > > >>
>> > > > > > > > > >> 40)
>> > > > > > > > > >> My concern is that if we have some delay in the client
>> to
>> > > bump
>> > > > > the
>> > > > > > > > > epoch,
>> > > > > > > > > >> it could continue to send epoch 73 and those records
>> would
>> > > not
>> > > > > be
>> > > > > > > > > fenced.
>> > > > > > > > > >> Perhaps this is not an issue if we don't allow the next
>> > > > produce
>> > > > > to
>> > > > > > > go
>> > > > > > > > > >> through before the EndTxn request returns. I'm also
>> > thinking
>> > > > > about
>> > > > > > > > > cases of
>> > > > > > > > > >> failure. I will need to think on this a bit.
>> > > > > > > > > >>
>> > > > > > > > > >> I wasn't sure if it was that confusing. But if we
>> think it
>> > > is,
>> > > > > we
>> > > > > > > can
>> > > > > > > > > >> investigate other ways.
>> > > > > > > > > >>
>> > > > > > > > > >>
>> > > > > > > > > >> 60)
>> > > > > > > > > >>
>> > > > > > > > > >> I'm not sure these are the same purgatories since one
>> is a
>> > > > > produce
>> > > > > > > > > >> purgatory (I was planning on using a callback rather
>> than
>> > > > > > purgatory)
>> > > > > > > > and
>> > > > > > > > > >> the other is simply a request to append to the log. Not
>> > sure
>> > > > we
>> > > > > > have
>> > > > > > > > any
>> > > > > > > > > >> structure here for ordering, but my understanding is
>> that
>> > > the
>> > > > > > broker
>> > > > > > > > > could
>> > > > > > > > > >> handle the write request before it hears back from the
>> Txn
>> > > > > > > > Coordinator.
>> > > > > > > > > >>
>> > > > > > > > > >> Let me know if I misunderstood something or something
>> was
>> > > > > unclear.
>> > > > > > > > > >>
>> > > > > > > > > >> Justine
>> > > > > > > > > >>
>> > > > > > > > > >> On Thu, Dec 1, 2022 at 12:15 PM Matthias J. Sax <
>> > > > > mjsax@apache.org
>> > > > > > >
>> > > > > > > > > wrote:
>> > > > > > > > > >>
>> > > > > > > > > >>> Thanks for the details Justine!
>> > > > > > > > > >>>
>> > > > > > > > > >>>> 20)
>> > > > > > > > > >>>>
>> > > > > > > > > >>>> The client side change for 2 is removing the
>> > addPartitions
>> > > > to
>> > > > > > > > > >>> transaction
>> > > > > > > > > >>>> call. We don't need to make this from the producer to
>> > the
>> > > > txn
>> > > > > > > > > >>> coordinator,
>> > > > > > > > > >>>> only server side.
>> > > > > > > > > >>>
>> > > > > > > > > >>> I think I did not express myself clearly. I understand
>> > that
>> > > > we
>> > > > > > can
>> > > > > > > > (and
>> > > > > > > > > >>> should) change the producer to not send the
>> > `addPartitions`
>> > > > > > request
>> > > > > > > > any
>> > > > > > > > > >>> longer. But I don't thinks it's requirement to change
>> the
>> > > > > broker?
>> > > > > > > > > >>>
>> > > > > > > > > >>> What I am trying to say is: as a safe-guard and
>> > improvement
>> > > > for
>> > > > > > > older
>> > > > > > > > > >>> producers, the partition leader can just send the
>> > > > > `addPartitions`
>> > > > > > > > > >>> request to the TX-coordinator in any case -- if the
>> old
>> > > > > producer
>> > > > > > > > > >>> correctly did send the `addPartition` request to the
>> > > > > > TX-coordinator
>> > > > > > > > > >>> already, the TX-coordinator can just "ignore" is as
>> > > > idempotent.
>> > > > > > > > > However,
>> > > > > > > > > >>> if the old producer has a bug and did forget to sent
>> the
>> > > > > > > > `addPartition`
>> > > > > > > > > >>> request, we would now ensure that the partition is
>> indeed
>> > > > added
>> > > > > > to
>> > > > > > > > the
>> > > > > > > > > >>> TX and thus fix a potential producer bug (even if we
>> > don't
>> > > > get
>> > > > > > the
>> > > > > > > > > >>> fencing via the bump epoch). -- It seems to be a good
>> > > > > > improvement?
>> > > > > > > Or
>> > > > > > > > > is
>> > > > > > > > > >>> there a reason to not do this?
>> > > > > > > > > >>>
>> > > > > > > > > >>>
>> > > > > > > > > >>>
>> > > > > > > > > >>>> 30)
>> > > > > > > > > >>>>
>> > > > > > > > > >>>> Transaction is ongoing = partition was added to
>> > > transaction
>> > > > > via
>> > > > > > > > > >>>> addPartitionsToTxn. We check this with the
>> > > > > DescribeTransactions
>> > > > > > > > call.
>> > > > > > > > > >>> Let
>> > > > > > > > > >>>> me know if this wasn't sufficiently explained here:
>> > > > > > > > > >>>
>> > > > > > > > > >>> If we do what I propose in (20), we don't really need
>> to
>> > > make
>> > > > > > this
>> > > > > > > > > >>> `DescribeTransaction` call, as the partition leader
>> adds
>> > > the
>> > > > > > > > partition
>> > > > > > > > > >>> for older clients and we get this check for free.
>> > > > > > > > > >>>
>> > > > > > > > > >>>
>> > > > > > > > > >>>> 40)
>> > > > > > > > > >>>>
>> > > > > > > > > >>>> The idea here is that if any messages somehow come in
>> > > before
>> > > > > we
>> > > > > > > get
>> > > > > > > > > the
>> > > > > > > > > >>> new
>> > > > > > > > > >>>> epoch to the producer, they will be fenced. However,
>> if
>> > we
>> > > > > don't
>> > > > > > > > think
>> > > > > > > > > >>> this
>> > > > > > > > > >>>> is necessary, it can be discussed
>> > > > > > > > > >>>
>> > > > > > > > > >>> I agree that we should have epoch fencing. My
>> question is
>> > > > > > > different:
>> > > > > > > > > >>> Assume we are at epoch 73, and we have an ongoing
>> > > > transaction,
>> > > > > > that
>> > > > > > > > is
>> > > > > > > > > >>> committed. It seems natural to write the "prepare
>> commit"
>> > > > > marker
>> > > > > > > and
>> > > > > > > > > the
>> > > > > > > > > >>> `WriteTxMarkerRequest` both with epoch 73, too, as it
>> > > belongs
>> > > > > to
>> > > > > > > the
>> > > > > > > > > >>> current transaction. Of course, we now also bump the
>> > epoch
>> > > > and
>> > > > > > > expect
>> > > > > > > > > >>> the next requests to have epoch 74, and would reject
>> an
>> > > > request
>> > > > > > > with
>> > > > > > > > > >>> epoch 73, as the corresponding TX for epoch 73 was
>> > already
>> > > > > > > committed.
>> > > > > > > > > >>>
>> > > > > > > > > >>> It seems you propose to write the "prepare commit
>> marker"
>> > > and
>> > > > > > > > > >>> `WriteTxMarkerRequest` with epoch 74 though, what
>> would
>> > > work,
>> > > > > but
>> > > > > > > it
>> > > > > > > > > >>> seems confusing. Is there a reason why we would use
>> the
>> > > > bumped
>> > > > > > > epoch
>> > > > > > > > 74
>> > > > > > > > > >>> instead of the current epoch 73?
>> > > > > > > > > >>>
>> > > > > > > > > >>>
>> > > > > > > > > >>>> 60)
>> > > > > > > > > >>>>
>> > > > > > > > > >>>> When we are checking if the transaction is ongoing,
>> we
>> > > need
>> > > > to
>> > > > > > > make
>> > > > > > > > a
>> > > > > > > > > >>> round
>> > > > > > > > > >>>> trip from the leader partition to the transaction
>> > > > coordinator.
>> > > > > > In
>> > > > > > > > the
>> > > > > > > > > >>> time
>> > > > > > > > > >>>> we are waiting for this message to come back, in
>> theory
>> > we
>> > > > > could
>> > > > > > > > have
>> > > > > > > > > >>> sent
>> > > > > > > > > >>>> a commit/abort call that would make the original
>> result
>> > of
>> > > > the
>> > > > > > > check
>> > > > > > > > > >>> out of
>> > > > > > > > > >>>> date. That is why we can check the leader state
>> before
>> > we
>> > > > > write
>> > > > > > to
>> > > > > > > > the
>> > > > > > > > > >>> log.
>> > > > > > > > > >>>
>> > > > > > > > > >>> Thanks. Got it.
>> > > > > > > > > >>>
>> > > > > > > > > >>> However, is this really an issue? We put the produce
>> > > request
>> > > > in
>> > > > > > > > > >>> purgatory, so how could we process the
>> > > > `WriteTxnMarkerRequest`
>> > > > > > > first?
>> > > > > > > > > >>> Don't we need to put the `WriteTxnMarkerRequest` into
>> > > > > purgatory,
>> > > > > > > too,
>> > > > > > > > > >>> for this case, and process both request in-order?
>> (Again,
>> > > my
>> > > > > > broker
>> > > > > > > > > >>> knowledge is limited and maybe we don't maintain
>> request
>> > > > order
>> > > > > > for
>> > > > > > > > this
>> > > > > > > > > >>> case, what seems to be an issue IMHO, and I am
>> wondering
>> > if
>> > > > > > > changing
>> > > > > > > > > >>> request handling to preserve order for this case
>> might be
>> > > the
>> > > > > > > cleaner
>> > > > > > > > > >>> solution?)
>> > > > > > > > > >>>
>> > > > > > > > > >>>
>> > > > > > > > > >>>
>> > > > > > > > > >>> -Matthias
>> > > > > > > > > >>>
>> > > > > > > > > >>>
>> > > > > > > > > >>>
>> > > > > > > > > >>>
>> > > > > > > > > >>> On 11/30/22 3:28 PM, Artem Livshits wrote:
>> > > > > > > > > >>>> Hi Justine,
>> > > > > > > > > >>>>
>> > > > > > > > > >>>> I think the interesting part is not in this logic
>> > (because
>> > > > it
>> > > > > > > tries
>> > > > > > > > to
>> > > > > > > > > >>>> figure out when UNKNOWN_PRODUCER_ID is retriable and
>> if
>> > > it's
>> > > > > > > > > retryable,
>> > > > > > > > > >>>> it's definitely not fatal), but what happens when
>> this
>> > > logic
>> > > > > > > doesn't
>> > > > > > > > > >>> return
>> > > > > > > > > >>>> 'true' and falls through.  In the old clients it
>> seems
>> > to
>> > > be
>> > > > > > > fatal,
>> > > > > > > > if
>> > > > > > > > > >>> we
>> > > > > > > > > >>>> keep the behavior in the new clients, I'd expect it
>> > would
>> > > be
>> > > > > > fatal
>> > > > > > > > as
>> > > > > > > > > >>> well.
>> > > > > > > > > >>>>
>> > > > > > > > > >>>> -Artem
>> > > > > > > > > >>>>
>> > > > > > > > > >>>> On Tue, Nov 29, 2022 at 11:57 AM Justine Olshan
>> > > > > > > > > >>>> <jo...@confluent.io.invalid> wrote:
>> > > > > > > > > >>>>
>> > > > > > > > > >>>>> Hi Artem and Jeff,
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>> Thanks for taking a look and sorry for the slow
>> > response.
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>> You both mentioned the change to handle
>> > > UNKNOWN_PRODUCER_ID
>> > > > > > > errors.
>> > > > > > > > > To
>> > > > > > > > > >>> be
>> > > > > > > > > >>>>> clear — this error code will only be sent again when
>> > the
>> > > > > > client's
>> > > > > > > > > >>> request
>> > > > > > > > > >>>>> version is high enough to ensure we handle it
>> > correctly.
>> > > > > > > > > >>>>> The current (Java) client handles this by the
>> following
>> > > > > > (somewhat
>> > > > > > > > > long)
>> > > > > > > > > >>>>> code snippet:
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>> // An UNKNOWN_PRODUCER_ID means that we have lost
>> the
>> > > > > producer
>> > > > > > > > state
>> > > > > > > > > >>> on the
>> > > > > > > > > >>>>> broker. Depending on the log start
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>> // offset, we may want to retry these, as described
>> for
>> > > > each
>> > > > > > case
>> > > > > > > > > >>> below. If
>> > > > > > > > > >>>>> none of those apply, then for the
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>> // idempotent producer, we will locally bump the
>> epoch
>> > > and
>> > > > > > reset
>> > > > > > > > the
>> > > > > > > > > >>>>> sequence numbers of in-flight batches from
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>> // sequence 0, then retry the failed batch, which
>> > should
>> > > > now
>> > > > > > > > succeed.
>> > > > > > > > > >>> For
>> > > > > > > > > >>>>> the transactional producer, allow the
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>> // batch to fail. When processing the failed batch,
>> we
>> > > will
>> > > > > > > > > transition
>> > > > > > > > > >>> to
>> > > > > > > > > >>>>> an abortable error and set a flag
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>> // indicating that we need to bump the epoch (if
>> > > supported
>> > > > by
>> > > > > > the
>> > > > > > > > > >>> broker).
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>> if (error == Errors.*UNKNOWN_PRODUCER_ID*) {
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>       if (response.logStartOffset == -1) {
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           // We don't know the log start offset with
>> > this
>> > > > > > > response.
>> > > > > > > > > We
>> > > > > > > > > >>> should
>> > > > > > > > > >>>>> just retry the request until we get it.
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           // The UNKNOWN_PRODUCER_ID error code was
>> > added
>> > > > > along
>> > > > > > > > with
>> > > > > > > > > >>> the new
>> > > > > > > > > >>>>> ProduceResponse which includes the
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           // logStartOffset. So the '-1' sentinel is
>> > not
>> > > > for
>> > > > > > > > backward
>> > > > > > > > > >>>>> compatibility. Instead, it is possible for
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           // a broker to not know the
>> logStartOffset at
>> > > > when
>> > > > > it
>> > > > > > > is
>> > > > > > > > > >>> returning
>> > > > > > > > > >>>>> the response because the partition
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           // may have moved away from the broker
>> from
>> > the
>> > > > > time
>> > > > > > > the
>> > > > > > > > > >>> error was
>> > > > > > > > > >>>>> initially raised to the time the
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           // response was being constructed. In
>> these
>> > > > cases,
>> > > > > we
>> > > > > > > > > should
>> > > > > > > > > >>> just
>> > > > > > > > > >>>>> retry the request: we are guaranteed
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           // to eventually get a logStartOffset once
>> > > things
>> > > > > > > settle
>> > > > > > > > > down.
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           return true;
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>       }
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>       if (batch.sequenceHasBeenReset()) {
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           // When the first inflight batch fails
>> due to
>> > > the
>> > > > > > > > > truncation
>> > > > > > > > > >>> case,
>> > > > > > > > > >>>>> then the sequences of all the other
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           // in flight batches would have been
>> > restarted
>> > > > from
>> > > > > > the
>> > > > > > > > > >>> beginning.
>> > > > > > > > > >>>>> However, when those responses
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           // come back from the broker, they would
>> also
>> > > > come
>> > > > > > with
>> > > > > > > > an
>> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID error. In this case, we should
>> not
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           // reset the sequence numbers to the
>> > beginning.
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           return true;
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>       } else if
>> > > > > (lastAckedOffset(batch.topicPartition).orElse(
>> > > > > > > > > >>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
>> > > > response.logStartOffset) {
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           // The head of the log has been removed,
>> > > probably
>> > > > > due
>> > > > > > > to
>> > > > > > > > > the
>> > > > > > > > > >>>>> retention time elapsing. In this case,
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           // we expect to lose the producer state.
>> For
>> > > the
>> > > > > > > > > transactional
>> > > > > > > > > >>>>> producer, reset the sequences of all
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           // inflight batches to be from the
>> beginning
>> > > and
>> > > > > > retry
>> > > > > > > > > them,
>> > > > > > > > > >>> so
>> > > > > > > > > >>>>> that the transaction does not need to
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           // be aborted. For the idempotent
>> producer,
>> > > bump
>> > > > > the
>> > > > > > > > epoch
>> > > > > > > > > to
>> > > > > > > > > >>> avoid
>> > > > > > > > > >>>>> reusing (sequence, epoch) pairs
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           if (isTransactional()) {
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>
>> > > > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
>> > > > > > > > > >>>>> this.producerIdAndEpoch);
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           } else {
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>
>> > > > > >  requestEpochBumpForPartition(batch.topicPartition);
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           }
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           return true;
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>       }
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>       if (!isTransactional()) {
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           // For the idempotent producer, always
>> retry
>> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
>> > > > > > > > > >>>>> errors. If the batch has the current
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           // producer ID and epoch, request a bump
>> of
>> > the
>> > > > > > epoch.
>> > > > > > > > > >>> Otherwise
>> > > > > > > > > >>>>> just retry the produce.
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>
>> > > >  requestEpochBumpForPartition(batch.topicPartition);
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>           return true;
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>       }
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>> }
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>> I was considering keeping this behavior — but am
>> open
>> > to
>> > > > > > > > simplifying
>> > > > > > > > > >>> it.
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>> We are leaving changes to older clients off the
>> table
>> > > here
>> > > > > > since
>> > > > > > > it
>> > > > > > > > > >>> caused
>> > > > > > > > > >>>>> many issues for clients in the past. Previously this
>> > was
>> > > a
>> > > > > > fatal
>> > > > > > > > > error
>> > > > > > > > > >>> and
>> > > > > > > > > >>>>> we didn't have the mechanisms in place to detect
>> when
>> > > this
>> > > > > was
>> > > > > > a
>> > > > > > > > > >>> legitimate
>> > > > > > > > > >>>>> case vs some bug or gap in the protocol. Ensuring
>> each
>> > > > > > > transaction
>> > > > > > > > > has
>> > > > > > > > > >>> its
>> > > > > > > > > >>>>> own epoch should close this gap.
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>> And to address Jeff's second point:
>> > > > > > > > > >>>>> *does the typical produce request path append
>> records
>> > to
>> > > > > local
>> > > > > > > log
>> > > > > > > > > >>> along*
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>> *with the currentTxnFirstOffset information? I would
>> > like
>> > > > to
>> > > > > > > > > >>> understand*
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>> *when the field is written to disk.*
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>> Yes, the first produce request populates this field
>> and
>> > > > > writes
>> > > > > > > the
>> > > > > > > > > >>> offset
>> > > > > > > > > >>>>> as part of the record batch and also to the producer
>> > > state
>> > > > > > > > snapshot.
>> > > > > > > > > >>> When
>> > > > > > > > > >>>>> we reload the records on restart and/or
>> reassignment,
>> > we
>> > > > > > > repopulate
>> > > > > > > > > >>> this
>> > > > > > > > > >>>>> field with the snapshot from disk along with the
>> rest
>> > of
>> > > > the
>> > > > > > > > producer
>> > > > > > > > > >>>>> state.
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>> Let me know if there are further comments and/or
>> > > questions.
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>> Thanks,
>> > > > > > > > > >>>>> Justine
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>> On Tue, Nov 22, 2022 at 9:00 PM Jeff Kim
>> > > > > > > > > <jeff.kim@confluent.io.invalid
>> > > > > > > > > >>>>
>> > > > > > > > > >>>>> wrote:
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>>> Hi Justine,
>> > > > > > > > > >>>>>>
>> > > > > > > > > >>>>>> Thanks for the KIP! I have two questions:
>> > > > > > > > > >>>>>>
>> > > > > > > > > >>>>>> 1) For new clients, we can once again return an
>> error
>> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
>> > > > > > > > > >>>>>> for sequences
>> > > > > > > > > >>>>>> that are non-zero when there is no producer state
>> > > present
>> > > > on
>> > > > > > the
>> > > > > > > > > >>> server.
>> > > > > > > > > >>>>>> This will indicate we missed the 0 sequence and we
>> > don't
>> > > > yet
>> > > > > > > want
>> > > > > > > > to
>> > > > > > > > > >>>>> write
>> > > > > > > > > >>>>>> to the log.
>> > > > > > > > > >>>>>>
>> > > > > > > > > >>>>>> I would like to understand the current behavior to
>> > > handle
>> > > > > > older
>> > > > > > > > > >>> clients,
>> > > > > > > > > >>>>>> and if there are any changes we are making. Maybe
>> I'm
>> > > > > missing
>> > > > > > > > > >>> something,
>> > > > > > > > > >>>>>> but we would want to identify whether we missed
>> the 0
>> > > > > sequence
>> > > > > > > for
>> > > > > > > > > >>> older
>> > > > > > > > > >>>>>> clients, no?
>> > > > > > > > > >>>>>>
>> > > > > > > > > >>>>>> 2) Upon returning from the transaction
>> coordinator, we
>> > > can
>> > > > > set
>> > > > > > > the
>> > > > > > > > > >>>>>> transaction
>> > > > > > > > > >>>>>> as ongoing on the leader by populating
>> > > > currentTxnFirstOffset
>> > > > > > > > > >>>>>> through the typical produce request handling.
>> > > > > > > > > >>>>>>
>> > > > > > > > > >>>>>> does the typical produce request path append
>> records
>> > to
>> > > > > local
>> > > > > > > log
>> > > > > > > > > >>> along
>> > > > > > > > > >>>>>> with the currentTxnFirstOffset information? I would
>> > like
>> > > > to
>> > > > > > > > > understand
>> > > > > > > > > >>>>>> when the field is written to disk.
>> > > > > > > > > >>>>>>
>> > > > > > > > > >>>>>> Thanks,
>> > > > > > > > > >>>>>> Jeff
>> > > > > > > > > >>>>>>
>> > > > > > > > > >>>>>>
>> > > > > > > > > >>>>>> On Tue, Nov 22, 2022 at 4:44 PM Artem Livshits
>> > > > > > > > > >>>>>> <al...@confluent.io.invalid> wrote:
>> > > > > > > > > >>>>>>
>> > > > > > > > > >>>>>>> Hi Justine,
>> > > > > > > > > >>>>>>>
>> > > > > > > > > >>>>>>> Thank you for the KIP.  I have one question.
>> > > > > > > > > >>>>>>>
>> > > > > > > > > >>>>>>> 5) For new clients, we can once again return an
>> error
>> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
>> > > > > > > > > >>>>>>>
>> > > > > > > > > >>>>>>> I believe we had problems in the past with
>> returning
>> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
>> > > > > > > > > >>>>>>> because it was considered fatal and required
>> client
>> > > > > restart.
>> > > > > > > It
>> > > > > > > > > >>> would
>> > > > > > > > > >>>>> be
>> > > > > > > > > >>>>>>> good to spell out the new client behavior when it
>> > > > receives
>> > > > > > the
>> > > > > > > > > error.
>> > > > > > > > > >>>>>>>
>> > > > > > > > > >>>>>>> -Artem
>> > > > > > > > > >>>>>>>
>> > > > > > > > > >>>>>>> On Tue, Nov 22, 2022 at 10:00 AM Justine Olshan
>> > > > > > > > > >>>>>>> <jo...@confluent.io.invalid> wrote:
>> > > > > > > > > >>>>>>>
>> > > > > > > > > >>>>>>>> Thanks for taking a look Matthias. I've tried to
>> > > answer
>> > > > > your
>> > > > > > > > > >>>>> questions
>> > > > > > > > > >>>>>>>> below:
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>> 10)
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>> Right — so the hanging transaction only occurs
>> when
>> > we
>> > > > > have
>> > > > > > a
>> > > > > > > > late
>> > > > > > > > > >>>>>>> message
>> > > > > > > > > >>>>>>>> come in and the partition is never added to a
>> > > > transaction
>> > > > > > > again.
>> > > > > > > > > If
>> > > > > > > > > >>>>> we
>> > > > > > > > > >>>>>>>> never add the partition to a transaction, we will
>> > > never
>> > > > > > write
>> > > > > > > a
>> > > > > > > > > >>>>> marker
>> > > > > > > > > >>>>>>> and
>> > > > > > > > > >>>>>>>> never advance the LSO.
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>> If we do end up adding the partition to the
>> > > transaction
>> > > > (I
>> > > > > > > > suppose
>> > > > > > > > > >>>>> this
>> > > > > > > > > >>>>>>> can
>> > > > > > > > > >>>>>>>> happen before or after the late message comes in)
>> > then
>> > > > we
>> > > > > > will
>> > > > > > > > > >>>>> include
>> > > > > > > > > >>>>>>> the
>> > > > > > > > > >>>>>>>> late message in the next (incorrect) transaction.
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>> So perhaps it is clearer to make the distinction
>> > > between
>> > > > > > > > messages
>> > > > > > > > > >>>>> that
>> > > > > > > > > >>>>>>>> eventually get added to the transaction (but the
>> > wrong
>> > > > > one)
>> > > > > > or
>> > > > > > > > > >>>>> messages
>> > > > > > > > > >>>>>>>> that never get added and become hanging.
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>> 20)
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>> The client side change for 2 is removing the
>> > > > addPartitions
>> > > > > > to
>> > > > > > > > > >>>>>> transaction
>> > > > > > > > > >>>>>>>> call. We don't need to make this from the
>> producer
>> > to
>> > > > the
>> > > > > > txn
>> > > > > > > > > >>>>>>> coordinator,
>> > > > > > > > > >>>>>>>> only server side.
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>> In my opinion, the issue with the
>> addPartitionsToTxn
>> > > > call
>> > > > > > for
>> > > > > > > > > older
>> > > > > > > > > >>>>>>> clients
>> > > > > > > > > >>>>>>>> is that we don't have the epoch bump, so we don't
>> > know
>> > > > if
>> > > > > > the
>> > > > > > > > > >>> message
>> > > > > > > > > >>>>>>>> belongs to the previous transaction or this one.
>> We
>> > > need
>> > > > > to
>> > > > > > > > check
>> > > > > > > > > if
>> > > > > > > > > >>>>>> the
>> > > > > > > > > >>>>>>>> partition has been added to this transaction. Of
>> > > course,
>> > > > > > this
>> > > > > > > > > means
>> > > > > > > > > >>>>> we
>> > > > > > > > > >>>>>>>> won't completely cover the case where we have a
>> > really
>> > > > > late
>> > > > > > > > > message
>> > > > > > > > > >>>>> and
>> > > > > > > > > >>>>>>> we
>> > > > > > > > > >>>>>>>> have added the partition to the new transaction,
>> but
>> > > > > that's
>> > > > > > > > > >>>>>> unfortunately
>> > > > > > > > > >>>>>>>> something we will need the new clients to cover.
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>> 30)
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>> Transaction is ongoing = partition was added to
>> > > > > transaction
>> > > > > > > via
>> > > > > > > > > >>>>>>>> addPartitionsToTxn. We check this with the
>> > > > > > > DescribeTransactions
>> > > > > > > > > >>> call.
>> > > > > > > > > >>>>>> Let
>> > > > > > > > > >>>>>>>> me know if this wasn't sufficiently explained
>> here:
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>
>> > > > > > > > > >>>>>>
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>> 40)
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>> The idea here is that if any messages somehow
>> come
>> > in
>> > > > > before
>> > > > > > > we
>> > > > > > > > > get
>> > > > > > > > > >>>>> the
>> > > > > > > > > >>>>>>> new
>> > > > > > > > > >>>>>>>> epoch to the producer, they will be fenced.
>> However,
>> > > if
>> > > > we
>> > > > > > > don't
>> > > > > > > > > >>>>> think
>> > > > > > > > > >>>>>>> this
>> > > > > > > > > >>>>>>>> is necessary, it can be discussed
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>> 50)
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>> It should be synchronous because if we have an
>> event
>> > > > (ie,
>> > > > > an
>> > > > > > > > > error)
>> > > > > > > > > >>>>>> that
>> > > > > > > > > >>>>>>>> causes us to need to abort the transaction, we
>> need
>> > to
>> > > > > know
>> > > > > > > > which
>> > > > > > > > > >>>>>>>> partitions to send transaction markers to. We
>> know
>> > the
>> > > > > > > > partitions
>> > > > > > > > > >>>>>> because
>> > > > > > > > > >>>>>>>> we added them to the coordinator via the
>> > > > > addPartitionsToTxn
>> > > > > > > > call.
>> > > > > > > > > >>>>>>>> Previously we have had asynchronous calls in the
>> > past
>> > > > (ie,
>> > > > > > > > writing
>> > > > > > > > > >>>>> the
>> > > > > > > > > >>>>>>>> commit markers when the transaction is completed)
>> > but
>> > > > > often
>> > > > > > > this
>> > > > > > > > > >>> just
>> > > > > > > > > >>>>>>>> causes confusion as we need to wait for some
>> > > operations
>> > > > to
>> > > > > > > > > complete.
>> > > > > > > > > >>>>> In
>> > > > > > > > > >>>>>>> the
>> > > > > > > > > >>>>>>>> writing commit markers case, clients often see
>> > > > > > > > > >>>>> CONCURRENT_TRANSACTIONs
>> > > > > > > > > >>>>>>>> error messages and that can be confusing. For
>> that
>> > > > reason,
>> > > > > > it
>> > > > > > > > may
>> > > > > > > > > be
>> > > > > > > > > >>>>>>>> simpler to just have synchronous calls —
>> especially
>> > if
>> > > > we
>> > > > > > need
>> > > > > > > > to
>> > > > > > > > > >>>>> block
>> > > > > > > > > >>>>>>> on
>> > > > > > > > > >>>>>>>> some operation's completion anyway before we can
>> > start
>> > > > the
>> > > > > > > next
>> > > > > > > > > >>>>>>>> transaction. And yes, I meant coordinator. I will
>> > fix
>> > > > > that.
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>> 60)
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>> When we are checking if the transaction is
>> ongoing,
>> > we
>> > > > > need
>> > > > > > to
>> > > > > > > > > make
>> > > > > > > > > >>> a
>> > > > > > > > > >>>>>>> round
>> > > > > > > > > >>>>>>>> trip from the leader partition to the transaction
>> > > > > > coordinator.
>> > > > > > > > In
>> > > > > > > > > >>> the
>> > > > > > > > > >>>>>>> time
>> > > > > > > > > >>>>>>>> we are waiting for this message to come back, in
>> > > theory
>> > > > we
>> > > > > > > could
>> > > > > > > > > >>> have
>> > > > > > > > > >>>>>>> sent
>> > > > > > > > > >>>>>>>> a commit/abort call that would make the original
>> > > result
>> > > > of
>> > > > > > the
>> > > > > > > > > check
>> > > > > > > > > >>>>>> out
>> > > > > > > > > >>>>>>> of
>> > > > > > > > > >>>>>>>> date. That is why we can check the leader state
>> > before
>> > > > we
>> > > > > > > write
>> > > > > > > > to
>> > > > > > > > > >>>>> the
>> > > > > > > > > >>>>>>> log.
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>> I'm happy to update the KIP if some of these
>> things
>> > > were
>> > > > > not
>> > > > > > > > > clear.
>> > > > > > > > > >>>>>>>> Thanks,
>> > > > > > > > > >>>>>>>> Justine
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>> On Mon, Nov 21, 2022 at 7:11 PM Matthias J. Sax <
>> > > > > > > > mjsax@apache.org
>> > > > > > > > > >
>> > > > > > > > > >>>>>>> wrote:
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>>> Thanks for the KIP.
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>> Couple of clarification questions (I am not a
>> > broker
>> > > > > expert
>> > > > > > > do
>> > > > > > > > > >>>>> maybe
>> > > > > > > > > >>>>>>>>> some question are obvious for others, but not
>> for
>> > me
>> > > > with
>> > > > > > my
>> > > > > > > > lack
>> > > > > > > > > >>>>> of
>> > > > > > > > > >>>>>>>>> broker knowledge).
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>> (10)
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>>> 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.
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>> What happens if the message come in before the
>> next
>> > > > > > > > > >>>>>> addPartitionsToTxn
>> > > > > > > > > >>>>>>>>> request? It seems the broker hosting the data
>> > > > partitions
>> > > > > > > won't
>> > > > > > > > > know
>> > > > > > > > > >>>>>>>>> anything about it and append it to the
>> partition,
>> > > too?
>> > > > > What
>> > > > > > > is
>> > > > > > > > > the
>> > > > > > > > > >>>>>>>>> difference between both cases?
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>> Also, it seems a TX would only hang, if there
>> is no
>> > > > > > following
>> > > > > > > > TX
>> > > > > > > > > >>>>> that
>> > > > > > > > > >>>>>>> is
>> > > > > > > > > >>>>>>>>> either committer or aborted? Thus, for the case
>> > > above,
>> > > > > the
>> > > > > > TX
>> > > > > > > > > might
>> > > > > > > > > >>>>>>>>> actually not hang (of course, we might get an
>> EOS
>> > > > > violation
>> > > > > > > if
>> > > > > > > > > the
>> > > > > > > > > >>>>>>> first
>> > > > > > > > > >>>>>>>>> TX was aborted and the second committed, or the
>> > other
>> > > > way
>> > > > > > > > > around).
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>> (20)
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>>> Of course, 1 and 2 require client-side
>> changes, so
>> > > for
>> > > > > > older
>> > > > > > > > > >>>>>> clients,
>> > > > > > > > > >>>>>>>>> those approaches won’t apply.
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>> For (1) I understand why a client change is
>> > > necessary,
>> > > > > but
>> > > > > > > not
>> > > > > > > > > sure
>> > > > > > > > > >>>>>> why
>> > > > > > > > > >>>>>>>>> we need a client change for (2). Can you
>> elaborate?
>> > > --
>> > > > > > Later
>> > > > > > > > you
>> > > > > > > > > >>>>>>> explain
>> > > > > > > > > >>>>>>>>> that we should send a
>> DescribeTransactionRequest,
>> > > but I
>> > > > > am
>> > > > > > > not
>> > > > > > > > > sure
>> > > > > > > > > >>>>>>> why?
>> > > > > > > > > >>>>>>>>> Can't we not just do an implicit
>> AddPartiitonToTx,
>> > > too?
>> > > > > If
>> > > > > > > the
>> > > > > > > > > old
>> > > > > > > > > >>>>>>>>> producer correctly registered the partition
>> > already,
>> > > > the
>> > > > > > > > > >>>>>> TX-coordinator
>> > > > > > > > > >>>>>>>>> can just ignore it as it's an idempotent
>> operation?
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>> (30)
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>>> To cover older clients, we will ensure a
>> > transaction
>> > > > is
>> > > > > > > > ongoing
>> > > > > > > > > >>>>>>> before
>> > > > > > > > > >>>>>>>>> we write to a transaction
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>> Not sure what you mean by this? Can you
>> elaborate?
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>> (40)
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>>> [the TX-coordinator] will write the prepare
>> commit
>> > > > > message
>> > > > > > > > with
>> > > > > > > > > a
>> > > > > > > > > >>>>>>>> bumped
>> > > > > > > > > >>>>>>>>> epoch and send WriteTxnMarkerRequests with the
>> > bumped
>> > > > > > epoch.
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>> Why do we use the bumped epoch for both? It
>> seems
>> > > more
>> > > > > > > > intuitive
>> > > > > > > > > to
>> > > > > > > > > >>>>>> use
>> > > > > > > > > >>>>>>>>> the current epoch, and only return the bumped
>> epoch
>> > > to
>> > > > > the
>> > > > > > > > > >>>>> producer?
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>> (50) "Implicit AddPartitionToTransaction"
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>> Why does the implicitly sent request need to be
>> > > > > > synchronous?
>> > > > > > > > The
>> > > > > > > > > >>>>> KIP
>> > > > > > > > > >>>>>>>>> also says
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>>> in case we need to abort and need to know which
>> > > > > partitions
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>> What do you mean by this?
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>>> we don’t want to write to it before we store in
>> > the
>> > > > > > > > transaction
>> > > > > > > > > >>>>>>> manager
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>> Do you mean TX-coordinator instead of "manager"?
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>> (60)
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>> For older clients and ensuring that the TX is
>> > > ongoing,
>> > > > > you
>> > > > > > > > > >>>>> describe a
>> > > > > > > > > >>>>>>>>> race condition. I am not sure if I can follow
>> here.
>> > > Can
>> > > > > you
>> > > > > > > > > >>>>>> elaborate?
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>> -Matthias
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>> On 11/18/22 1:21 PM, Justine Olshan wrote:
>> > > > > > > > > >>>>>>>>>> Hey all!
>> > > > > > > > > >>>>>>>>>>
>> > > > > > > > > >>>>>>>>>> I'd like to start a discussion on my proposal
>> to
>> > add
>> > > > > some
>> > > > > > > > > >>>>>> server-side
>> > > > > > > > > >>>>>>>>>> checks on transactions to avoid hanging
>> > > transactions.
>> > > > I
>> > > > > > know
>> > > > > > > > > this
>> > > > > > > > > >>>>>> has
>> > > > > > > > > >>>>>>>>> been
>> > > > > > > > > >>>>>>>>>> an issue for some time, so I really hope this
>> KIP
>> > > will
>> > > > > be
>> > > > > > > > > helpful
>> > > > > > > > > >>>>>> for
>> > > > > > > > > >>>>>>>>> many
>> > > > > > > > > >>>>>>>>>> users of EOS.
>> > > > > > > > > >>>>>>>>>>
>> > > > > > > > > >>>>>>>>>> The KIP includes changes that will be
>> compatible
>> > > with
>> > > > > old
>> > > > > > > > > clients
>> > > > > > > > > >>>>>> and
>> > > > > > > > > >>>>>>>>>> changes to improve performance and correctness
>> on
>> > > new
>> > > > > > > clients.
>> > > > > > > > > >>>>>>>>>>
>> > > > > > > > > >>>>>>>>>> Please take a look and leave any comments you
>> may
>> > > > have!
>> > > > > > > > > >>>>>>>>>>
>> > > > > > > > > >>>>>>>>>> KIP:
>> > > > > > > > > >>>>>>>>>>
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>
>> > > > > > > > > >>>>>>
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
>> > > > > > > > > >>>>>>>>>> JIRA:
>> > > > https://issues.apache.org/jira/browse/KAFKA-14402
>> > > > > > > > > >>>>>>>>>>
>> > > > > > > > > >>>>>>>>>> Thanks!
>> > > > > > > > > >>>>>>>>>> Justine
>> > > > > > > > > >>>>>>>>>>
>> > > > > > > > > >>>>>>>>>
>> > > > > > > > > >>>>>>>>
>> > > > > > > > > >>>>>>>
>> > > > > > > > > >>>>>>
>> > > > > > > > > >>>>>
>> > > > > > > > > >>>>
>> > > > > > > > > >>>
>> > > > > > > > > >>
>> > > > > > > > > >
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
Hey Jason -- thanks for the input -- I was just digging a bit deeper into
the design + implementation of the validation calls here and what you say
makes sense.

I was wondering about compatibility here. When we send requests
between brokers, we want to ensure that the receiving broker understands
the request (specifically the new fields). Typically this is done via
IBP/metadata version.
I'm trying to think if there is a way around it but I'm not sure there is.

As for the improvements -- can you clarify how the multiple transactional
IDs would help here? Were you thinking of a case where we wait/batch
multiple produce requests together? My understanding for now was 1
transactional ID and one validation per 1 produce request.

Finally with respect to the authorizations, I think it makes sense to skip
topic authorizations, but I'm a bit confused by the "leader ID" field.
Wouldn't we just want to flag the request as from a broker (does it matter
which one?).

I think I want to adopt these suggestions, just had a few questions on the
details.

Thanks,
Justine

On Thu, Jan 5, 2023 at 5:05 PM Jason Gustafson <ja...@confluent.io.invalid>
wrote:

> Hi Justine,
>
> Thanks for the proposal.
>
> I was thinking about the implementation a little bit. In the current
> proposal, the behavior depends on whether we have an old or new client. For
> old clients, we send `DescribeTransactions` and verify the result and for
> new clients, we send `AddPartitionsToTxn`. We might be able to simplify the
> implementation if we can use the same request type. For example, what if we
> bump the protocol version for `AddPartitionsToTxn` and add a `validateOnly`
> flag? For older versions, we can set `validateOnly=true` so that the
> request only returns successfully if the partition had already been added.
> For new versions, we can set `validateOnly=false` and the partition will be
> added to the transaction. The other slightly annoying thing that this would
> get around is the need to collect the transaction state for all partitions
> even when we only care about a subset.
>
> Some additional improvements to consider:
>
> - We can give `AddPartitionsToTxn` better batch support for inter-broker
> usage. Currently we only allow one `TransactionalId` to be specified, but
> the broker may get some benefit being able to batch across multiple
> transactions.
> - Another small improvement is skipping topic authorization checks for
> `AddPartitionsToTxn` when the request is from a broker. Perhaps we can add
> a field for the `LeaderId` or something like that and require CLUSTER
> permission when set.
>
> Best,
> Jason
>
>
>
> On Mon, Dec 19, 2022 at 3:56 PM Jun Rao <ju...@confluent.io.invalid> wrote:
>
> > Hi, Justine,
> >
> > Thanks for the explanation. It makes sense to me now.
> >
> > Jun
> >
> > On Mon, Dec 19, 2022 at 1:42 PM Justine Olshan
> > <jo...@confluent.io.invalid>
> > wrote:
> >
> > > Hi Jun,
> > >
> > > My understanding of the mechanism is that when we get to the last
> epoch,
> > we
> > > increment to the fencing/last epoch and if any further requests come in
> > for
> > > this producer ID they are fenced. Then the producer gets a new ID and
> > > restarts with epoch/sequence 0. The fenced epoch sticks around for the
> > > duration of producer.id.expiration.ms and blocks any late messages
> > there.
> > > The new ID will get to take advantage of the improved semantics around
> > > non-zero start sequences. So I think we are covered.
> > >
> > > The only potential issue is overloading the cache, but hopefully the
> > > improvements (lowered producer.id.expiration.ms) will help with that.
> > Let
> > > me know if you still have concerns.
> > >
> > > Thanks,
> > > Justine
> > >
> > > On Mon, Dec 19, 2022 at 10:24 AM Jun Rao <ju...@confluent.io.invalid>
> > wrote:
> > >
> > > > Hi, Justine,
> > > >
> > > > Thanks for the explanation.
> > > >
> > > > 70. The proposed fencing logic doesn't apply when pid changes, is
> that
> > > > right? If so, I am not sure how complete we are addressing this issue
> > if
> > > > the pid changes more frequently.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > >
> > > >
> > > > On Fri, Dec 16, 2022 at 9:16 AM Justine Olshan
> > > > <jo...@confluent.io.invalid>
> > > > wrote:
> > > >
> > > > > Hi Jun,
> > > > >
> > > > > Thanks for replying!
> > > > >
> > > > > 70.We already do the overflow mechanism, so my change would just
> make
> > > it
> > > > > happen more often.
> > > > > I was also not suggesting a new field in the log, but in the
> > response,
> > > > > which would be gated by the client version. Sorry if something
> there
> > is
> > > > > unclear. I think we are starting to diverge.
> > > > > The goal of this KIP is to not change to the marker format at all.
> > > > >
> > > > > 71. Yes, I guess I was going under the assumption that the log
> would
> > > just
> > > > > look at its last epoch and treat it as the current epoch. I suppose
> > we
> > > > can
> > > > > have some special logic that if the last epoch was on a marker we
> > > > actually
> > > > > expect the next epoch or something like that. We just need to
> > > distinguish
> > > > > based on whether we had a commit/abort marker.
> > > > >
> > > > > 72.
> > > > > > if the producer epoch hasn't been bumped on the
> > > > > broker, it seems that the stucked message will fail the sequence
> > > > validation
> > > > > and will be ignored. If the producer epoch has been bumped, we
> ignore
> > > the
> > > > > sequence check and the stuck message could be appended to the log.
> > So,
> > > is
> > > > > the latter case that we want to guard?
> > > > >
> > > > > I'm not sure I follow that "the message will fail the sequence
> > > > validation".
> > > > > In some of these cases, we had an abort marker (due to an error)
> and
> > > then
> > > > > the late message comes in with the correct sequence number. This
> is a
> > > > case
> > > > > covered by the KIP.
> > > > > The latter case is actually not something we've considered here. I
> > > think
> > > > > generally when we bump the epoch, we are accepting that the
> sequence
> > > does
> > > > > not need to be checked anymore. My understanding is also that we
> > don't
> > > > > typically bump epoch mid transaction (based on a quick look at the
> > > code)
> > > > > but let me know if that is the case.
> > > > >
> > > > > Thanks,
> > > > > Justine
> > > > >
> > > > > On Thu, Dec 15, 2022 at 12:23 PM Jun Rao <jun@confluent.io.invalid
> >
> > > > wrote:
> > > > >
> > > > > > Hi, Justine,
> > > > > >
> > > > > > Thanks for the reply.
> > > > > >
> > > > > > 70. Assigning a new pid on int overflow seems a bit hacky. If we
> > > need a
> > > > > txn
> > > > > > level id, it will be better to model this explicitly. Adding a
> new
> > > > field
> > > > > > would require a bit more work since it requires a new txn marker
> > > format
> > > > > in
> > > > > > the log. So, we probably need to guard it with an IBP or metadata
> > > > version
> > > > > > and document the impact on downgrade once the new format is
> written
> > > to
> > > > > the
> > > > > > log.
> > > > > >
> > > > > > 71. Hmm, once the marker is written, the partition will expect
> the
> > > next
> > > > > > append to be on the next epoch. Does that cover the case you
> > > mentioned?
> > > > > >
> > > > > > 72. Also, just to be clear on the stucked message issue described
> > in
> > > > the
> > > > > > motivation. With EoS, we also validate the sequence id for
> > > idempotency.
> > > > > So,
> > > > > > with the current logic, if the producer epoch hasn't been bumped
> on
> > > the
> > > > > > broker, it seems that the stucked message will fail the sequence
> > > > > validation
> > > > > > and will be ignored. If the producer epoch has been bumped, we
> > ignore
> > > > the
> > > > > > sequence check and the stuck message could be appended to the
> log.
> > > So,
> > > > is
> > > > > > the latter case that we want to guard?
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Wed, Dec 14, 2022 at 10:44 AM Justine Olshan
> > > > > > <jo...@confluent.io.invalid> wrote:
> > > > > >
> > > > > > > Matthias — thanks again for taking time to look a this. You
> said:
> > > > > > >
> > > > > > > > My proposal was only focusing to avoid dangling
> > > > > > >
> > > > > > > transactions if records are added without registered partition.
> > --
> > > > > Maybe
> > > > > > >
> > > > > > > you can add a few more details to the KIP about this scenario
> for
> > > > > better
> > > > > > >
> > > > > > > documentation purpose?
> > > > > > >
> > > > > > >
> > > > > > > I'm not sure I understand what you mean here. The motivation
> > > section
> > > > > > > describes two scenarios about how the record can be added
> > without a
> > > > > > > registered partition:
> > > > > > >
> > > > > > >
> > > > > > > > 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.
> > > > > > >
> > > > > > >
> > > > > > > > Another way hanging transactions can occur is that a client
> is
> > > > buggy
> > > > > > and
> > > > > > > may somehow try to write to a partition before it adds the
> > > partition
> > > > to
> > > > > > the
> > > > > > > transaction.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > For the first example of this would it be helpful to say that
> > this
> > > > > > message
> > > > > > > comes in after the abort, but before the partition is added to
> > the
> > > > next
> > > > > > > transaction so it becomes "hanging." Perhaps the next sentence
> > > > > describing
> > > > > > > the message becoming part of the next transaction (a different
> > > case)
> > > > > was
> > > > > > > not properly differentiated.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > Jun — thanks for reading the KIP.
> > > > > > >
> > > > > > > 70. The int typing was a concern. Currently we have a mechanism
> > in
> > > > > place
> > > > > > to
> > > > > > > fence the final epoch when the epoch is about to overflow and
> > > assign
> > > > a
> > > > > > new
> > > > > > > producer ID with epoch 0. Of course, this is a bit tricky when
> it
> > > > comes
> > > > > > to
> > > > > > > the response back to the client.
> > > > > > > Making this a long could be another option, but I wonder are
> > there
> > > > any
> > > > > > > implications on changing this field if the epoch is persisted
> to
> > > > disk?
> > > > > > I'd
> > > > > > > need to check the usages.
> > > > > > >
> > > > > > > 71.This was something Matthias asked about as well. I was
> > > > considering a
> > > > > > > possible edge case where a produce request from a new
> transaction
> > > > > somehow
> > > > > > > gets sent right after the marker is written, but before the
> > > producer
> > > > is
> > > > > > > alerted of the newly bumped epoch. In this case, we may include
> > > this
> > > > > > record
> > > > > > > when we don't want to. I suppose we could try to do something
> > > client
> > > > > side
> > > > > > > to bump the epoch after sending an endTxn as well in this
> > scenario
> > > —
> > > > > but
> > > > > > I
> > > > > > > wonder how it would work when the server is aborting based on a
> > > > > > server-side
> > > > > > > error. I could also be missing something and this scenario is
> > > > actually
> > > > > > not
> > > > > > > possible.
> > > > > > >
> > > > > > > Thanks again to everyone reading and commenting. Let me know
> > about
> > > > any
> > > > > > > further questions or comments.
> > > > > > >
> > > > > > > Justine
> > > > > > >
> > > > > > > On Wed, Dec 14, 2022 at 9:41 AM Jun Rao
> <jun@confluent.io.invalid
> > >
> > > > > > wrote:
> > > > > > >
> > > > > > > > Hi, Justine,
> > > > > > > >
> > > > > > > > Thanks for the KIP. A couple of comments.
> > > > > > > >
> > > > > > > > 70. Currently, the producer epoch is an int. I am not sure if
> > > it's
> > > > > > enough
> > > > > > > > to accommodate all transactions in the lifetime of a
> producer.
> > > > Should
> > > > > > we
> > > > > > > > change that to a long or add a new long field like txnId?
> > > > > > > >
> > > > > > > > 71. "it will write the prepare commit message with a bumped
> > epoch
> > > > and
> > > > > > > send
> > > > > > > > WriteTxnMarkerRequests with the bumped epoch." Hmm, the epoch
> > is
> > > > > > > associated
> > > > > > > > with the current txn right? So, it seems weird to write a
> > commit
> > > > > > message
> > > > > > > > with a bumped epoch. Should we only bump up the epoch in
> > > > > EndTxnResponse
> > > > > > > and
> > > > > > > > rename the field to sth like nextProducerEpoch?
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > On Mon, Dec 12, 2022 at 8:54 PM Matthias J. Sax <
> > > mjsax@apache.org>
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Thanks for the background.
> > > > > > > > >
> > > > > > > > > 20/30: SGTM. My proposal was only focusing to avoid
> dangling
> > > > > > > > > transactions if records are added without registered
> > partition.
> > > > --
> > > > > > > Maybe
> > > > > > > > > you can add a few more details to the KIP about this
> scenario
> > > for
> > > > > > > better
> > > > > > > > > documentation purpose?
> > > > > > > > >
> > > > > > > > > 40: I think you hit a fair point about race conditions or
> > > client
> > > > > bugs
> > > > > > > > > (incorrectly not bumping the epoch). The
> complexity/confusion
> > > for
> > > > > > using
> > > > > > > > > the bumped epoch I see, is mainly for internal debugging,
> ie,
> > > > > > > inspecting
> > > > > > > > > log segment dumps -- it seems harder to reason about the
> > system
> > > > for
> > > > > > us
> > > > > > > > > humans. But if we get better guarantees, it would be worth
> to
> > > use
> > > > > the
> > > > > > > > > bumped epoch.
> > > > > > > > >
> > > > > > > > > 60: as I mentioned already, I don't know the broker
> internals
> > > to
> > > > > > > provide
> > > > > > > > > more input. So if nobody else chimes in, we should just
> move
> > > > > forward
> > > > > > > > > with your proposal.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > -Matthias
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On 12/6/22 4:22 PM, Justine Olshan wrote:
> > > > > > > > > > Hi all,
> > > > > > > > > > After Artem's questions about error behavior, I've
> > > re-evaluated
> > > > > the
> > > > > > > > > > unknown producer ID exception and had some discussions
> > > offline.
> > > > > > > > > >
> > > > > > > > > > I think generally it makes sense to simplify error
> handling
> > > in
> > > > > > cases
> > > > > > > > like
> > > > > > > > > > this and the UNKNOWN_PRODUCER_ID error has a pretty long
> > and
> > > > > > > > complicated
> > > > > > > > > > history. Because of this, I propose adding a new error
> code
> > > > > > > > > ABORTABLE_ERROR
> > > > > > > > > > that when encountered by new clients (gated by the
> produce
> > > > > request
> > > > > > > > > version)
> > > > > > > > > > will simply abort the transaction. This allows the server
> > to
> > > > have
> > > > > > > some
> > > > > > > > > say
> > > > > > > > > > in whether the client aborts and makes handling much
> > simpler.
> > > > In
> > > > > > the
> > > > > > > > > > future, we can also use this error in other situations
> > where
> > > we
> > > > > > want
> > > > > > > to
> > > > > > > > > > abort the transactions. We can even use on other apis.
> > > > > > > > > >
> > > > > > > > > > I've added this to the KIP. Let me know if there are any
> > > > > questions
> > > > > > or
> > > > > > > > > > issues.
> > > > > > > > > >
> > > > > > > > > > Justine
> > > > > > > > > >
> > > > > > > > > > On Fri, Dec 2, 2022 at 10:22 AM Justine Olshan <
> > > > > > jolshan@confluent.io
> > > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > >> Hey Matthias,
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> 20/30 — Maybe I also didn't express myself clearly. For
> > > older
> > > > > > > clients
> > > > > > > > we
> > > > > > > > > >> don't have a way to distinguish between a previous and
> the
> > > > > current
> > > > > > > > > >> transaction since we don't have the epoch bump. This
> means
> > > > that
> > > > > a
> > > > > > > late
> > > > > > > > > >> message from the previous transaction may be added to
> the
> > > new
> > > > > one.
> > > > > > > > With
> > > > > > > > > >> older clients — we can't guarantee this won't happen if
> we
> > > > > already
> > > > > > > > sent
> > > > > > > > > the
> > > > > > > > > >> addPartitionsToTxn call (why we make changes for the
> newer
> > > > > client)
> > > > > > > but
> > > > > > > > > we
> > > > > > > > > >> can at least gate some by ensuring that the partition
> has
> > > been
> > > > > > added
> > > > > > > > to
> > > > > > > > > the
> > > > > > > > > >> transaction. The rationale here is that there are likely
> > > LESS
> > > > > late
> > > > > > > > > arrivals
> > > > > > > > > >> as time goes on, so hopefully most late arrivals will
> come
> > > in
> > > > > > BEFORE
> > > > > > > > the
> > > > > > > > > >> addPartitionsToTxn call. Those that arrive before will
> be
> > > > > properly
> > > > > > > > gated
> > > > > > > > > >> with the describeTransactions approach.
> > > > > > > > > >>
> > > > > > > > > >> If we take the approach you suggested, ANY late arrival
> > > from a
> > > > > > > > previous
> > > > > > > > > >> transaction will be added. And we don't want that. I
> also
> > > > don't
> > > > > > see
> > > > > > > > any
> > > > > > > > > >> benefit in sending addPartitionsToTxn over the
> > describeTxns
> > > > > call.
> > > > > > > They
> > > > > > > > > will
> > > > > > > > > >> both be one extra RPC to the Txn coordinator.
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> To be clear — newer clients will use addPartitionsToTxn
> > > > instead
> > > > > of
> > > > > > > the
> > > > > > > > > >> DescribeTxns.
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> 40)
> > > > > > > > > >> My concern is that if we have some delay in the client
> to
> > > bump
> > > > > the
> > > > > > > > > epoch,
> > > > > > > > > >> it could continue to send epoch 73 and those records
> would
> > > not
> > > > > be
> > > > > > > > > fenced.
> > > > > > > > > >> Perhaps this is not an issue if we don't allow the next
> > > > produce
> > > > > to
> > > > > > > go
> > > > > > > > > >> through before the EndTxn request returns. I'm also
> > thinking
> > > > > about
> > > > > > > > > cases of
> > > > > > > > > >> failure. I will need to think on this a bit.
> > > > > > > > > >>
> > > > > > > > > >> I wasn't sure if it was that confusing. But if we think
> it
> > > is,
> > > > > we
> > > > > > > can
> > > > > > > > > >> investigate other ways.
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> 60)
> > > > > > > > > >>
> > > > > > > > > >> I'm not sure these are the same purgatories since one
> is a
> > > > > produce
> > > > > > > > > >> purgatory (I was planning on using a callback rather
> than
> > > > > > purgatory)
> > > > > > > > and
> > > > > > > > > >> the other is simply a request to append to the log. Not
> > sure
> > > > we
> > > > > > have
> > > > > > > > any
> > > > > > > > > >> structure here for ordering, but my understanding is
> that
> > > the
> > > > > > broker
> > > > > > > > > could
> > > > > > > > > >> handle the write request before it hears back from the
> Txn
> > > > > > > > Coordinator.
> > > > > > > > > >>
> > > > > > > > > >> Let me know if I misunderstood something or something
> was
> > > > > unclear.
> > > > > > > > > >>
> > > > > > > > > >> Justine
> > > > > > > > > >>
> > > > > > > > > >> On Thu, Dec 1, 2022 at 12:15 PM Matthias J. Sax <
> > > > > mjsax@apache.org
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > >>
> > > > > > > > > >>> Thanks for the details Justine!
> > > > > > > > > >>>
> > > > > > > > > >>>> 20)
> > > > > > > > > >>>>
> > > > > > > > > >>>> The client side change for 2 is removing the
> > addPartitions
> > > > to
> > > > > > > > > >>> transaction
> > > > > > > > > >>>> call. We don't need to make this from the producer to
> > the
> > > > txn
> > > > > > > > > >>> coordinator,
> > > > > > > > > >>>> only server side.
> > > > > > > > > >>>
> > > > > > > > > >>> I think I did not express myself clearly. I understand
> > that
> > > > we
> > > > > > can
> > > > > > > > (and
> > > > > > > > > >>> should) change the producer to not send the
> > `addPartitions`
> > > > > > request
> > > > > > > > any
> > > > > > > > > >>> longer. But I don't thinks it's requirement to change
> the
> > > > > broker?
> > > > > > > > > >>>
> > > > > > > > > >>> What I am trying to say is: as a safe-guard and
> > improvement
> > > > for
> > > > > > > older
> > > > > > > > > >>> producers, the partition leader can just send the
> > > > > `addPartitions`
> > > > > > > > > >>> request to the TX-coordinator in any case -- if the old
> > > > > producer
> > > > > > > > > >>> correctly did send the `addPartition` request to the
> > > > > > TX-coordinator
> > > > > > > > > >>> already, the TX-coordinator can just "ignore" is as
> > > > idempotent.
> > > > > > > > > However,
> > > > > > > > > >>> if the old producer has a bug and did forget to sent
> the
> > > > > > > > `addPartition`
> > > > > > > > > >>> request, we would now ensure that the partition is
> indeed
> > > > added
> > > > > > to
> > > > > > > > the
> > > > > > > > > >>> TX and thus fix a potential producer bug (even if we
> > don't
> > > > get
> > > > > > the
> > > > > > > > > >>> fencing via the bump epoch). -- It seems to be a good
> > > > > > improvement?
> > > > > > > Or
> > > > > > > > > is
> > > > > > > > > >>> there a reason to not do this?
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>> 30)
> > > > > > > > > >>>>
> > > > > > > > > >>>> Transaction is ongoing = partition was added to
> > > transaction
> > > > > via
> > > > > > > > > >>>> addPartitionsToTxn. We check this with the
> > > > > DescribeTransactions
> > > > > > > > call.
> > > > > > > > > >>> Let
> > > > > > > > > >>>> me know if this wasn't sufficiently explained here:
> > > > > > > > > >>>
> > > > > > > > > >>> If we do what I propose in (20), we don't really need
> to
> > > make
> > > > > > this
> > > > > > > > > >>> `DescribeTransaction` call, as the partition leader
> adds
> > > the
> > > > > > > > partition
> > > > > > > > > >>> for older clients and we get this check for free.
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>> 40)
> > > > > > > > > >>>>
> > > > > > > > > >>>> The idea here is that if any messages somehow come in
> > > before
> > > > > we
> > > > > > > get
> > > > > > > > > the
> > > > > > > > > >>> new
> > > > > > > > > >>>> epoch to the producer, they will be fenced. However,
> if
> > we
> > > > > don't
> > > > > > > > think
> > > > > > > > > >>> this
> > > > > > > > > >>>> is necessary, it can be discussed
> > > > > > > > > >>>
> > > > > > > > > >>> I agree that we should have epoch fencing. My question
> is
> > > > > > > different:
> > > > > > > > > >>> Assume we are at epoch 73, and we have an ongoing
> > > > transaction,
> > > > > > that
> > > > > > > > is
> > > > > > > > > >>> committed. It seems natural to write the "prepare
> commit"
> > > > > marker
> > > > > > > and
> > > > > > > > > the
> > > > > > > > > >>> `WriteTxMarkerRequest` both with epoch 73, too, as it
> > > belongs
> > > > > to
> > > > > > > the
> > > > > > > > > >>> current transaction. Of course, we now also bump the
> > epoch
> > > > and
> > > > > > > expect
> > > > > > > > > >>> the next requests to have epoch 74, and would reject an
> > > > request
> > > > > > > with
> > > > > > > > > >>> epoch 73, as the corresponding TX for epoch 73 was
> > already
> > > > > > > committed.
> > > > > > > > > >>>
> > > > > > > > > >>> It seems you propose to write the "prepare commit
> marker"
> > > and
> > > > > > > > > >>> `WriteTxMarkerRequest` with epoch 74 though, what would
> > > work,
> > > > > but
> > > > > > > it
> > > > > > > > > >>> seems confusing. Is there a reason why we would use the
> > > > bumped
> > > > > > > epoch
> > > > > > > > 74
> > > > > > > > > >>> instead of the current epoch 73?
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>> 60)
> > > > > > > > > >>>>
> > > > > > > > > >>>> When we are checking if the transaction is ongoing, we
> > > need
> > > > to
> > > > > > > make
> > > > > > > > a
> > > > > > > > > >>> round
> > > > > > > > > >>>> trip from the leader partition to the transaction
> > > > coordinator.
> > > > > > In
> > > > > > > > the
> > > > > > > > > >>> time
> > > > > > > > > >>>> we are waiting for this message to come back, in
> theory
> > we
> > > > > could
> > > > > > > > have
> > > > > > > > > >>> sent
> > > > > > > > > >>>> a commit/abort call that would make the original
> result
> > of
> > > > the
> > > > > > > check
> > > > > > > > > >>> out of
> > > > > > > > > >>>> date. That is why we can check the leader state before
> > we
> > > > > write
> > > > > > to
> > > > > > > > the
> > > > > > > > > >>> log.
> > > > > > > > > >>>
> > > > > > > > > >>> Thanks. Got it.
> > > > > > > > > >>>
> > > > > > > > > >>> However, is this really an issue? We put the produce
> > > request
> > > > in
> > > > > > > > > >>> purgatory, so how could we process the
> > > > `WriteTxnMarkerRequest`
> > > > > > > first?
> > > > > > > > > >>> Don't we need to put the `WriteTxnMarkerRequest` into
> > > > > purgatory,
> > > > > > > too,
> > > > > > > > > >>> for this case, and process both request in-order?
> (Again,
> > > my
> > > > > > broker
> > > > > > > > > >>> knowledge is limited and maybe we don't maintain
> request
> > > > order
> > > > > > for
> > > > > > > > this
> > > > > > > > > >>> case, what seems to be an issue IMHO, and I am
> wondering
> > if
> > > > > > > changing
> > > > > > > > > >>> request handling to preserve order for this case might
> be
> > > the
> > > > > > > cleaner
> > > > > > > > > >>> solution?)
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>> -Matthias
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>> On 11/30/22 3:28 PM, Artem Livshits wrote:
> > > > > > > > > >>>> Hi Justine,
> > > > > > > > > >>>>
> > > > > > > > > >>>> I think the interesting part is not in this logic
> > (because
> > > > it
> > > > > > > tries
> > > > > > > > to
> > > > > > > > > >>>> figure out when UNKNOWN_PRODUCER_ID is retriable and
> if
> > > it's
> > > > > > > > > retryable,
> > > > > > > > > >>>> it's definitely not fatal), but what happens when this
> > > logic
> > > > > > > doesn't
> > > > > > > > > >>> return
> > > > > > > > > >>>> 'true' and falls through.  In the old clients it seems
> > to
> > > be
> > > > > > > fatal,
> > > > > > > > if
> > > > > > > > > >>> we
> > > > > > > > > >>>> keep the behavior in the new clients, I'd expect it
> > would
> > > be
> > > > > > fatal
> > > > > > > > as
> > > > > > > > > >>> well.
> > > > > > > > > >>>>
> > > > > > > > > >>>> -Artem
> > > > > > > > > >>>>
> > > > > > > > > >>>> On Tue, Nov 29, 2022 at 11:57 AM Justine Olshan
> > > > > > > > > >>>> <jo...@confluent.io.invalid> wrote:
> > > > > > > > > >>>>
> > > > > > > > > >>>>> Hi Artem and Jeff,
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> Thanks for taking a look and sorry for the slow
> > response.
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> You both mentioned the change to handle
> > > UNKNOWN_PRODUCER_ID
> > > > > > > errors.
> > > > > > > > > To
> > > > > > > > > >>> be
> > > > > > > > > >>>>> clear — this error code will only be sent again when
> > the
> > > > > > client's
> > > > > > > > > >>> request
> > > > > > > > > >>>>> version is high enough to ensure we handle it
> > correctly.
> > > > > > > > > >>>>> The current (Java) client handles this by the
> following
> > > > > > (somewhat
> > > > > > > > > long)
> > > > > > > > > >>>>> code snippet:
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> // An UNKNOWN_PRODUCER_ID means that we have lost the
> > > > > producer
> > > > > > > > state
> > > > > > > > > >>> on the
> > > > > > > > > >>>>> broker. Depending on the log start
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> // offset, we may want to retry these, as described
> for
> > > > each
> > > > > > case
> > > > > > > > > >>> below. If
> > > > > > > > > >>>>> none of those apply, then for the
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> // idempotent producer, we will locally bump the
> epoch
> > > and
> > > > > > reset
> > > > > > > > the
> > > > > > > > > >>>>> sequence numbers of in-flight batches from
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> // sequence 0, then retry the failed batch, which
> > should
> > > > now
> > > > > > > > succeed.
> > > > > > > > > >>> For
> > > > > > > > > >>>>> the transactional producer, allow the
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> // batch to fail. When processing the failed batch,
> we
> > > will
> > > > > > > > > transition
> > > > > > > > > >>> to
> > > > > > > > > >>>>> an abortable error and set a flag
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> // indicating that we need to bump the epoch (if
> > > supported
> > > > by
> > > > > > the
> > > > > > > > > >>> broker).
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> if (error == Errors.*UNKNOWN_PRODUCER_ID*) {
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>       if (response.logStartOffset == -1) {
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           // We don't know the log start offset with
> > this
> > > > > > > response.
> > > > > > > > > We
> > > > > > > > > >>> should
> > > > > > > > > >>>>> just retry the request until we get it.
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           // The UNKNOWN_PRODUCER_ID error code was
> > added
> > > > > along
> > > > > > > > with
> > > > > > > > > >>> the new
> > > > > > > > > >>>>> ProduceResponse which includes the
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           // logStartOffset. So the '-1' sentinel is
> > not
> > > > for
> > > > > > > > backward
> > > > > > > > > >>>>> compatibility. Instead, it is possible for
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           // a broker to not know the logStartOffset
> at
> > > > when
> > > > > it
> > > > > > > is
> > > > > > > > > >>> returning
> > > > > > > > > >>>>> the response because the partition
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           // may have moved away from the broker from
> > the
> > > > > time
> > > > > > > the
> > > > > > > > > >>> error was
> > > > > > > > > >>>>> initially raised to the time the
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           // response was being constructed. In these
> > > > cases,
> > > > > we
> > > > > > > > > should
> > > > > > > > > >>> just
> > > > > > > > > >>>>> retry the request: we are guaranteed
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           // to eventually get a logStartOffset once
> > > things
> > > > > > > settle
> > > > > > > > > down.
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           return true;
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>       }
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>       if (batch.sequenceHasBeenReset()) {
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           // When the first inflight batch fails due
> to
> > > the
> > > > > > > > > truncation
> > > > > > > > > >>> case,
> > > > > > > > > >>>>> then the sequences of all the other
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           // in flight batches would have been
> > restarted
> > > > from
> > > > > > the
> > > > > > > > > >>> beginning.
> > > > > > > > > >>>>> However, when those responses
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           // come back from the broker, they would
> also
> > > > come
> > > > > > with
> > > > > > > > an
> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID error. In this case, we should
> not
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           // reset the sequence numbers to the
> > beginning.
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           return true;
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>       } else if
> > > > > (lastAckedOffset(batch.topicPartition).orElse(
> > > > > > > > > >>>>> *NO_LAST_ACKED_SEQUENCE_NUMBER*) <
> > > > response.logStartOffset) {
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           // The head of the log has been removed,
> > > probably
> > > > > due
> > > > > > > to
> > > > > > > > > the
> > > > > > > > > >>>>> retention time elapsing. In this case,
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           // we expect to lose the producer state.
> For
> > > the
> > > > > > > > > transactional
> > > > > > > > > >>>>> producer, reset the sequences of all
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           // inflight batches to be from the
> beginning
> > > and
> > > > > > retry
> > > > > > > > > them,
> > > > > > > > > >>> so
> > > > > > > > > >>>>> that the transaction does not need to
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           // be aborted. For the idempotent producer,
> > > bump
> > > > > the
> > > > > > > > epoch
> > > > > > > > > to
> > > > > > > > > >>> avoid
> > > > > > > > > >>>>> reusing (sequence, epoch) pairs
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           if (isTransactional()) {
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>
> > > > txnPartitionMap.startSequencesAtBeginning(batch.topicPartition,
> > > > > > > > > >>>>> this.producerIdAndEpoch);
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           } else {
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > >  requestEpochBumpForPartition(batch.topicPartition);
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           }
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           return true;
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>       }
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>       if (!isTransactional()) {
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           // For the idempotent producer, always
> retry
> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > > > > > > > >>>>> errors. If the batch has the current
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           // producer ID and epoch, request a bump of
> > the
> > > > > > epoch.
> > > > > > > > > >>> Otherwise
> > > > > > > > > >>>>> just retry the produce.
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > >  requestEpochBumpForPartition(batch.topicPartition);
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>           return true;
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>       }
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> }
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> I was considering keeping this behavior — but am open
> > to
> > > > > > > > simplifying
> > > > > > > > > >>> it.
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> We are leaving changes to older clients off the table
> > > here
> > > > > > since
> > > > > > > it
> > > > > > > > > >>> caused
> > > > > > > > > >>>>> many issues for clients in the past. Previously this
> > was
> > > a
> > > > > > fatal
> > > > > > > > > error
> > > > > > > > > >>> and
> > > > > > > > > >>>>> we didn't have the mechanisms in place to detect when
> > > this
> > > > > was
> > > > > > a
> > > > > > > > > >>> legitimate
> > > > > > > > > >>>>> case vs some bug or gap in the protocol. Ensuring
> each
> > > > > > > transaction
> > > > > > > > > has
> > > > > > > > > >>> its
> > > > > > > > > >>>>> own epoch should close this gap.
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> And to address Jeff's second point:
> > > > > > > > > >>>>> *does the typical produce request path append records
> > to
> > > > > local
> > > > > > > log
> > > > > > > > > >>> along*
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> *with the currentTxnFirstOffset information? I would
> > like
> > > > to
> > > > > > > > > >>> understand*
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> *when the field is written to disk.*
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> Yes, the first produce request populates this field
> and
> > > > > writes
> > > > > > > the
> > > > > > > > > >>> offset
> > > > > > > > > >>>>> as part of the record batch and also to the producer
> > > state
> > > > > > > > snapshot.
> > > > > > > > > >>> When
> > > > > > > > > >>>>> we reload the records on restart and/or reassignment,
> > we
> > > > > > > repopulate
> > > > > > > > > >>> this
> > > > > > > > > >>>>> field with the snapshot from disk along with the rest
> > of
> > > > the
> > > > > > > > producer
> > > > > > > > > >>>>> state.
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> Let me know if there are further comments and/or
> > > questions.
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> Thanks,
> > > > > > > > > >>>>> Justine
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> On Tue, Nov 22, 2022 at 9:00 PM Jeff Kim
> > > > > > > > > <jeff.kim@confluent.io.invalid
> > > > > > > > > >>>>
> > > > > > > > > >>>>> wrote:
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>> Hi Justine,
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> Thanks for the KIP! I have two questions:
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> 1) For new clients, we can once again return an
> error
> > > > > > > > > >>> UNKNOWN_PRODUCER_ID
> > > > > > > > > >>>>>> for sequences
> > > > > > > > > >>>>>> that are non-zero when there is no producer state
> > > present
> > > > on
> > > > > > the
> > > > > > > > > >>> server.
> > > > > > > > > >>>>>> This will indicate we missed the 0 sequence and we
> > don't
> > > > yet
> > > > > > > want
> > > > > > > > to
> > > > > > > > > >>>>> write
> > > > > > > > > >>>>>> to the log.
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> I would like to understand the current behavior to
> > > handle
> > > > > > older
> > > > > > > > > >>> clients,
> > > > > > > > > >>>>>> and if there are any changes we are making. Maybe
> I'm
> > > > > missing
> > > > > > > > > >>> something,
> > > > > > > > > >>>>>> but we would want to identify whether we missed the
> 0
> > > > > sequence
> > > > > > > for
> > > > > > > > > >>> older
> > > > > > > > > >>>>>> clients, no?
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> 2) Upon returning from the transaction coordinator,
> we
> > > can
> > > > > set
> > > > > > > the
> > > > > > > > > >>>>>> transaction
> > > > > > > > > >>>>>> as ongoing on the leader by populating
> > > > currentTxnFirstOffset
> > > > > > > > > >>>>>> through the typical produce request handling.
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> does the typical produce request path append records
> > to
> > > > > local
> > > > > > > log
> > > > > > > > > >>> along
> > > > > > > > > >>>>>> with the currentTxnFirstOffset information? I would
> > like
> > > > to
> > > > > > > > > understand
> > > > > > > > > >>>>>> when the field is written to disk.
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> Thanks,
> > > > > > > > > >>>>>> Jeff
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> On Tue, Nov 22, 2022 at 4:44 PM Artem Livshits
> > > > > > > > > >>>>>> <al...@confluent.io.invalid> wrote:
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>> Hi Justine,
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> Thank you for the KIP.  I have one question.
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> 5) For new clients, we can once again return an
> error
> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> I believe we had problems in the past with
> returning
> > > > > > > > > >>>>> UNKNOWN_PRODUCER_ID
> > > > > > > > > >>>>>>> because it was considered fatal and required client
> > > > > restart.
> > > > > > > It
> > > > > > > > > >>> would
> > > > > > > > > >>>>> be
> > > > > > > > > >>>>>>> good to spell out the new client behavior when it
> > > > receives
> > > > > > the
> > > > > > > > > error.
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> -Artem
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> On Tue, Nov 22, 2022 at 10:00 AM Justine Olshan
> > > > > > > > > >>>>>>> <jo...@confluent.io.invalid> wrote:
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>> Thanks for taking a look Matthias. I've tried to
> > > answer
> > > > > your
> > > > > > > > > >>>>> questions
> > > > > > > > > >>>>>>>> below:
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> 10)
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> Right — so the hanging transaction only occurs
> when
> > we
> > > > > have
> > > > > > a
> > > > > > > > late
> > > > > > > > > >>>>>>> message
> > > > > > > > > >>>>>>>> come in and the partition is never added to a
> > > > transaction
> > > > > > > again.
> > > > > > > > > If
> > > > > > > > > >>>>> we
> > > > > > > > > >>>>>>>> never add the partition to a transaction, we will
> > > never
> > > > > > write
> > > > > > > a
> > > > > > > > > >>>>> marker
> > > > > > > > > >>>>>>> and
> > > > > > > > > >>>>>>>> never advance the LSO.
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> If we do end up adding the partition to the
> > > transaction
> > > > (I
> > > > > > > > suppose
> > > > > > > > > >>>>> this
> > > > > > > > > >>>>>>> can
> > > > > > > > > >>>>>>>> happen before or after the late message comes in)
> > then
> > > > we
> > > > > > will
> > > > > > > > > >>>>> include
> > > > > > > > > >>>>>>> the
> > > > > > > > > >>>>>>>> late message in the next (incorrect) transaction.
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> So perhaps it is clearer to make the distinction
> > > between
> > > > > > > > messages
> > > > > > > > > >>>>> that
> > > > > > > > > >>>>>>>> eventually get added to the transaction (but the
> > wrong
> > > > > one)
> > > > > > or
> > > > > > > > > >>>>> messages
> > > > > > > > > >>>>>>>> that never get added and become hanging.
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> 20)
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> The client side change for 2 is removing the
> > > > addPartitions
> > > > > > to
> > > > > > > > > >>>>>> transaction
> > > > > > > > > >>>>>>>> call. We don't need to make this from the producer
> > to
> > > > the
> > > > > > txn
> > > > > > > > > >>>>>>> coordinator,
> > > > > > > > > >>>>>>>> only server side.
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> In my opinion, the issue with the
> addPartitionsToTxn
> > > > call
> > > > > > for
> > > > > > > > > older
> > > > > > > > > >>>>>>> clients
> > > > > > > > > >>>>>>>> is that we don't have the epoch bump, so we don't
> > know
> > > > if
> > > > > > the
> > > > > > > > > >>> message
> > > > > > > > > >>>>>>>> belongs to the previous transaction or this one.
> We
> > > need
> > > > > to
> > > > > > > > check
> > > > > > > > > if
> > > > > > > > > >>>>>> the
> > > > > > > > > >>>>>>>> partition has been added to this transaction. Of
> > > course,
> > > > > > this
> > > > > > > > > means
> > > > > > > > > >>>>> we
> > > > > > > > > >>>>>>>> won't completely cover the case where we have a
> > really
> > > > > late
> > > > > > > > > message
> > > > > > > > > >>>>> and
> > > > > > > > > >>>>>>> we
> > > > > > > > > >>>>>>>> have added the partition to the new transaction,
> but
> > > > > that's
> > > > > > > > > >>>>>> unfortunately
> > > > > > > > > >>>>>>>> something we will need the new clients to cover.
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> 30)
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> Transaction is ongoing = partition was added to
> > > > > transaction
> > > > > > > via
> > > > > > > > > >>>>>>>> addPartitionsToTxn. We check this with the
> > > > > > > DescribeTransactions
> > > > > > > > > >>> call.
> > > > > > > > > >>>>>> Let
> > > > > > > > > >>>>>>>> me know if this wasn't sufficiently explained
> here:
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense#KIP890:TransactionsServerSideDefense-EnsureOngoingTransactionforOlderClients(3)
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> 40)
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> The idea here is that if any messages somehow come
> > in
> > > > > before
> > > > > > > we
> > > > > > > > > get
> > > > > > > > > >>>>> the
> > > > > > > > > >>>>>>> new
> > > > > > > > > >>>>>>>> epoch to the producer, they will be fenced.
> However,
> > > if
> > > > we
> > > > > > > don't
> > > > > > > > > >>>>> think
> > > > > > > > > >>>>>>> this
> > > > > > > > > >>>>>>>> is necessary, it can be discussed
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> 50)
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> It should be synchronous because if we have an
> event
> > > > (ie,
> > > > > an
> > > > > > > > > error)
> > > > > > > > > >>>>>> that
> > > > > > > > > >>>>>>>> causes us to need to abort the transaction, we
> need
> > to
> > > > > know
> > > > > > > > which
> > > > > > > > > >>>>>>>> partitions to send transaction markers to. We know
> > the
> > > > > > > > partitions
> > > > > > > > > >>>>>> because
> > > > > > > > > >>>>>>>> we added them to the coordinator via the
> > > > > addPartitionsToTxn
> > > > > > > > call.
> > > > > > > > > >>>>>>>> Previously we have had asynchronous calls in the
> > past
> > > > (ie,
> > > > > > > > writing
> > > > > > > > > >>>>> the
> > > > > > > > > >>>>>>>> commit markers when the transaction is completed)
> > but
> > > > > often
> > > > > > > this
> > > > > > > > > >>> just
> > > > > > > > > >>>>>>>> causes confusion as we need to wait for some
> > > operations
> > > > to
> > > > > > > > > complete.
> > > > > > > > > >>>>> In
> > > > > > > > > >>>>>>> the
> > > > > > > > > >>>>>>>> writing commit markers case, clients often see
> > > > > > > > > >>>>> CONCURRENT_TRANSACTIONs
> > > > > > > > > >>>>>>>> error messages and that can be confusing. For that
> > > > reason,
> > > > > > it
> > > > > > > > may
> > > > > > > > > be
> > > > > > > > > >>>>>>>> simpler to just have synchronous calls —
> especially
> > if
> > > > we
> > > > > > need
> > > > > > > > to
> > > > > > > > > >>>>> block
> > > > > > > > > >>>>>>> on
> > > > > > > > > >>>>>>>> some operation's completion anyway before we can
> > start
> > > > the
> > > > > > > next
> > > > > > > > > >>>>>>>> transaction. And yes, I meant coordinator. I will
> > fix
> > > > > that.
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> 60)
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> When we are checking if the transaction is
> ongoing,
> > we
> > > > > need
> > > > > > to
> > > > > > > > > make
> > > > > > > > > >>> a
> > > > > > > > > >>>>>>> round
> > > > > > > > > >>>>>>>> trip from the leader partition to the transaction
> > > > > > coordinator.
> > > > > > > > In
> > > > > > > > > >>> the
> > > > > > > > > >>>>>>> time
> > > > > > > > > >>>>>>>> we are waiting for this message to come back, in
> > > theory
> > > > we
> > > > > > > could
> > > > > > > > > >>> have
> > > > > > > > > >>>>>>> sent
> > > > > > > > > >>>>>>>> a commit/abort call that would make the original
> > > result
> > > > of
> > > > > > the
> > > > > > > > > check
> > > > > > > > > >>>>>> out
> > > > > > > > > >>>>>>> of
> > > > > > > > > >>>>>>>> date. That is why we can check the leader state
> > before
> > > > we
> > > > > > > write
> > > > > > > > to
> > > > > > > > > >>>>> the
> > > > > > > > > >>>>>>> log.
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> I'm happy to update the KIP if some of these
> things
> > > were
> > > > > not
> > > > > > > > > clear.
> > > > > > > > > >>>>>>>> Thanks,
> > > > > > > > > >>>>>>>> Justine
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> On Mon, Nov 21, 2022 at 7:11 PM Matthias J. Sax <
> > > > > > > > mjsax@apache.org
> > > > > > > > > >
> > > > > > > > > >>>>>>> wrote:
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>>> Thanks for the KIP.
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>> Couple of clarification questions (I am not a
> > broker
> > > > > expert
> > > > > > > do
> > > > > > > > > >>>>> maybe
> > > > > > > > > >>>>>>>>> some question are obvious for others, but not for
> > me
> > > > with
> > > > > > my
> > > > > > > > lack
> > > > > > > > > >>>>> of
> > > > > > > > > >>>>>>>>> broker knowledge).
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>> (10)
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>>> 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.
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>> What happens if the message come in before the
> next
> > > > > > > > > >>>>>> addPartitionsToTxn
> > > > > > > > > >>>>>>>>> request? It seems the broker hosting the data
> > > > partitions
> > > > > > > won't
> > > > > > > > > know
> > > > > > > > > >>>>>>>>> anything about it and append it to the partition,
> > > too?
> > > > > What
> > > > > > > is
> > > > > > > > > the
> > > > > > > > > >>>>>>>>> difference between both cases?
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>> Also, it seems a TX would only hang, if there is
> no
> > > > > > following
> > > > > > > > TX
> > > > > > > > > >>>>> that
> > > > > > > > > >>>>>>> is
> > > > > > > > > >>>>>>>>> either committer or aborted? Thus, for the case
> > > above,
> > > > > the
> > > > > > TX
> > > > > > > > > might
> > > > > > > > > >>>>>>>>> actually not hang (of course, we might get an EOS
> > > > > violation
> > > > > > > if
> > > > > > > > > the
> > > > > > > > > >>>>>>> first
> > > > > > > > > >>>>>>>>> TX was aborted and the second committed, or the
> > other
> > > > way
> > > > > > > > > around).
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>> (20)
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>>> Of course, 1 and 2 require client-side changes,
> so
> > > for
> > > > > > older
> > > > > > > > > >>>>>> clients,
> > > > > > > > > >>>>>>>>> those approaches won’t apply.
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>> For (1) I understand why a client change is
> > > necessary,
> > > > > but
> > > > > > > not
> > > > > > > > > sure
> > > > > > > > > >>>>>> why
> > > > > > > > > >>>>>>>>> we need a client change for (2). Can you
> elaborate?
> > > --
> > > > > > Later
> > > > > > > > you
> > > > > > > > > >>>>>>> explain
> > > > > > > > > >>>>>>>>> that we should send a DescribeTransactionRequest,
> > > but I
> > > > > am
> > > > > > > not
> > > > > > > > > sure
> > > > > > > > > >>>>>>> why?
> > > > > > > > > >>>>>>>>> Can't we not just do an implicit
> AddPartiitonToTx,
> > > too?
> > > > > If
> > > > > > > the
> > > > > > > > > old
> > > > > > > > > >>>>>>>>> producer correctly registered the partition
> > already,
> > > > the
> > > > > > > > > >>>>>> TX-coordinator
> > > > > > > > > >>>>>>>>> can just ignore it as it's an idempotent
> operation?
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>> (30)
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>>> To cover older clients, we will ensure a
> > transaction
> > > > is
> > > > > > > > ongoing
> > > > > > > > > >>>>>>> before
> > > > > > > > > >>>>>>>>> we write to a transaction
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>> Not sure what you mean by this? Can you
> elaborate?
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>> (40)
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>>> [the TX-coordinator] will write the prepare
> commit
> > > > > message
> > > > > > > > with
> > > > > > > > > a
> > > > > > > > > >>>>>>>> bumped
> > > > > > > > > >>>>>>>>> epoch and send WriteTxnMarkerRequests with the
> > bumped
> > > > > > epoch.
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>> Why do we use the bumped epoch for both? It seems
> > > more
> > > > > > > > intuitive
> > > > > > > > > to
> > > > > > > > > >>>>>> use
> > > > > > > > > >>>>>>>>> the current epoch, and only return the bumped
> epoch
> > > to
> > > > > the
> > > > > > > > > >>>>> producer?
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>> (50) "Implicit AddPartitionToTransaction"
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>> Why does the implicitly sent request need to be
> > > > > > synchronous?
> > > > > > > > The
> > > > > > > > > >>>>> KIP
> > > > > > > > > >>>>>>>>> also says
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>>> in case we need to abort and need to know which
> > > > > partitions
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>> What do you mean by this?
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>>> we don’t want to write to it before we store in
> > the
> > > > > > > > transaction
> > > > > > > > > >>>>>>> manager
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>> Do you mean TX-coordinator instead of "manager"?
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>> (60)
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>> For older clients and ensuring that the TX is
> > > ongoing,
> > > > > you
> > > > > > > > > >>>>> describe a
> > > > > > > > > >>>>>>>>> race condition. I am not sure if I can follow
> here.
> > > Can
> > > > > you
> > > > > > > > > >>>>>> elaborate?
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>> -Matthias
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>> On 11/18/22 1:21 PM, Justine Olshan wrote:
> > > > > > > > > >>>>>>>>>> Hey all!
> > > > > > > > > >>>>>>>>>>
> > > > > > > > > >>>>>>>>>> I'd like to start a discussion on my proposal to
> > add
> > > > > some
> > > > > > > > > >>>>>> server-side
> > > > > > > > > >>>>>>>>>> checks on transactions to avoid hanging
> > > transactions.
> > > > I
> > > > > > know
> > > > > > > > > this
> > > > > > > > > >>>>>> has
> > > > > > > > > >>>>>>>>> been
> > > > > > > > > >>>>>>>>>> an issue for some time, so I really hope this
> KIP
> > > will
> > > > > be
> > > > > > > > > helpful
> > > > > > > > > >>>>>> for
> > > > > > > > > >>>>>>>>> many
> > > > > > > > > >>>>>>>>>> users of EOS.
> > > > > > > > > >>>>>>>>>>
> > > > > > > > > >>>>>>>>>> The KIP includes changes that will be compatible
> > > with
> > > > > old
> > > > > > > > > clients
> > > > > > > > > >>>>>> and
> > > > > > > > > >>>>>>>>>> changes to improve performance and correctness
> on
> > > new
> > > > > > > clients.
> > > > > > > > > >>>>>>>>>>
> > > > > > > > > >>>>>>>>>> Please take a look and leave any comments you
> may
> > > > have!
> > > > > > > > > >>>>>>>>>>
> > > > > > > > > >>>>>>>>>> KIP:
> > > > > > > > > >>>>>>>>>>
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense
> > > > > > > > > >>>>>>>>>> JIRA:
> > > > https://issues.apache.org/jira/browse/KAFKA-14402
> > > > > > > > > >>>>>>>>>>
> > > > > > > > > >>>>>>>>>> Thanks!
> > > > > > > > > >>>>>>>>>> Justine
> > > > > > > > > >>>>>>>>>>
> > > > > > > > > >>>>>>>>>
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>
> > > > > > > > > >>
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>