Over the past few years, we’ve often heard the same question from the community, forums, and even customers:
“Since your system is written in Rust, why not just rewrite Debezium in Rust?”
There have indeed been open-source projects attempting this—some try to build a Change Data Capture (CDC) engine from scratch in Rust, while others attempt to parse Postgres WAL or MySQL binlog directly. Three years ago, when we began designing our CDC module, we seriously considered this idea and ran several experiments. But after multiple attempts (and failures), we ultimately decided not to rewrite Debezium in Rust. Instead, we chose to build on top of the Debezium Embedded Engine and perform deep customizations.
Here’s why.
Why Not Rewrite in Rust
First, it’s worth acknowledging that Rust offers undeniable advantages in performance, safety, and concurrency control—we wrote our own core compute and storage engine in Rust for those reasons. But CDC is a highly ecosystem-dependent domain, and Debezium is already the de facto industry standard. Whether it’s Postgres, MySQL, SQL Server, or more obscure databases, Debezium already has mature connectors, well-tested log parsers, and an active community.
If we chose to rewrite it in Rust, it would mean:
Protocol parsing from scratch
Every database’s log format is different. Postgres WAL and MySQL binlog have unique structures, field encodings, and edge cases, requiring extensive adaptation and debugging. Every parsing pitfall Debezium has already solved, we’d have to solve again ourselves.Extremely long compatibility testing cycles
Differences in database versions, configuration, character sets, time zones, DDL behavior—all of these impact parsing accuracy in CDC. The Debezium community submits bug-fix PRs almost daily. If we maintained a separate Rust version, we’d have to shoulder the entire long-term maintenance cost alone.Falling behind on database feature updates
Databases add new capabilities every year—like updates to Postgres logical decoding plugins or changes to MySQL GTID behavior. Debezium’s community actively monitors and adapts to these changes. If we rewrote it, we’d have to dedicate a team just to keep up.
So even if we could theoretically beat Debezium in performance, abandoning it means committing to what could easily become a decade-long engineering project. At a stage when we need to rapidly iterate RisingWave’s core features, that’s simply not a good trade-off.
Why Not Debezium Server
If we’re not rewriting it, why not use Debezium Server instead of Embedded Engine?
The main difference is:
Debezium Server is a standalone process. You configure the source database and downstream target (Kafka, Pulsar, etc.) via a config file, and it continuously runs. Think of it as a generic CDC forwarding tool.
Embedded Engine is a Java library you embed directly into your own process. You get CDC events through an API, and you control how they’re consumed, cached, forwarded, or recovered from errors.
We chose Embedded Engine for three main reasons:
Fine-grained control
Server mode is a black box—you’d need hacks to modify its default behaviors around data handling, memory management, backpressure, and error recovery. With Embedded, we drive the engine directly in code, so we can hook into any point we want.Deep integration with our compute engine
RisingWave’s compute nodes have their own scheduling, checkpointing, and DAG topology. CDC data must integrate tightly with these mechanisms. Embedded mode lets us inject logic right at the CDC entry point—Server mode can’t do that.Operational simplicity
With Server mode, we’d need to deploy, monitor, and upgrade a separate fleet of Debezium processes, and handle their failover and configuration management. Embedded mode keeps CDC in the same deployment, sharing the same monitoring, logging, and alerting systems.
Key Upgrades We Made to the Debezium Embedded Engine
Continuing with Debezium doesn’t mean we use it unmodified. To make it fit perfectly into our real-time compute engine, we’ve deeply customized the Embedded Engine in ways that wouldn’t be feasible with Server mode.
We’ve customized the Debezium embedded engine to power streaming ingestion in RisingWave, and we use Apache DataFusion for compaction when sinking data to Apache Iceberg. More here: https://risingwave.com/blog/postgres-cdc-iceberg-production-lessons/
1. Lock-free snapshot + parallel incremental stitching
Debezium’s default snapshot can lock tables, impacting the source DB’s operations. Our approach is to first record a snapshot boundary (high-watermark LSN or binlog position), then stream both the snapshot and WAL/binlog events simultaneously. The downstream deduplicates and overwrites as needed, guaranteeing consistency without long locks and getting us into real-time sync much faster.
2. Parallel backfill
The native Debezium initial snapshot processes a single table in a single thread, which can be slow. Starting from version 2.2, Debezium introduced the snapshot.max.threads
configuration, allowing multiple tables to be snapshotted in parallel - but this does not speed up the backfill of a single table. In our practice, for large individual tables, we introduced primary key range slicing and used multiple threads to backfill these slices in parallel. We also dynamically adjusted the number of threads based on the source database’s load, which significantly boosted performance while avoiding excessive pressure on the source system.
3. Cross-cloud schema history
Debezium’s native schema history storage supports Kafka, local files, S3, Azure Blob, etc., but it doesn’t cover everything we need - e.g., no GCS or Alibaba OSS. In RisingWave, we implemented a pluggable schema history layer supporting S3, GCS, Azure Blob, OSS, and more. This ensures seamless multi-cloud deployment and consistent schema history even during migrations.
4. Memory and backpressure control for schema history
In Debezium’s native schema history, the entire history file is loaded into memory at once. Over time, especially with frequent DDL changes, this can cause excessive memory usage - or, in extreme cases, OOM.
We modified the storage to split history into time-based or size-based segments, loading only what’s necessary for the current operation. This reduces memory peaks and improves stability in long-running, large-scale, multi-cloud environments.
5. TOAST column value completion
When PostgreSQL updates a non-TOAST column and a TOAST column remains unchanged, Debezium sends a placeholder value (e.g., __debezium_unavailable_value
) for that TOAST column rather than the full value. At our CDC ingestion point, we query the historical value, replace the placeholder with the actual data, and merge it with the new event. This ensures that downstream systems always receive complete and usable records.
6. PostgreSQL Schema Change Detection
In native Debezium, schema evolution handling is limited - especially in Embedded Engine mode - where it often relies on periodically polling metadata to detect column additions or deletions. This approach not only introduces high latency but also adds extra load to the upstream database.
We leverage the R
(Relation) message in PostgreSQL’s replication protocol to directly detect schema changes. When an R
message appears in the logical replication stream, it indicates that the schema of the corresponding table has changed (for example, an ADD COLUMN
or DROP COLUMN
). Upon receiving such an event, RisingWave immediately triggers its schema update logic, automatically completing column addition or deletion and synchronizing updated schema metadata downstream - eliminating the need for manual intervention and reducing the risk of data inconsistency.
These modifications are only possible because we run in Embedded Engine mode - Server mode offers almost no way to intercept and transform data at these critical points.
Conclusion
Rewriting Debezium in Rust might sound exciting, but in reality, its ecosystem maturity, stability, version compatibility, and testing depth are not things you can match in a short time. What we need is a stable, controllable, and deeply integrated CDC entry point for RisingWave - not a years-long rebuild project.
Choosing Debezium Embedded Engine lets us stand on the shoulders of a mature ecosystem while optimizing the parts that matter most for our workload. It’s the best balance we can strike right now between engineering efficiency, stability, and maintainability.