The Transactional Outbox Is Not a Queue
The transactional outbox is a ledger, not a queue. Treating it like one is what breaks Postgres under load. This post walks through the specific failure modes — autovacuum stalls, xmin horizon drift, replication slot lag, poison pills — and the operational rules that actually keep it working in production.
Every team I've seen adopt the outbox pattern starts from the same premise: we need atomic "write and publish," so we drop an outbox table next to our domain tables, insert a row in the same transaction as the business write, and let a poller ship those rows to Kafka. Textbook. The diagram fits on a napkin.
Six months later the same team is paging someone at 2 AM because pg_wal is at 87% on the primary, the outbox table is 40 GB of mostly-dead tuples, and autovacuum has been running against it for three hours without finishing. The pattern didn't fail. The mental model did.
The outbox is not a queue. It's a write-heavy, short-lived ledger that lives inside your primary OLTP database. Once you stop treating it like RabbitMQ-in-a-table and start treating it like a piece of Postgres infrastructure with its own operational budget, most of the pain goes away. This post is about what that shift actually means.
The Problem
The canonical outbox implementation looks like this:
@Transactional
fun placeOrder(cmd: PlaceOrder): Order {
val order = orderRepository.save(cmd.toOrder())
outboxRepository.save(
OutboxEvent(
aggregateType = "order",
aggregateId = order.id,
type = "OrderPlaced",
payload = json.encodeToString(OrderPlaced(order)),
)
)
return order
}A separate worker polls, publishes to Kafka, and marks rows as sent — or, in the CDC variant, Debezium tails the WAL and emits each inserted row. Either way, the domain write and the "intent to publish" commit atomically. That part works. That part is not the problem.
The problem is what happens after commit. The outbox table now absorbs the full write rate of every aggregate in the service. At a modest 2k writes/second, that's 172 million rows per day. If the consumer is healthy, you insert a row and delete (or mark) it seconds later. If it isn't — if Kafka is slow, if the poller is deployed, if a poison message wedges the worker — rows pile up. And in Postgres, rows that sit around while being heavily written and deleted are not a benign condition. They're an operational hazard with four distinct shapes.
Autovacuum can't keep up. Every delete or update creates a dead tuple. Autovacuum reclaims them, but it's rate-limited by autovacuum_vacuum_cost_limit and competes with foreground traffic. A hot outbox will generate dead tuples faster than the default settings can clean them, the table bloats, index scans slow down, and the poller's SELECT ... WHERE published_at IS NULL ORDER BY id LIMIT 100 starts taking 400ms instead of 2ms. The poller falls further behind. Dead tuples grow faster. You've built a feedback loop.
The xmin horizon drifts. Postgres can only vacuum tuples older than the oldest running transaction's xmin. One long-running analytics query, one forgotten BEGIN in a psql session, one hung replica with hot_standby_feedback = on — any of these pins xmin and prevents the outbox from being cleaned at all. I've watched a 12-hour reporting query balloon an outbox table from 200 MB to 18 GB. The write path was fine. Vacuum was just forbidden from doing its job.
The replication slot lags. If you're using Debezium or any logical decoding consumer, its replication slot holds WAL until the consumer acknowledges it. Slot falls behind, WAL can't be recycled, pg_wal fills the disk, and the primary stops accepting writes. This failure is particularly nasty because nothing about the outbox table itself is wrong — the table is the tip of the iceberg; the real state lives in the WAL the slot is pinning.
Poison pills stop the line. The poller processes rows in order. One row with malformed JSON, or a Kafka topic that no longer exists, or a payload that triggers a serializer bug, and the worker retries forever. All rows behind it wait. The outbox is now an unbounded queue of the next three hours of business events, and your consumers see nothing.
None of these are bugs in the outbox pattern. They're the cost of running a write-heavy ledger inside a transactional database, and they don't show up in tutorials because tutorials don't run at 2k writes/second for six months.
The Approach
The shift is this: stop reasoning about the outbox as "a queue that happens to be in Postgres." Start reasoning about it as "a bounded, append-mostly table whose job is to transfer rows out of Postgres as fast as they arrive."
That reframing changes what you optimize for. A queue's job is to hold things. The outbox's job is to not hold things. If rows are accumulating, something is already wrong — the correct response is to alert, not to scale the table. Every design decision follows from that:
- The outbox should be small. Rows should live seconds, not minutes. If steady-state depth is more than a few thousand rows, the consumer is the bottleneck and you need to know.
- The outbox should be observable as infrastructure. You monitor depth, oldest row age, consumer lag, and vacuum health — not just "is the publisher running."
- The outbox should fail loudly and selectively. A single poison row must not block the other ninety-nine.
- The outbox should not live forever in the same table. Successfully published rows should leave — ideally via
DELETE, not apublished=trueflag that turns the table into a graveyard.
That last point is the one most teams get wrong, and it's worth its own section.
Technical Deep Dive
Delete, don't mark
The first instinct is to add a published_at TIMESTAMP column and UPDATE it when the row is sent. This is wrong for a hot table. Every update is a dead tuple. You've now doubled your vacuum load — one dead tuple from the update, another when the row is eventually deleted — and your indexes grow with every pass. The correct pattern is:
CREATE UNLOGGED TABLE outbox (
id BIGSERIAL PRIMARY KEY,
aggregate_id UUID NOT NULL,
type TEXT NOT NULL,
payload JSONB NOT NULL,
created_at TIMESTAMPTZ NOT NULL DEFAULT now()
);The consumer reads a batch, publishes it, and DELETEs by primary key. No flag column, no state machine on the row. If you need an audit trail of what was published, that's a different table with different access patterns — write it from the consumer after Kafka acks, not from the transaction.
UNLOGGED is worth a pause. It means the table isn't written to the WAL, which roughly doubles insert throughput and eliminates the outbox from replica streams. The trade-off: the table is truncated on crash recovery. For a pure poller-based outbox, that's survivable — you lose in-flight events that weren't yet published, which is the same failure mode as a consumer that crashed mid-batch. For a Debezium/CDC-based outbox, UNLOGGED is not an option because CDC reads the WAL. Pick your consumer model first, then the table type follows.
SKIP LOCKED for parallel pollers
A single-threaded poller is a single point of throughput. Scale it horizontally with FOR UPDATE SKIP LOCKED:
SELECT id, aggregate_id, type, payload
FROM outbox
ORDER BY id
LIMIT 500
FOR UPDATE SKIP LOCKED;Each worker grabs a disjoint batch without blocking the others. The critical detail is to keep the transaction short — read, publish, delete, commit. Holding the lock while publishing to Kafka means a slow broker holds a row lock which blocks the next poller which extends xmin which blocks vacuum. The entire chain unravels from one slow network call.
If your Kafka producer supports async sends with per-message callbacks, even better: batch the sends, commit the deletes only after all acks return, and size the batch so worst-case publish latency fits inside your vacuum budget.
CDC changes the trade-offs, but not the fundamentals
Debezium with the Outbox Event Router is the cleanest version of this pattern — the consumer never polls, the WAL is the source of truth, and ordering per aggregate is preserved. Debezium 3 in particular improved throughput significantly on busy tables.
But CDC moves the failure mode, it doesn't eliminate it. Instead of a poller falling behind, now a replication slot falls behind, and the consequence is worse: the primary's WAL can't be recycled. max_slot_wal_keep_size (Postgres 13+) is not optional — set it, alert on it, and understand that when it trips, your slot is dead and you'll need to resnapshot. Pick your poison: a poller that falls behind degrades publishing latency; a CDC slot that falls behind degrades the entire primary.
A rough rule: if your outbox rate is under ~5k events/second and you want operational simplicity, poll. If you need strict per-aggregate ordering with Kafka partition keys and you're already running Debezium, use CDC and accept the slot management burden. Don't reach for CDC because it sounds more elegant.
Partition the table if it's hot enough
At high write rates, even a well-vacuumed outbox can suffer from index bloat on id. Range-partitioning by created_at (daily or hourly) lets you DROP PARTITION instead of DELETE for old data, which skips the dead-tuple dance entirely. It's overkill for most services. It's a lifesaver above ~10k inserts/second.
Pitfalls & Edge Cases
A few things that are not obvious until they bite:
The outbox row and the domain row must be in the same transaction. This sounds obvious, but I've seen setups where the domain write goes to one connection and the outbox write to another, both inside a Spring @Transactional method that happened to span two DataSources. It works in staging. It races in production. Use one connection, one transaction, or use XA and understand what you're signing up for.
Don't trust published_at IS NULL indexes. A partial index on "unpublished rows" sounds efficient, but as the table grows and the unpublished set stays small, the planner's statistics lag reality and it occasionally picks a sequential scan. Either delete on publish (so there's no "unpublished" state to index) or use a dedicated partition for in-flight rows.
Poison rows need a sidecar. The simplest reliable design: after N failed publish attempts, move the row to outbox_dead_letter in a separate transaction and delete it from outbox. The primary poller never blocks on a single bad message. A separate process — or a human — drains the DLQ. Skipping this step is how a single malformed event takes down publishing for the whole service.
Kafka's transactional producer is not the same as the outbox. The outbox solves "atomic database write and intent to publish." Kafka transactions solve "atomic write across multiple Kafka partitions." You can combine them, but the outbox is what gives you exactly-once relative to your database, which is almost always the guarantee you actually wanted.
Schema evolution of payloads is a real problem. The outbox row captures a point-in-time serialization; if you change the OrderPlaced schema, rows still in the outbox are on the old schema. Either ensure the consumer can read both versions, or drain the outbox before the deploy. "We'll figure it out in the consumer" is how you end up with unpublishable rows that block the table.
And one operational note: include pg_stat_all_tables.n_dead_tup for the outbox table in your standard monitoring. It's a leading indicator for every failure mode above, and it costs nothing to collect.
Practical Takeaways
- Treat the outbox as infrastructure with an operational budget, not as application state.
- Delete published rows. Don't mark them. Dead tuples are the enemy.
- Use
FOR UPDATE SKIP LOCKEDto scale pollers horizontally, and keep the transaction shorter than the publish call. - Alert on outbox depth and oldest row age, not just "is the publisher up."
- Watch the xmin horizon. Long-running queries elsewhere in the database can silently kill your outbox.
- Set
max_slot_wal_keep_sizeif you use CDC. Plan for the day a replication slot dies. - Give poison messages a dead-letter path. One bad row should never block the rest.
- Consider
UNLOGGEDfor poller-based outboxes. Consider partitioning above ~10k writes/second. - Monitor
n_dead_tupon the outbox table as a first-class signal.
Conclusion
The outbox pattern isn't broken. It's a good pattern. What breaks is the mental model that it's "just a queue." The moment you accept that it's a write-heavy ledger inside your primary database, the operational work it demands stops feeling like overhead and starts feeling like the actual job.
Use the outbox when you need atomic "write and publish" and you're already running Postgres with Kafka downstream. Skip it when your write rate doesn't justify the operational weight, or when an eventually-consistent CDC stream directly off your domain tables gives you what you need without the extra hop. And when you do use it: monitor it like the critical infrastructure it is, not like a side table you wrote once and forgot about.
Written by Tiarê Balbi