Hey Guozhang, I have added an example of the producer API usage under new improvements. Let me know if this looks good to you.
Boyang On Wed, Apr 8, 2020 at 1:38 PM Boyang Chen <reluctanthero...@gmail.com> wrote: > That's a good suggestion Jason. Adding a dedicated PRODUCER_FENCED error > should help distinguish exceptions and could safely mark > INVALID_PRODUCER_EPOCH exception as non-fatal in the new code. Updated the > KIP. > > Boyang > > On Wed, Apr 8, 2020 at 12:18 PM Jason Gustafson <ja...@confluent.io> > wrote: > >> Hey Boyang, >> >> Thanks for the KIP. I think the main problem we've identified here is that >> the current errors conflate transaction timeouts with producer fencing. >> The >> first of these ought to be recoverable, but we cannot distinguish it. The >> suggestion to add a new error code makes sense to me, but it leaves this >> bit of awkwardness: >> >> > One extra issue that needs to be addressed is how to handle >> `ProducerFenced` from Produce requests. >> >> In fact, the underlying error code here is INVALID_PRODUCER_EPOCH. It's >> just that the code treats this as equivalent to `ProducerFenced`. One >> thought I had is maybe PRODUCER_FENCED needs to be a separate error code >> as >> well. After all, only the transaction coordinator knows whether a producer >> has been fenced or not. So maybe the handling could be something like the >> following: >> >> 1. Produce requests may return INVALID_PRODUCER_EPOCH. The producer >> recovers by following KIP-360 logic to see whether the epoch can be >> bumped. >> If it cannot because the broker version is too old, we fail. >> 2. Transactional APIs may return either TRANSACTION_TIMEOUT or >> PRODUCER_FENCED. In the first case, we do the same as above. We try to >> recover by bumping the epoch. If the error is PRODUCER_FENCED, it is >> fatal. >> 3. Older brokers may return INVALID_PRODUCER_EPOCH as well from >> transactional APIs. We treat this the same as 1. >> >> What do you think? >> >> Thanks, >> Jason >> >> >> >> >> >> >> >> >> >> >> On Mon, Apr 6, 2020 at 3:41 PM Boyang Chen <reluctanthero...@gmail.com> >> wrote: >> >> > Yep, updated the KIP, thanks! >> > >> > On Mon, Apr 6, 2020 at 3:11 PM Guozhang Wang <wangg...@gmail.com> >> wrote: >> > >> > > Regarding 2), sounds good, I saw UNKNOWN_PRODUCER_ID is properly >> handled >> > > today in produce / add-partitions-to-txn / add-offsets-to-txn / >> end-txn >> > > responses, so that should be well covered. >> > > >> > > Could you reflect this in the wiki page that the broker should be >> > > responsible for using different error codes given client request >> versions >> > > as well? >> > > >> > > >> > > >> > > Guozhang >> > > >> > > On Mon, Apr 6, 2020 at 9:20 AM Boyang Chen < >> reluctanthero...@gmail.com> >> > > wrote: >> > > >> > > > Thanks Guozhang for the review! >> > > > >> > > > On Sun, Apr 5, 2020 at 5:47 PM Guozhang Wang <wangg...@gmail.com> >> > wrote: >> > > > >> > > > > Hello Boyang, >> > > > > >> > > > > Thank you for the proposed KIP. Just some minor comments below: >> > > > > >> > > > > 1. Could you also describe which producer APIs could potentially >> > throw >> > > > the >> > > > > new TransactionTimedOutException, and also how should callers >> handle >> > > them >> > > > > differently (i.e. just to make your description more concrete as >> > > > javadocs). >> > > > > >> > > > > Good point, I will add example java doc changes. >> > > > >> > > > >> > > > > 2. It's straight-forward if client is on newer version while >> broker's >> > > on >> > > > > older version; however If the client is on older version while >> > broker's >> > > > on >> > > > > newer version, today would the internal module of producers treat >> it >> > > as a >> > > > > general fatal error or not? If not, should the broker set a >> different >> > > > error >> > > > > code upon detecting older request versions? >> > > > > >> > > > > That's a good suggestion, my understanding is that the >> prerequisite >> > of >> > > > this change is the new KIP-360 API which is going out with 2.5, >> > > > so we could just return UNKNOWN_PRODUCER_ID instead of >> PRODUCER_FENCED >> > as >> > > > it could be interpreted as abortable error >> > > > in 2.5 producer and retry. Producers older than 2.5 will not be >> > covered. >> > > > WDYT? >> > > > >> > > > > >> > > > > Guozhang >> > > > > >> > > > > On Thu, Apr 2, 2020 at 1:40 PM Boyang Chen < >> > reluctanthero...@gmail.com >> > > > >> > > > > wrote: >> > > > > >> > > > > > Hey there, >> > > > > > >> > > > > > I would like to start discussion for KIP-588: >> > > > > > >> > > > > > >> > > > > >> > > > >> > > >> > >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts >> > > > > > >> > > > > > which aims to improve Producer resilience to transaction timeout >> > due >> > > to >> > > > > > transient system gaps. >> > > > > > >> > > > > > Best, >> > > > > > Boyang >> > > > > > >> > > > > >> > > > > >> > > > > -- >> > > > > -- Guozhang >> > > > > >> > > > >> > > >> > > >> > > -- >> > > -- Guozhang >> > > >> > >> >