KIP-98 says
> transaction.app.id: A unique and persistent way to identify a
producer. This is used to ensure idempotency and to enable transaction
recovery or rollback across producer sessions. This is optional: you
will lose cross-session guarantees if this is blank.
which might suggest that a producer that does not use the transactional
features, but does set the transaction.app.id, could get cross-session
idempotency. But the design document "Exactly Once Delivery and
Transactional Messaging in Kafka" rules that out:
> For the idempotent producer (i.e., producer that do not use
transactional APIs), currently we do not make any cross-session
guarantees in any case. In the future, we can extend this guarantee by
having the producer to periodically send InitPIDRequest to the
transaction coordinator to keep the AppID from expiring, which preserves
the producer's zombie defence.
Until that point in the future, could my non-transactional producer send
a InitPIDRequest once and then heartbeat via
BeginTxnRequest/EndTxnRequest(ABORT) in intervals less than
transaction.app.id.timeout.ms in order to guarantee cross-session
itempotency? Or is that not guaranteed because "currently we do not make
any cross-session guarantees in any case"? I know this is would be an
ugly hack.
I guess that is also what the recently added "Producer HeartBeat"
feature proposal would address - although it is described to prevent
idle transactional producers from having their AppIds expired.
Related question: If KIP-98 does not make cross-session guarantees for
idempotent producers, is the only improvement over the current
idempotency situation the prevention of duplicate messages in case of a
partition leader migration? Because if a broker fails or the publisher
fails, KIP-98 does not seem to change the risk of dupes for
non-transactional producers.
Btw: Good job! Both in terms of Kafka in general, and KIP-98 in particular
Cheers
Eugen