Kafka 0.11 landed with exactly-once semantics and a lot of marketing. We were running trade event pipelines in a regulated environment and the promise was appealing: no duplicate trades in the downstream risk system, no idempotency logic sprinkled through consuming services.
After three months with it in production, the honest summary is: EOS (exactly-once semantics) works as advertised within its scope, and that scope is narrower than it sounds.
What EOS Actually Guarantees
Kafka’s EOS guarantee is: a message produced by a transactional producer is written to the log exactly once, and a consumer with isolation.level=read_committed sees only committed messages.
This is a real and useful guarantee. It means:
- Producer retries on network failure don’t cause duplicates in the log
- Transactions that span multiple partitions are atomic — either all partitions see the write or none do
- Consumers don’t see uncommitted writes from in-flight transactions
What it does not guarantee:
- That your consumer processes each message exactly once (the consumer’s processing is outside Kafka’s scope)
- That downstream side effects (database writes, REST calls, emails) happen exactly once
- Anything about the state of external systems the consumer interacts with
The Consume-Transform-Produce Pattern
EOS is most useful in the “consume-transform-produce” pattern, where a Kafka consumer reads messages, does some transformation, and writes results back to Kafka. With EOS, the read-transform-write can be made atomic: the consumer’s offset commit and the producer’s output writes are part of the same transaction.
| |
If the process crashes between beginTransaction and commitTransaction, the transaction is aborted on recovery. The consumer’s offsets are rolled back (as part of the transaction), so the records are reprocessed. The downstream Kafka topic sees no partial writes.
This is genuinely useful. In our trade enrichment pipeline, it eliminated a class of bugs where process crashes caused some records to be enriched and written downstream but the consumer offset not committed, leading to re-enrichment and duplicate records in the risk system.
The Performance Cost
EOS is not free:
- Idempotent producer: requires PID (producer ID) and sequence numbers per partition. Small overhead per message.
- Transactions:
beginTransaction/commitTransactionare synchronous round-trips to the transaction coordinator. Commit latency was 5–15ms in our cluster depending on load. read_committedconsumers: they must buffer messages until the transaction they belong to is committed or aborted. This adds latency to the consumer and increases memory pressure.- Throughput reduction: in our benchmarks, EOS reduced peak throughput by about 20% vs at-least-once. At our volumes this was acceptable; at millions of messages/second it might not be.
What We Still Had to Build
Even with EOS, our consuming services had to handle:
Consumer reprocessing idempotency: if a consumer crashes after committing the transaction but before acknowledging downstream (e.g., a REST call, a DB write), the message is not redelivered by Kafka — the offset is committed. We had to design downstream writes to be idempotent anyway, using the Kafka message key + offset as a natural idempotency key.
External system failures: if the DB write fails after the Kafka transaction commits, we have a Kafka record with no corresponding DB row. EOS doesn’t help here. We needed a separate reconciliation process.
Exactly-once across systems: achieving exactly-once semantics across Kafka + Postgres requires distributed transactions (XA), which we chose not to use, or the outbox pattern. We used the outbox pattern.
Verdict
EOS in Kafka is worth it for internal Kafka-to-Kafka pipelines. It eliminates a real class of duplicate-write bugs in the consume-transform-produce pattern at acceptable performance cost.
For pipelines that touch external systems, EOS reduces the problem surface but doesn’t eliminate it. You still need idempotent consumers and reconciliation. The value is that you have fewer places to worry about duplicates, and the internal Kafka state is consistent.
In a regulated environment where every duplicate trade in a reporting system requires an investigation, even a partial reduction in the failure surface was worth the 20% throughput trade-off.