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) => localLog.highWatermark case None => localLog.logEndOffset } If it is READ_COMMITTED, it would only return the last stable offset. But this requires the broker version to be newer enough to actually know the new format of the request, is your broker on the newer version? Other than that, I cannot think of a reason explaining what you saw. On Wed, Mar 16, 2022 at 5:12 AM Chris Jansen <chris.jan...@permutive.com> wrote: > Hello all, > > I have the need to query end offsets for a particular topic using a > consumer that has been configured with the "READ_COMMITTED" isolation > level. The response I get via the Java client > includes offsets at the end of the log that have not yet been committed and > therefore can't be consumed. > > After digging around in the Java client code, it looks like the isolation > level is being transmitted in the API request to the broker. So my question > is does the broker use this information when crafting a response and, if it > is taken into account, why does it return log end offsets for transactions > that have not yet been committed? > > For my own future reference I was struggling to find any details on the > broker API anywhere, if someone could point me in the right direction, I'd > really appreciate it. > > Thanks, > > > Chris Jansen > -- -- Guozhang