Re: Transactions and `endOffsets` Java client consumer method

2022-03-22 Thread Guozhang Wang
Hi Chris, Since you are using read_committed mode, the txn marker from the `endOffsets()` should indeed be skipped, no matter for committed of aborted txns. For example if the log looks like this: offsets: 0, 1, 2, 3, t_c(4) // or t_a(4) which means "abort marker" then the endOffset should ret

Re: Transactions and `endOffsets` Java client consumer method

2022-03-22 Thread Chris Jansen
Thanks Luke, If the transaction marker should be hidden, does it follow that aborted transaction at the end of the log should also be hidden for clients that are in read committed mode? Happy to do a KIP/PR. Thanks again, Chris On Tue, Mar 22, 2022 at 10:21 AM Luke Chen wrote: > Hi Chris, >

Re: Transactions and `endOffsets` Java client consumer method

2022-03-22 Thread Luke Chen
Hi Chris, Yes, the transaction marker should be hidden to clients. There is similar issues reported: https://issues.apache.org/jira/browse/KAFKA-10683 Welcome to submit KIP/PR to improve it. Thank you. Luke On Tue, Mar 22, 2022 at 5:16 PM Chris Jansen wrote: > Hi Guozhang, > > Sorry, I should

Re: Transactions and `endOffsets` Java client consumer method

2022-03-22 Thread Chris Jansen
Hi Guozhang, Sorry, I should have been more clear. By "an unreadable end of the log", I mean the `endOffsets` method returns an offset for a record that is never surfaced to the caller of `poll`. I've done some more digging and I think I understand why that is now. The API `endOffsets` calls tell

Re: Transactions and `endOffsets` Java client consumer method

2022-03-21 Thread Guozhang Wang
Hi Chris, What do you mean by "an unreadable end of the log"? Could you elaborate? The version you used is recent enough, and the configs seems okay. So I think there are some issues elsewhere. On Mon, Mar 21, 2022 at 5:51 AM Chris Jansen wrote: > So an update on this, it seems that the consum

Re: Transactions and `endOffsets` Java client consumer method

2022-03-21 Thread Chris Jansen
So an update on this, it seems that the consumer reports an unreadable end of the log if the transaction has been aborted. Is this the intended behaviour? Thanks again, Chris On Mon, Mar 21, 2022 at 12:25 PM Chris Jansen wrote: > Hi Guozhang, > > Thanks for getting back to me. I'm using Conflu

Re: Transactions and `endOffsets` Java client consumer method

2022-03-21 Thread Chris Jansen
Hi Guozhang, Thanks for getting back to me. I'm using Confluent's distribution version 6.2.2, so Kafka 2.8.1. Could there be some consumer or broker configuration I'm missing that would change this behaviour? Just to confirm, here are my consumer settings: auto.offset.reset -> earliest, isolatio

Re: Transactions and `endOffsets` Java client consumer method

2022-03-19 Thread Guozhang Wang
Hi Chris, The broker does take the isolation level in the ListOffset API (which is used for the endoffset call) in to consideration: val lastFetchableOffset = isolationLevel match { case Some(IsolationLevel.READ_COMMITTED) => localLog.lastStableOffset case Some(IsolationLevel.READ_UNCOMMITTED