Hi Boyang,

Your reply to 3) seems conflicting with your other answers which is a bit
confusing to me. Following your other answers, it seems you suggest
returning UNKNOWN_PRODUCER_ID so that 2.5 clients can trigger retry logic
as well?

To complete my reasoning here as a complete picture:

a) post KIP-360 (2.5+) the partition leader broker does not return
UNKNOWN_PRODUCER_ID any more.
b) upon seeing an old epoch, partition leader cannot tell if it is due to
fencing or timeout; so it could only return INVALID_PRODUCER_EPOCH.

So the basic idea is to let the clients ask the transaction coordinator for
the source of truth:

1) 2.5+ client would handle UNKNOWN_PRODUCER_ID (which could only be
returned from old brokers) by trying to re-initialize with the transaction
coordinator; the coordinator would then tell it whether it is
PRODUCER_FENCED or TXN_TIMEOUT. And for old brokers, it would always return
PRODUCER_FENCED anyways.
2) 2.6+ client would also handle INVALID_PRODUCER_EPOCH with the retry
initializing logic; and similarly the transaction coordinator would
return PRODUCER_FENCED or TXN_TIMEOUT if it is new or always
return PRODUCER_FENCED if it is old.

The question open is, whether

* 3) the new broker should return UNKNOWN_PRODUCER_ID now when it is
*supposed* to return INVALID_PRODUCER_EPOCH and it found the request is
from 2.5 client (note as mentioned in a) right now we do not
return UNKNOWN_PRODUCER_ID from brokers anymore).

If it does, then 2.5 client could still do the retry logic to the
transaction coordinator, i.e. benefit from KIP-360; but the cost is complex
logic on the broker side as well as producer API version bump up.
If it does not, then when INVALID_PRODUCER_EPOCH is returned to the old
client it would treat it as fatal and not ask the txn coordinator; but it
simplifies the broker logic and also do not require producer API version
bump.

Personally I'd suggest we do the latter, knowing that it would not benefit
2.5 client when the partition leader gets an old epoch and does not know
whether it is Fenced or Timed Out.


Guozhang

On Thu, Apr 16, 2020 at 7:59 PM Boyang Chen <reluctanthero...@gmail.com>
wrote:

