Gunnar Morling

Gunnar Morling

Random Musings on All Things Software Engineering

Gunnar Morling

Gunnar Morling

Random Musings on All Things Software Engineering

On Idempotency Keys

Posted at Nov 25, 2025

In distributed systems, there’s a common understanding that it is not possible to guarantee exactly-once delivery of messages. What is possible though is exactly-once processing. By adding a unique idempotency key to each message, you can enable consumers to recognize and ignore duplicate messages, i.e. messages which they have received and successfully processed before.

Now, how does this work exactly? When receiving a message, a consumer takes the message’s idempotency key and compares it to the keys of the messages which it already has processed. If it has seen the key before, the incoming message is a duplicate and can be ignored. Otherwise, the consumer goes on to process the message, for instance by storing the message itself, or a view derived from it, in some kind of database.

In addition, it stores the idempotency key of the message. Critically, these two things must happen atomically, typically by wrapping them in a database transaction. Either the message gets processed and its idempotency key gets persisted. Or, the transaction gets rolled back and no changes are applied at all. That way, it is ensured that the consumer will process a message again upon redelivery, if it failed to do so before. It also is ensured that duplicates received after successfully processing the message are skipped over.

UUIDs

So let’s discuss what makes for a good idempotency key then. One possible option would be to use a UUIDv4. These random identifiers solve the requirement of uniquely identifying each message. However, they require the consumer to store the UUIDs of all the previous messages it ever has received in order to reliably identify a duplicate. Depending on the message volume, this may not be practical. Pragmatically, you might get away with discarding received UUIDs after some time period, if it is acceptable to occasionally receive and process a duplicate after that period. Unfortunately, neither the producer of the message nor the consumer will have any indication of the duplicated processing in that case.

We can somewhat improve this situation by adding a timestamp to the idempotency key, for instance by using a UUIDv7) which contains both a timestamp part (first 48 bits) and a random part (remaining bits), or an ULID. That way, the consumer can detect when it receives a message with an idempotency key which is “too old”. While it can’t decide whether the message is a duplicate or not, it can flag to the producer that it can’t handle that message. It is then upon the producer to decide how to proceed. For instance, if the message is part of a payment flow, the system might suggest to the user to first check in their banking account whether this payment has already been executed or not. Only if that’s not the case, a new message with the same payload and a fresh UUID would be sent.

Monotonically Increasing Sequences

All these intricacies can be avoided when it is possible to use a monotonically increasing sequence value as the idempotency key. In that case, the consumer does not need to store all the keys it ever has processed (or a reasonably sized subset thereof). It only needs to store a single value, the one of the latest message which it has processed. If it receives a message with the same or a lower idempotency key, that message must be a duplicate and can be ignored. When receiving messages from a partitioned source, such as a Kafka topic with multiple partitions, or from multiple independent producers (e.g., different clients of a REST API, each using their own separate sequence), then the latest key value per partition must be stored.

Monotonically increasing idempotency keys are a great improvement from the perspective of the message consumer. On the flipside, they may make things more complicated for producers: creating monotonically increasing sequence values isn’t without its own challenges. It is trivial if producers are single-threaded, producing one message at a time. In that case, a database sequence, or even a simple in-memory counter, can be used for creating the idempotency keys. Gaps in the sequence are fine, hence it is possible to increment the persistent state of the sequence or counter in larger steps, and dispense the actual values from an in-memory copy. That way, disk IO can be reduced. From a consumer perspective, Kafka partition offsets fall into that bucket, as they can be considered a monotonically increasing idempotency key for the messages consumed from a given partition.

Things get more complicated when the producer is subject to multiple concurrent requests at once, for instance a REST service with multiple request workers, perhaps even scaled out to multiple compute nodes in a cluster. To ensure monotonicity, retrieval of the idempotency key and emitting a message with that key must happen atomically, uninterrupted by other worker threads. Otherwise, you may end up in a situation where thread A fetches sequence value 100, thread B fetches sequence value 101, B emits a message with idempotency key 101, and then A emits a message with idempotency key 100\. A consumer would then, incorrectly, discard A’s message as a duplicate.

