-----BEGIN PGP SIGNED MESSAGE-----
Hash: SHA512

A fetch-offset request would return the latest "stable" offset, ie,
either non-transactional or transactional+committed.

If there is a pending transaction, the corresponding offset would not
be returned.

Btw: Kafka 2.5 allows you to block a fetch-offset request for this
case: ie, if there is a pending transaction, you can wait until the
transaction is either committed (and the committed offset would be
returned) or aborted (and the "old" offset would be returned).

Check out KIP-447 for more details:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-447%3A+Producer+sc
alability+for+exactly+once+semantics

The broker side changes will be included in 2.5 release.


- -Matthias



On 3/3/20 6:11 AM, Reftel, Magnus wrote:
> If a consumer sends its offset for a topic-partition as part of a
> transaction, and someone sends an OffsetFetch request for that
> consumer group and topic-partition before the transaction is
> committed, is the OffsetFetch response meant to include that
> pending offset, or only the last offset sent outside of a
> non-committed transaction? I find no discussion of it in the Kafka
> protocol guide, and the code in GroupMetadata.scala seems to
> indicate that pending offsets are not included, but I'm observing
> some behavior that might suggest otherwise.
>
> Best Regards Magnus Reftel
>
>
> ________________________________ Denne e-posten og eventuelle
> vedlegg er beregnet utelukkende for den institusjon eller person
> den er rettet til og kan vaere belagt med lovbestemt taushetsplikt.
> Dersom e-posten er feilsendt, vennligst slett den og kontakt
> Skatteetaten. The contents of this email message and any
> attachments are intended solely for the addressee(s) and may
> contain confidential information and may be legally protected from
> disclosure. If you are not the intended recipient of this message,
> please immediately delete the message and alert the Norwegian Tax
> Administration.
>
-----BEGIN PGP SIGNATURE-----

iQIzBAEBCgAdFiEEI8mthP+5zxXZZdDSO4miYXKq/OgFAl5fLikACgkQO4miYXKq
/OjqWBAAxM6S8fmOhD99h4xdg9oV4ce6p2UnPficAuLhiAJT3yG7zrP3K9roz47/
Kqrw5sO0Hu1cGDGDJyjE3ODPRF1IA5Dou5/5H094biElzJIf6170hYkjvLKZUum6
5tjSAFXuotZn6CXDaD2l3/LORqufdq9qYVkfe6S89zTz4cD1v4ULe1+B6zddh8+A
VDanCB1usJo6VyJ2kU3/IGPkDPKYLoSN3+ijBIdGX7rj/d/RaaH6HO5G4fWXhAae
9RJ05BLSuTo1WqglfZs0PHAhMqurzkHlyXNHxa1W+llxh8AJ/eYz3NmwAKHBrW3M
bw/PEPIAcF5xj2xR1p+2FjYNJbeK1qxBwLRw8jbUaX+yoqn7YQmEvjAuOizr4moF
qHFrIpjIi5SCG5iXpSUJRxY0Wlt/RJG1WYqwdCOIlJtSzgL3+aEbaKdQ8EFMckhA
K7jUaF/TQrsfNszOCI9YvtwqYdDI1b85K0l6+5H5Ki69akQwWSR1nI/2M1WQ07oS
YJLENV17qEJBKdK7wBrqRMRKgBYwlQvjDhthrroCgPdQe0jySwpMwIHzKdi2yhVH
hem8Q8u6fjtfTMDLD7S/+sTATEhJsjN97b/t+wUrK2L3BjkXm83BUziftd+6+L2S
ahDwuZJciYt5U0eFkP+4co26U/BkNituTmGRCTCtmlSyahbocus=
=aSd3
-----END PGP SIGNATURE-----

Reply via email to