> > Eugen, moving your email to the main thread so that it doesn't get split. >> >> The `transaction.app.id` is a prerequisite for using transactional APIs. >> And only messages wrapped inside transactions will enjoy idempotent >> guarantees across sessions, and that too only when they employ a >> consume-process-produce pattern. >> > > Say I have a producer, producing messages into a topic and I only want to > guarantee the producer cannot insert duplicates. In other words, there's no > downstream consumer/processor to be worried about - which, when considering > the correctness of the data only, is all I need for idempotent producers, > as every message has a unique id (offset), so downstream processes can take > care of exactly once processing by any number of means. (If you need > transactional all-or-none behavior, which KIP-98 also addresses, that's of > course a more complex story) > > I was under the impression that KIP-98 would fulfill above requirement, > i.e. the prevention of duplicate inserts of the same message into a topic > per producer, without using transactions, and guaranteed across tcp > connections to handle producer/broker crashes and network problems.
The KIP-98 idempotent producer solution only protects against duplicates in the stream when there are broker failures and network problems. For instance, if a producer writes a message, and the leader commits and replicates the message but dies before the acknowledgement is sent to the client. Today, the client will resend the message which will be accepted by the new leader, hence causing duplicates. Also, the offsets of the duplicate messages in this case will be unique, so they can't be de-duped downstream based on the offset. If the client application itself dies, it needs to know which messages were previously sent so that it doesn't resend them when it comes back online. The proposed solution to this situation is to use transactional APIs and the consume-process-produce pattern. If you do so, partially processed previous inputs will be discarded, and processing will resume from the last committed state. > > In other words, producers where the `transaction.app.id` is specified will >> not enjoy idempotence across sessions unless their messages are >> transactional. ie. that the sends are wrapped between `beginTransaction`, >> `sendOffsets`, and `commitTransaction`. >> > > From the KIP-98 wiki and the design document, I understand that AppIDs, > PIDs, and sequence numbers are enforced regardless of their being wrapped > in a transaction or not. Is that not so? > > The PID and sequence numbers are totally transparent to applications. If you enable idempotent production, these will be created and managed by Kafka. AppIds only need to be specified if you use the four new transactional APIs. This is enforced at runtime.