For most cases, ensuring this level of atomicity will impose a severe bottleneck, essentially serializing all requests of the producer system, regardless of how many worker threads or service instances you deploy. Note that if you really wanted to go down that route, solely using a database sequence for producing the idempotency key will not work. Instead, you’d have to use a mechanism such as Postgres advisory locks in order to guarantee monotonicity of idempotency keys in the outgoing messages.

Deriving Idempotency Keys From the Transaction Log

Now, is there a way for us to have this cake and eat it too? Can we get the space efficiency for consumers when using monotonically increasing idempotency keys, without hampering performance of multi-threaded producers? Turns out we can, at least when the emission of messages can be made an asynchronous activity in the producer system, happening independently from processing inbound requests. This means clients of the producer system receive confirmation that the intent to send a message or request was persisted, but they don’t get the result of the same right away. If a use case can be modeled with these semantics, the problem can be reduced to the single-threaded situation above: instead of emitting messages directly to the target system, each producer thread inserts them into a queue. This queue is processed by a single-threaded worker process which emits all the messages sequentially. As argued in The Synchrony Budget, making activities asynchronous can be generally advantageous, if we don’t require their outcome right away.

One specific way to do so would be a variation of the widely used outbox pattern, utilizing the transaction log of the producer service’s database. After all, it’s not necessary to sequence inbound requests ourselves as the database already is doing that for us when serializing the transactions in its log. When producers persist the intent to send a message in the transaction log—for instance by writing a record into a specific table—a process tailing the log can assign idempotency keys to these messages based on their position in the transaction log.

An implementation of this is straight-forward using tools for log-based Change Data Capture (CDC), such as Debezium: You retrieve the messages to be sent from the log by capturing the INSERT events from the outbox table, and assign an idempotency key before emitting them, derived from their log offset. The exact details are going to depend on the specific database.

For example, in Postgres it is ensured that the log sequence numbers (LSN) of commit events within its write-ahead log (WAL) are monotonically increasing: the commit event of a transaction committing after another transaction will have a higher LSN. Furthermore, it is guaranteed that within a given transaction, the LSNs of the events are also monotonically increasing. This makes the tuple of { Commit LSN, Event LSN } a great fit for an idempotency key. In order to not leak the fact that a producer is using a Postgres database, both values can be encoded into a single 128 bit number value. Note that you don’t need to deploy Kafka or Kafka Connect for this solution. Debezium’s embedded engine is a great fit for this use case, allowing you to assign idempotency keys from within a callback method in the producer service itself, not requiring any further infrastructure. When using Postgres to implement this pattern, you don’t even need a dedicated outbox table, as it lets you write arbitrary contents into the transaction log via pg_logical_emit_message(), which is perfect for the use case at hand.

Discussion

So, when to use which kind of idempotency key then? As always, there are no silver bullets, and the answer depends on your specific use case. For many scenarios, using UUIDs and dropping them after some time will probably be sufficient, provided you can tolerate that messages occasionally can be processed a second time when duplicates arrive after the retention period of processed keys.

The more messages you need to process overall, the more attractive a solution centered around monotonically increasing sequences becomes, as it allows for space-efficient duplicate detection and exclusion, no matter how many messages you have. The proposed log-based approach can be an efficient solution for doing so, but it also adds operational complexity: your database needs to support logical replication, you need to run a CDC connector, etc. However, many organizations already operate CDC pipelines for other purposes (analytics, search indexing, cache invalidation, etc.). If you’re in that category, the incremental complexity is minimal. If you’re not, you should weigh the operational overhead against the benefits (constant-space duplicate detection) for your specific scale.

Gunnar Morling

Gunnar Morling is an open-source software engineer in the Java and data streaming space. He currently works as a Technologist for Confluent. In his past role at Decodable he focused on developer outreach and helped them build their stream processing platform based on Apache Flink. Prior to that, he spent ten years at Red Hat, where he led the Debezium project, a platform for change data capture.