
At Zalando, we run hundreds of event streams powered by PostgreSQL logical replication through our Fabric Event Streams platform, a Kubernetes-based approach that allows teams to declare event streams sourcing directly from their Postgres databases. Each stream declaration provisions a micro application that uses Debezium in embedded mode to publish row-level change events as they occur. At peak traffic, these combined connectors process hundreds of thousands of events per second across our 100+ Kubernetes clusters.
This infrastructure has been in operation since late 2018, processing billions of events over the years, but getting here re…

At Zalando, we run hundreds of event streams powered by PostgreSQL logical replication through our Fabric Event Streams platform, a Kubernetes-based approach that allows teams to declare event streams sourcing directly from their Postgres databases. Each stream declaration provisions a micro application that uses Debezium in embedded mode to publish row-level change events as they occur. At peak traffic, these combined connectors process hundreds of thousands of events per second across our 100+ Kubernetes clusters.
This infrastructure has been in operation since late 2018, processing billions of events over the years, but getting here required solving some hard problems with logical replication. This is the story of how we contributed two features to Debezium that we hope will help everyone using logical replication at scale.
The WAL Growth Problem Returns
A couple of years ago, our colleague Declan Murphy wrote about a critical issue with PostgreSQL logical replication where low-activity databases experienced runaway Write-Ahead Log (WAL) growth. The problem was simple: replication slots wouldn’t advance without table activity, causing WAL to pile up until disk space ran out. Our single biggest operational issue when rolling out this event infrastructure at scale was uncontrolled WAL growth on low-activity databases, even with heartbeat configured.
As detailed in Declan’s blog post, we fixed this upstream in the PostgreSQL JDBC driver by having the driver respond to keepalive messages from Postgres, advancing the replication slot when no relevant changes are pending. For a deeper dive into the hazards of logical decoding in PostgreSQL, our colleague Polina Bungina gave an excellent talk at the Posette conference in 2024.
We deployed this fix in production with Debezium 2.7.4, pinning to pgjdbc version 42.7.2, and ran it for nearly two years, processing billions of events with zero detected data loss from this mechanism. The fix completely eliminated our WAL growth issues.
Then came the problem.
Debezium Disables the Fix
As we prepared to upgrade Debezium, we discovered that a recent PR had hard-coded the pgjdbc keepalive flush feature to disabled by setting withAutomaticFlush(false) in the replication stream builder. The Debezium team had good reasons for this change: the feature conflicted with Debezium’s own LSN management logic, users reported issues online, and the safest path forward was to disable it entirely.
For us, this was a blocker because we couldn’t upgrade Debezium without losing the fix that kept our production systems stable. We needed a way forward that would work for both the broader Debezium community and teams like us who had verified this behavior at scale.
First Contribution: Make It Opt-In
We opened DBZ-9641 proposing a simple solution: expose the underlying pgjdbc setting as a connector configuration option, allowing users to opt-in to this proven-safe feature while defaulting to the safer disabled behavior. The Debezium team was receptive and engaged with our use case.
We submitted PR #6881, introducing a new lsn.flush.mode configuration property to replace and deprecate the existing flush.lsn.source boolean with three explicit modes:
manual - LSN flushing is managed externally by your application or another mechanism, with the connector not flushing the LSN at all.
connector (default) - Debezium flushes the LSN after processing each logical replication change event, with the PostgreSQL JDBC driver’s keep-alive thread not flushing LSNs.
connector_and_driver - Both Debezium and the PostgreSQL JDBC driver’s keep-alive thread can flush LSNs, preventing WAL growth on low-activity databases where monitored tables have infrequent changes. When the connector has no pending LSN to flush, the JDBC driver’s keep-alive mechanism can flush the server-reported keep-alive LSN, which reflects all WAL activity including unmonitored activity such as CHECKPOINT, VACUUM, or pg_switch_wal() that doesn’t produce logical replication change events.
To ensure a smooth transition, we implemented full backward compatibility where the deprecated flush.lsn.source boolean automatically maps to the new enum values: true maps to connector and false maps to manual. This gives users time to migrate to the new configuration during the deprecation period without breaking existing deployments.
This unblocked our upgrade path and gave us the opt-in mechanism we needed. But as we thought harder about why the feature caused problems for others, we realized we needed to understand why we were apparently the only ones who liked this feature.
Understanding Why We’re Different
We started digging through GitHub issues from other Debezium users to understand their concerns, and found issues like this one from Airbyte that illustrated the problem clearly. Users were reporting that after upgrading to pgjdbc 42.7.0+, their connectors would fail on restart with "Saved offset is before replication slot’s confirmed lsn," forcing them to perform full re-syncs of their databases or downgrade their pgjdbc version.
That’s when we realized we might be an unusual case at Zalando. Since we launched Fabric Event Streams in late 2018, we’ve always treated the PostgreSQL replication slot as the authoritative source of truth for stream position. Because we didn’t care what offset Debezium tracked internally, we ran with the ephemeral MemoryOffsetBackingStore, meaning our connectors always deferred to the slot position on startup and the keepalive flush advancing the slot ahead of the stored offset was never a problem for us.
But why did we trust our replication slots so completely? The answer lies in our PostgreSQL infrastructure. At Zalando, we’ve been running PostgreSQL at scale using Patroni (our open source solution) for automatic failover since the mid-2010s, and later built the Postgres Operator to manage PostgreSQL on Kubernetes. From day one of our logical replication rollout in late 2018, we implemented replication slot management that ensured slots survived failovers, so we could confidently trust the slot position as durable and correct.
Most other Debezium users, however, were using persistent offset stores like Kafka Connect’s offset topics to track their position, treating the offset as the authoritative source of truth and the replication slot as just a PostgreSQL implementation detail. For them, having the slot advance ahead of the stored offset due to keepalive flushes created an irreconcilable conflict that forced full re-syncs.
The Real Problem: When the Slot and Offset Disagree
The root issue became clear when we examined the conflict between the keepalive flush mechanism and Debezium’s offset management. When using logical replication, position is tracked in two places: Debezium tracks its position in an offset store (Kafka, memory, or another backing store), while Postgres tracks the replication slot position on the database server.
When Debezium starts, it compares these positions. By default, if the offset is behind the slot (offset_lsn < slot_lsn), Debezium attempts to stream from the stored offset without validation, and if the requested LSN is no longer available in the WAL, PostgreSQL returns an error. Users could optionally enable the internal.slot.seek.to.known.offset.on.start=true configuration for a stricter "fail fast" policy that would immediately fail with "Saved offset is before replication slot’s confirmed lsn" to detect slot recreation scenarios.
Neither approach handled the keepalive flush scenario well. The default behavior would fail with a cryptic WAL error when trying to stream from an LSN that had been cleaned up, while the strict validation would immediately fail even though no actual data had been lost.
Here’s why this conflicted with the pgjdbc keepalive flush:
- The JDBC driver advances the slot LSN to skip unmonitored WAL activity like vacuums and checkpoints
- The connector hasn’t flushed its offset yet because it’s waiting for the next change event
- The connector crashes or restarts before flushing its offset
- On restart,
offset_lsn < slot_lsnbecause the slot advanced past the stored offset - The connector refuses to start, even though no actual data has been lost
Users with durable offset stores would hit this constantly when using connector_and_driver mode because the keepalive flush was doing its job of preventing WAL growth, but Debezium’s strict validation made it operationally unsafe. We needed a way to allow users to trust the slot position when they knew it was reliable.
Second Contribution: Trust the Slot
We opened DBZ-9688 proposing a new way to handle offset/slot mismatches by introducing an offset.mismatch.strategy configuration property. Taking inspiration from Kafka’s auto.offset.reset configuration, which allows consumers to opt-in to trusting the broker’s position when their local state is invalid, we proposed allowing Debezium users to opt-in to trusting the PostgreSQL replication slot’s position.
PR #6948 introduced the offset.mismatch.strategy enum to control connector behavior when the stored offset LSN differs from the replication slot’s confirmed flush LSN. This property replaces and deprecates the existing internal.slot.seek.to.known.offset.on.start boolean with four explicit strategies:
no_validation (default) - The connector attempts to stream from the stored offset without validating the replication slot state. If the slot is ahead of the offset and the requested LSN is no longer available in the WAL, PostgreSQL will return an error. This maintains existing default behavior and provides backward compatibility.
trust_offset - The connector validates that the stored offset is not behind the replication slot’s confirmed flush LSN, failing with an error if the offset is behind the slot to indicate potential data loss. If the offset is ahead of or equal to the slot, the connector advances the slot to the offset position using pg_replication_slot_advance() when possible. This strategy replaces the internal.slot.seek.to.known.offset.on.start=true configuration and is useful when you want to detect and be alerted to unexpected slot state changes that could indicate data loss.
trust_slot - The connector treats the PostgreSQL replication slot as the authoritative source of truth. If the stored offset is behind the slot’s confirmed flush LSN, the connector automatically advances the offset to match the slot position, skipping replay of events between the stored offset and the slot position. This is appropriate when using lsn.flush.mode=connector_and_driver, which requires trusting the slot position.
trust_greater_lsn - The connector synchronizes to the maximum LSN between the stored offset and the slot’s confirmed flush LSN, providing bidirectional synchronization. If the offset is behind the slot, the connector advances the offset to the slot position. If the offset is ahead of the slot, the connector advances the slot to the offset position when possible.
Similar to the first contribution, we ensured backward compatibility by automatically mapping the deprecated internal.slot.seek.to.known.offset.on.start boolean to the new strategy enum: false maps to no_validation and true maps to trust_offset, preserving existing behavior while giving users time to adopt the new configuration.
This gives Debezium operators flexibility to match the connector’s behavior to their operational reality. Users configuring lsn.flush.mode=connector_and_driver can pair it with offset.mismatch.strategy=trust_slot for safe, production-ready operation with durable offset stores. It also helps in manual recovery scenarios where an operator needs to advance a slot past corrupted WAL using pg_replication_slot_advance(), allowing them to configure Debezium to respect that change instead of refusing to start.
One of the most important parts of contributing to open source is ensuring that users can actually discover and understand your features. We worked closely with the Debezium team to document both lsn.flush.mode and offset.mismatch.strategy in the PostgreSQL connector documentation, explaining the relationship between the two properties and providing guidance on when to use each mode.
A Note of Gratitude
Both features were merged to Debezium and are available in the nightly builds, with an official release coming soon. This unblocked our upgrade path and will enable safer logical replication for the broader community. Throughout the process, the Debezium engineers were incredibly responsive and helpful, engaging thoughtfully with our use cases, providing detailed feedback on our PRs, and helping us design solutions that worked for everyone.
This experience reminded us why we love working in open source: we had a problem, we proposed solutions, and the maintainers worked with us to make those solutions better. Now everyone benefits from these features, whether they’re running hundreds of connectors like we are at Zalando or just getting started with logical replication.
A big thank you to the Debezium team, not just for building such a critical piece of infrastructure that powers event streaming at Zalando and countless other organizations, but for being so open to contributions and discussions from the community. We’re grateful to give back.
What This Means for You
If you’re using PostgreSQL logical replication with Debezium, these features might help you:
- Experiencing WAL growth on low-activity databases? Configure
lsn.flush.mode=connector_and_driverpaired withoffset.mismatch.strategy=trust_greater_lsn(for persistent offset stores) to prevent WAL accumulation without requiring dummy writes. Thetrust_greater_lsnstrategy provides bidirectional synchronization and self-healing recovery. - Need to manually advance replication slots past corrupted WAL segments? Use
offset.mismatch.strategy=trust_slotortrust_greater_lsnto recover without re-snapshotting your entire database. - Want maximum safety and to detect unexpected slot changes? Use
offset.mismatch.strategy=trust_offsetto validate that your stored offset is never behind the slot, catching potential data loss scenarios early.
At Zalando, these features keep our event streaming infrastructure running smoothly across hundreds of Postgres databases, and we hope they help you build reliable logical replication systems too.
We’re hiring! Do you like working in an ever evolving organization such as Zalando? Consider joining our teams as a Software Engineer!
Related posts
Patching the PostgreSQL JDBC Driver
Contributing to the PostgreSQL JDBC Driver to address the issue of runaway WAL growth in Logical Replication Read more...

Nov 09
2023
From Event-Driven Chaos to a Blazingly Fast Serving API
In this post, we explain how we replaced an event-driven system with a high performance API capable of serving... Read more...


Mar 07
2025
Zalando Postgres Operator: One Year Later
The Postgres operator provides a managed Postgres service for Kubernetes. It extends the Kubernetes API with a... Read more...

Nov 26
2018