> Thanks Jason and Guozhang for the thoughts.
>
> On Thu, Apr 16, 2020 at 6:09 PM Guozhang Wang <wangg...@gmail.com> wrote:
>
> > For 2/3 above, originally I was not thinking that we will have a
> different
> > exception for INVALID_PRODUCER_EPOCH and hence was thinking that in order
> > to leverage KIP-360 for it, we'd have to let the broker to
> > return UNKNOWN_PRODUCER_ID. I.e. we'd change the logic of partition
> leader
> > as well to return UNKNOWN_PRODUCER_ID to let the producer try
> > re-initializing the PID on the coordinator, and if it is indeed due to
> > fencing, then coordinator can let the client know of the fatal error and
> > then fail. In that case, then we do need to bump up the producer API
> > version so that the partition leader knows if it is from older or newer
> > clients: if it is older client who do not have KIP-360, we'd return
> > INVALID_PRODUCER_EPOCH still; for newer client, we can
> > return UNKNOWN_PRODUCER_ID to let it seek what's the truth on
> coordinator.
> >
> I know this is a bit of reversed order. Feel free to check out my reply to
> Jason first and go back here :)
> I think the partition leader will have no change of returned code after we
> discussed that only new client should be able to retry.
>
>
> > Now since we are additionally mapping INVALID_PRODUCER_EPOCH to a
> different
> > error code now and letting clients to handle that similar
> > to UNKNOWN_PRODUCER_ID, this can be saved indeed. However, it also means
> > that 2.5.0 clients would not get benefited from this KIP, since they
> still
> > treat INVALID_PRODUCER_EPOCH as fatal. If people this that's okay then we
> > can simplify the partition leader behavior as well as not bumping up
> > producer APIs. I think I'm a bit inclined towards simplicity over
> > benefiting older clients.
> >
>
> 2.5 client should get benefits if we return UNKNOWN_PRODUCER_ID as I have
> described below.
>
>
> > Guozhang
> >
> > On Thu, Apr 16, 2020 at 5:37 PM Jason Gustafson <ja...@confluent.io>
> > wrote:
> >
> > > Hi Boyang,
> > >
> > > A few minor questions below:
> > >
> > > 1. You mention UNKNOWN_PRODUCER_ID in 2.a under Resilience
> Improvements.
> > I
> > > assume that should be INVALID_PRODUCER_EPOCH? I am not sure this case
> > makes
> > > sense for 2.5 clients which would view this error as fatal regardless
> of
> > > whatever the broker does. Not sure there's anything we can do about
> that.
> > > Similarly, if a newer client is talking to a 2.5 broker, it wouldn't be
> > > able to bump the epoch after a timeout because the broker would not
> know
> > to
> > > keep the last epoch. Unfortunately, I think the only improvements that
> > are
> > > possible here are newer clients talking to newer brokers, but I might
> > have
> > > missed something.
> > >
> >
> The reasoning for returning UNKNOWN_PRODUCER_ID is to trigger the retry
> logic on 2.5 client hopefully.
> As stated in the KIP, the benefited parties are *2.5 client and 2.6
> client *only. If
> transaction coordinator returns INVALID_PRODUCER_EPOCH for a transaction
> timeout case, 2.5 client still has to treat as fatal as it only recognizes
> PRODUCER_FENCED. Returning UNKNOWN_PRODUCER_ID for 2.5 client is a
> substitute for TRANSACTION_TIMED_OUT error code, think of 9656
> <https://issues.apache.org/jira/browse/KAFKA-9656> as a similar
> motivation.
>
>
> > > 2. The proposal says that newer clients talking to older brokers should
> > > treat PRODUCER_FENCED as non-fatal. Just to clarify, older brokers will
> > > only return INVALID_PRODUCER_EPOCH because the PRODUCER_FENCED error
> did
> > > not exist. I think the logic should be something like this.
> > >
> > > On the Transaction Coordinator:
> > > - For old request versions, we continue returning
> INVALID_PRODUCER_EPOCH.
> > > Clients will translate this to PRODUCER_FENCED as they do today and
> treat
> > > this as a fatal error.
> > > - For new request versions, we return either PRODUCER_FENCED or
> > > TRANSACTION_TIMED_OUT depending on the case. Clients will treat the
> first
> > > as fatal and will bump the epoch on the latter.
> > >
> > > On the partition leader:
> > > - Nothing changes. Old clients will treat INVALID_PRODUCER_EPOCH as
> > fatal.
> > > New clients will attempt to bump the epoch if the right version of
> > > InitProducerId is supported and fail otherwise.
> > >
> > > Does that seem right?
> > >
> >
> As I stated in the previous answer, the txn coordinator could also choose
> to return UNKNOWN_PRODUCER_ID for 2.5 clients or older
> when there is a txn timeout. The rest looks good to me.
>
>
> > > 3. Why do we need the bump to the Produce API? As far as I can tell, we
> > are
> > > not changing any errors that are used. The INVALID_PRODUCER_EPOCH error
> > is
> > > already possible today and the two new errors cannot be returned from
> > > Produce.
> > >
> >
> Guozhang had some thoughts about it. I think one vague point we haven't
> clarified is what's the expected error code for server and client. The
> relation should look like:
> old client INVALID_PRODUCER_EPOCH -> 47 (ProducerFencedException)
> new client INVALID_PRODUCER_EPOCH -> 47 (InvalidEpochException),
> PRODUCER_FENCED -> 91 (ProducerFencedException)
> old broker INVALID_PRODUCER_EPOCH -> 47 (ProducerFencedException)
> new broker INVALID_PRODUCER_EPOCH -> 47 (InvalidEpochException),
> PRODUCER_FENCED -> 91 (ProducerFencedException)
>
> As we could see here, for a new partition leader, we shall return code 47
> no matter what as stated in the KIP that PRODUCER_FENCED is not a valid
> case reported from partition leader. So new client should always trigger
> the retry logic for partition leader error. I guess the Produce API bump is
> not necessary then.
>
>
> > > 4. The javadoc examples suggest the user should call
> `initTransactions()`
> > > in the case that a timeout is reached. I don't think that's right. For
> > > KIP-360, we bump the epoch internally. I think we'd want to do the same
> > > here so that we continue the current pattern where `initTransactions()`
> > is
> > > used just once.
> > >
> >
> That's correct observation, will address it.
>
>
> > > Thanks,
> > > Jason
> > >
> > > On Thu, Apr 16, 2020 at 2:24 PM Guozhang Wang <wangg...@gmail.com>
> > wrote:
> > >
> > > > +1 (binding), thanks!
> > > >
> > > > On Tue, Apr 14, 2020 at 4:36 PM Boyang Chen <
> > reluctanthero...@gmail.com>
> > > > wrote:
> > > >
> > > > > Hey all,
> > > > >
> > > > > I would like to start the vote for KIP-588:
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-588
> > > > > %3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts
> > > > >
> > > > > Feel free to continue posting on discussion thread if you have
> > > > > any questions, thanks!
> > > > >
> > > > > Best,
> > > > > Boyang
> > > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> >
> >
> > --
> > -- Guozhang
> >
>


-- 
-- Guozhang

Reply via email to