It’s with immense joy and pleasure to announce the availability of Debezium 3.0.0.Final!
We released Debezium 2.0 nearly 2 years ago, and in that time, the platform has continued to grow, introducing sink-based connectors, new community-led connectors, and an extensive list of features and improvements to the core platform and connectors. With the community’s help, Debezium remains the de facto leader in CDC.
The 3.0 release marks another milestone for Debezium, one that we’re eager to share.
In this post, we’re going to take a deep dive into all the changes in Debezium 3.0, discussing new features, and explaining all the possible changes that could have any impact to your upgrade process. As always, we recommend you read the release notes to learn about all the bugs that were fixed, update procedures, and more.
Changes to core Debezium
In this section, we’re going to jump into changes that affect the core of Debezium, and discuss how these changes impact all users.
Java 17 is required
This release shifts the Java requirements needed to both build and run Debezium. In addition, this release requires a later version of Maven to build Debezium from source.
All Debezium connectors require a runtime baseline of Java 17.
If you are using Debezium Server, Operator, or the Quarkus Outbox Extension, a runtime baseline of Java 21 is required.
If you intend to build Debezium from source, all Debezium projects require Java 21 and Maven 3.9.8 or later. When building Debezium from source using a Java version before Java 21, the build will fail, reporting that Java 21 or later is required.
Please see the following chart, showing an at-a-glance view of the Maven and Java requirements by component:
Component | Java (Runtime) | Java (Building) | Maven (Building) |
---|---|---|---|
Debezium Server | Java 21+ | Java 21+ | 3.9.8+ |
Debezium Operator | Java 21+ | Java 21+ | 3.9.8+ |
Debezium Quarkus Outbox Extension | Java 21+ | Java 21+ | 3.9.8+ |
Debezium Connectors | Java 17+ | Java 21+ | 3.9.8+ |
Built on Kafka 3.8
This release moves to Kafka 3.8 as our baseline for testing and building Debezium. Kafka 3.8 changed a number of internal APIs that needed to be adapted for Debezium’s use (DBZ-8105).
For most users, this change has no impact; however, if you are extending Debezium, it’s important to be aware of these changes.
Removal of deprecated incremental signal fields
In Debezium 2.4, the additional-condition
field in the incremental snapshot signal payload was deprecated, replaced with the new additional-conditions
property, allowing the specification for conditions by table. In this release, the old additional-condition
field has been removed and is no longer supported (DBZ-8278). Please be sure to update your scripts, workflows, or documentation that may have referred to this old, deprecated field.
Detailed metrics per table
Debezium will now begin to track metrics based on the individual create, update, and delete operations performed per relational table. For some connectors such as PostgreSQL and Oracle, these new detailed metrics also track the truncate operations performed per relational table. This can be quite useful for situations where you need to detect specific mutation patterns or where you may want to integrate analytics or observability stacks where this detailed information could be valuable to identifying problems.
For users upgrading to Debezium 3, these new metrics are captured automatically. They are exposed using a map-based pattern of Map<String, Long>
where the key is the table name and the value is the number of events observed. The new metrics names are NumberOfCreateEventsSeen
, NumberOfDeleteEventsSeen
, NumberOfUpdateEventsSeen
, and NumberOfTruncateEventsSeen
(DBZ-8035).
Changes to MariaDB connector
Version 11.4.3 support
Debezium 3 shifts its baseline support for MariaDB for the most recent non-rolling release, 11.4.3 (DBZ-8226). We are also closely monitoring the MariaDB 11.6 release cycle and plan to introduce vector data type support when MariaDB 11.6 becomes stable.
Changes to MongoDB connector
MongoDB sink connector
Debezium introduced its first sink-based connector in Debezium 2.2, just over a year ago, and we’re pleased to announce the inclusion of another sink-based connector for MongoDB as a part of Debezium 3.
Unlike the JDBC sink relational connector that requires an additional plug-in to be installed to use it, the MongoDB sink connector is bundled alongside the MongoDB source connector in the same artifact. So if you have already installed or use the MongoDB source connector and are using Debezium 3 or later, you also have the MongoDB sink connector.
The configuration to get started with the MongoDB is quite straightforward, here’s an example:
{
"connector.class": "io.debezium.connector.mongodb.MongoDbSinkConnector",
"connection.string": "...",
"topics": "topic1,topic2",
"sink.database": "targetdb"
}
The connection.string
and sink.database
configuration properties are mandatory. These define the details for connecting to the target MongoDB database and the name of the target database where the changes will be written.
Additionally, the topics
configuration property is mandatory by Kafka Connect, and it describes a comma-separated list of regular expressions for the topics that the sink connector will observe.
Documentation for this connector is still a work-in-progress, so if you have any questions or problems, don’t hesitate to reach out via our community channels. |
Changes to MySQL connector
MySQL 9
Oracle unveiled the first innovation release of MySQL 9.0 on July 1st, 2024. We are pleased to announce that we’ve tested and verified that MySQL 9.0 works and is supported starting with Debezium 3.0 (DBZ-8030). If you experience any issues or problems, please be sure to open an issue.
MySQL Vector data types
One of the newest features being added to relational databases is the introduction of vector data types. In addition to support for MySQL 9.0, Debezium 3 also introduces support for the new VECTOR(n)
data type, which supports a list of floating-point values that can be expressed as a binary or list-formatted string. More information is available in the MySQL documentation about the vector data type (DBZ-8157).
In addition, the MySQL grammar has been updated to reflect support for the new MySQL 9.0 vector functions (DBZ-8210). More information about these functions are also in the MySQL documentation.
Changes to Oracle connector
Deprecated configuration properties removed
Several deprecated configuration properties have been removed:
-
log.mining.transaction.retention.hours
replaced bylog.mining.transaction.retention.ms
-
log.mining.archive.destination.name
replaced byarchive.destination.name
-
log.mining.archive.log.hours
replaced byarchive.log.hours
Please be sure to update your Oracle connector configuration when using the deprecated configuration options to retain old behavior (DBZ-8181).
Default mining strategy changed
The default log.mining.strategy
value has changed and is now online_catalog
. As a vast majority of users typically use this strategy, and it generally performs better than redo_log_catalog
, we felt this change made since in Debezium 3. If your deployments were previously relying on the default redo_log_catalog
strategy, you will need to explicitly add log.mining.strategy
to the connector configuration and specify the value redo_log_catalog
when upgrading (DBZ-3656).
Oracle Ehcache transaction buffer implementation
Debezium 3 introduces as new Oracle connector transaction buffer implementation, based on Ehcache to provide off-heap storage of transaction processing and event data. This new implementation adds to the existing Java Heap, Infinispan Embedded, and Infinispan Remote buffer types.
To begin taking advantage of the Ehcache implementation, the log.mining.buffer.type
must be set to ehcache
. By default, the buffer type is memory
to use the JVM’s heap for optimal performance.
In order to for the Ehcache library to start successfully, several additional configurations must be provided to explicitly configure the caches maintained by the cache manager. These new configuration options are:
-
log.mining.buffer.ehcache.global.config
-
log.mining.buffer.ehcache.transactions.config
-
log.mining.buffer.ehcache.processedtransactions.config
-
log.mining.buffer.ehcache.schemachanges.config
-
log.mining.buffer.ehcache.events.config
Debezium creates the Ehcache configuration using XML, so each of these configurations provide XML snippets.
The global configuration is optional, and allows you to provide details about persistence and other Ehcache attributes, excluding specifying <cache>
or <default-serializers>
tags, which are handled separately. The other individual cache configurations are meant to supply the inner XML bits of a <cache>
configuration tag, excluding its <key-type>
and <value-type>
, which are managed directly by Debezium.
{
"log.mining.buffer.type": "ehcache",
"log.mining.buffer.ehcache.global.config": "<persistence directory=\"./data\"/>",
"log.mining.buffer.ehcache.transactions.config": "<resources><heap unit=\"entries\">256</heap><disk unit=\"B\">10485760</disk></resources>",
"log.mining.buffer.ehcache.processedtransactions.config": "<resources><heap unit=\"entries\">256</heap><disk unit=\"B\">10485760</disk></resources>",
"log.mining.buffer.ehcache.schemachanges.config": "<resources><heap unit=\"entries\">256</heap><disk unit=\"B\">10485760</disk></resources>",
"log.mining.buffer.ehcache.events.config": "<resources><heap unit=\"entries\">256</heap><disk unit=\"B\">10485760</disk></resources>"
}
In this example, Ehcache will maintain a combination of heap and off-heap storage for the caches, maintaining at most 256 entries in heap at all times and flushing to disk. The disk caches will be stored at the relative path ./data
. This implies that you will need a persistent storage volume available when using disk-based caches.
This is a new feature and is experimental, so we would love your feedback on how we can improve this (DBZ-7758).
Oracle offline RAC node flush improvements
In recent improvements to the Oracle RAC node flush strategy, it was determined that a three-second delay was being forced when an Oracle RAC node was taken offline by the database administrator. Since an Oracle RAC node cannot perform any writes to the redo logs while offline, this three-second delay introduced an unnecessary amount of latency while the node remained offline.
In Debezium 3, the three-second delay is only imposed if a connection is active to an Oracle RAC node; however, the flush SQL operation was unsuccessful. This means that when database administrators take RAC nodes offline for maintenance, no latency overhead will be imposed by the connector (DBZ-8177).
Oracle EXTENDED max string size support
Oracle extended strings is a feature that allows the traditional 4000 byte limit on character data to be raised to 32K. This is done by applying a database upgrade to set the database parameter max_string_size
to EXTENDED
. The extended string feature then allows using the same SQL syntax used for 4000 byte or smaller character data to be used for character data up to 32K without forcing you to use CLOB-based operations.
With Debezium 3, you can now use the Oracle connector with databases that use extended strings and capture the changes directly from the transaction logs (DBZ-8039). As extended strings are effectively CLOB operations on the database level, mining such column types require setting lob.enabled
to true
.
As this new feature is experimental, we’d love to hear any feedback from the community!
Oracle CLOB/BLOB default value support
In some cases, Oracle users may define tables with a CLOB or BLOB as required, using the EMPTY_BLOB()
or EMPTY_CLOB()
function to define a default when the field isn’t supplied. In previous builds, these special functions were not evaluated by Debezium, and such columns would have been emitted as optional rather than not optional.
Starting with Debezium 3, when an EMPTY_BLOB()
or EMPTY_CLOB()
default value is specified, the field will be emitted as not optional. Additionally, the field contain the appropriate default value, an empty byte array or an empty string respectively (DBZ-8248).
Changes to PostgreSQL connector
PostgreSQL replication slot creation timeout
When the PostgreSQL connector is first deployed, one of its first tasks is to create a replication slot in the database if it doesn’t already exist. The replication slot is pivotal to how the connector works and facilitates the capture and dispatch of changes to Debezium. Unfortunately, there are some database operations that will block the creation of replication slots, such as in-progress transactions, forcing the connector to block indefinitely while waiting for the transaction to conclude. For short-lived transactions, this isn’t generally a concern; however, for long-running transactions that’s an entirely different situation.
In order to improve this experience, a new internal option was added, internal.create.slot.command.timeout
, which defaults to 90 seconds. If the creation of the replication slot does not complete within 90 seconds, it will retry up to slot.max.retries
. Once the retries are exhausted, the connector will throw an unrecoverable error (DBZ-8073).
Support for PostgreSQL PgVector
data types
The pgvector
extension introduces vector search functionality for PostgreSQL. There are three data types this extension introduces: vector
, halfvec
, and sparsevec
.
In Debezium 3, all three data types will be streamed like any other data type. Each data type is emitted based on the following semantic mappings:
-
vector
as anARRAY
of numeric values -
halfvec
as anARRAY
of numeric values -
sparsevec
as aStruct
with number of dimensions and map of index to values
There is no additional configuration required after enabling the pgvector
extension in your database. Please see the documentation for more details on the semantic mappings (DBZ-8121).
If you used a preview release of Debezium 3 before 3.0.0.CR1, the schema names were adjusted to be more generic to support multiple database vendors (DBZ-8183). Please review event schemas if you are upgrading from a prior Debezium 3 preview release. |
Transformation to decode PostgreSQL logical messages
PostgreSQL is unique in that you can implement the Outbox pattern without creating an outbox table, by writing logical messages directly into the WAL using pg_logical_emit_message
. The unfortunate part is that this data is then sent to Kafka as a series of bytes, which may not always be ideal for consumers who may be looking for structured messages.
Debezium 3 introduces a new PostgreSQL-specific transform called DecodeLogicalDecodingMessageContent
. This transform is specifically meant to convert the pg_logical_emit_message
event bytes to a structured event payload that consumer applications are capable of understanding.
Given the following configuration:
{
"transforms": "decode",
"transforms.decode.type": "io.debezium.connector.postgresql.transforms.DecodeLogicalDecodingMessageContent"
}
The event’s value
of an event written using pg_logical_emit_message
before the transform would be:
{
"op": "m",
"ts_ms": 1723115240065,
"source": {
...
},
"message": {
"prefix": "test-prefix",
"content": "eyJpZCI6IDEsICJpdGVtIjogIkRlYmV6aXVtIGluIEFjdGlvbiIsICJzdGF0dXMiOiAiRU5URVJFRCIsICJxdWFudGl0eSI6IDIsICJ0b3RhbFByaWNlIjogMzkuOTh9"
}
}
After applying the transformation, the event’s value
now looks like:
{
"op": "c",
"ts_ms": 1723115415729,
"source": {
...
},
"after": {
"id": 1,
"item": "Debezium in Action",
"status": "ENTERED",
"quantity": 2,
"totalPrice": 39.98
}
}
So you can safely implement the Outbox pattern without the physical outbox table! (DBZ-8103).
PostgreSQL isolation level support
A longstanding enhancement for snapshot isolation support for PostgreSQL is now here! A new connector configuration property, snapshot.isolation.mode
, allows the connector to control the consistency used while executing the initial and ad-hoc blocking snapshot steps. There are four isolation levels: serializable
(the default), repeatable_read
, read_committed
, and read_uncommitted
. You can find details about these isolation levels and how they work with PostgreSQL in the documentation (DBZ-1252).
Reselect post processor improvements
The ReselectPostProcessor
is a useful tool to handle populating change events that contain TOAST columns (the oversized-attribute storage technique). By default, when a TOAST column is found and is not mutated by the SQL operation, Debezium populates these fields with placeholders, indicating that the value wasn’t provided, but also wasn’t changed. A host of data types use this storage mechanism, including int/bigint arrays. With Debezium 3, these int/bigint array data types can be reselected by the post processor so that these fields are always populated, even when they’re not changed in the SQL operation (DBZ-8212).
Changes to SQL Server connector
Signal and notification MBean name changes
The JMX signaling and notifications for SQL Server did not work correctly when a connector was configured with multiple databases spawning multiple tasks. To resolve this issue, it was necessary to change the naming of signalling and notification MBean names to make sure they are unique per task (DBZ-8137).
Changes to JDBC sink connector
Relocation of JDBC sink repository
The JDBC sink repository has been relocated from debezium-connector-jdbc to debezium main repository (DBZ-8008). With the introduction of the MongoDB sink connector in Debezium 3, this allows the team to easily share common contracts across our sink connectors.
Moving forward, to raise pull requests for the JDBC sink, please use the main Debezium repository, as the old repository has been archived and is only read-only.
JDBC retry flushes on specific failures
The JDBC sink uses a set of buffers to improve the throughput writes to the target database. In some use cases, the flush operation of these buffers may face specific exceptions due to locks due to other applications that may have locked a specific row or table. To improve the user experience, two new configuration properties have been added:
flush.failure.max.retries
-
Defines the number of retries when a flush failure occurs.
flush.failure.retries.wait.ms
-
Defines the number of milliseconds to wait between a retry.
The retry feature is enabled by default, attempting to retry up to a maximum 5
attempts, with a 1-second delay between retries. If you prefer retries disabled, setting flush.failure.max.retries
to 0
would disable this feature (DBZ-7291).
Changes to Debezium Server
Breaking changes
- Debezium Server Kafka Sink
-
The Debezium Server Kafka sink adapter could wait indefinitely when a Kafka broker becomes unavailable. A new configurable timeout has been added to the sink adapter to force the adapter to fail when the timeout is reached. The new option,
debezium.sink.kafka.wait.message.delivery.timeout.ms
, has a default value of 30 seconds. Please adjust this accordingly if the default is insufficient for your needs (DBZ-7575). - Debezium Server RabbitMQ sink
-
The Debezium Server RabbitMQ sink adapter was sending all changes to the same single stream. While this may be useful for some scenarios, this does not align well with other broker systems where each table is streamed to its own unique topic or stream. With Debezium 3, this logic has changed and each table will be streamed to its own unique stream by default. When setting
debezium.sink.rabbitmqstream.stream
, you can enable the legacy behavior of streaming all changes to the same stream (DBZ-8118).
Support custom converter types
In prior releases of Debezium Server, there were a finite number of converters that could be used for headers, keys, and values. These included Json
, JsonByteArray
, CloudEvents
, Avro
, Protobuf
, Binary
, and SimpleString
. While these often satisfied a vast majority of use cases, it’s not uncommon that someone may have a unique requirement specific to their environment that is outside these options.
In this release, a new ClientProvided
converter option has been added, which allows for extending the header, key, and value converters with a custom, user-supplied implementation (DBZ-8040).
Improved logging for Kafka sink
The Kafka sink adapter will now log the record key when Debezium fails to send the record to the Kafka broker. This is useful to know what specific record was a problem without necessarily needing to increase the logging verbosity of the runtime (DBZ-8282).
Changes to Spanner connector
Support for 32-bit floats
The Google Spanner database introduced support for a 32-bit float data type. The Debezium Google Spanner connector has been adjusted to support this new data type (DBZ-8043).
Changes to Vitess connector
Empty shard support
In Vitess, it is possible for a keyspace to have shards that have no tablets. Debezium for Vitess has improved working with this use case, and now gracefully handles such a keyspace without fault (DBZ-8053).
Inherit shard epoch
A new Vitess connector configuration property has been added to control whether epochs of a new shard, after a re-shard operation, inherits epochs from its parent shard. This new configuration property, vitess.inherit.epoch
, defaults to false
and isn’t enabled by default (DBZ-8163).
Other fixes & improvements
There were many bugfixes, stability changes, and improvements throughout the development of Debezium 2.0. Altogether, a total of 202 issues were fixed for this release.
A big thank you to all the contributors from the community who worked on this major release: Jordan Pittier, Kanthi Subramanian, Katerina Galieva, Kaustuv chakrabarti, Keri Harris, Kevin Rothenberger, Kosta Kostelnik, Lars M. Johansson, Liz Chatman, Lokesh Sanapalli, Lourens Naudé, Luca Scannapieco, M. Gökhan Akgül, Maithem, Marcelo Avancini, Mario Fiore Vitale, Mark Banierink, Mark Bereznitsky, Mark Ducommun, Mark Lambert, Martin Medek, Massimo Fortunat, Matt Vance, Mehmet Firat Komurcu, Michal Augustýn, Michal Pioun, Mickael Maison, Miguel Angel Sotomayor, Mike Kamornikov, Minh Son Nguyen, Mohamed El Shaer, Mostafa Ghadimi, My Lang Pangzi, Nancy Xu, Nick Golubev, Nikhil Benesch, Nir Levy, Olivier Boudet, Ondrej Babec, Oren Elias, Paul Cheung, Pengwei Dou, Peter Hamer, Piotr Piastucki, Plugaru Tudor, Poonam Meghnani, Pradeep Nain, Praveen Burgu, RJ Nowling, Rafael Câmara, Rajendra Dangwal, Raúl Estrada, René Kerner, Richard Harrington, Robert Roldan, Robin Moffatt, Roman Kudryashov, Ronak Jain, Russell Mora, Ryan van Huuksloot, Sahap Asci, Sean C. Sullivan, Sean Wu, Sebastiaan Knijnenburg, Selman Genç, Seo Jae-kwon, Seongjoon Jeong, Sergei Kazakov, Sergei Morozov, Sergey Eizner, Sergey Ivanov, Stavros Champilomatis, Stefan Miklosovic, Stein Rolevink, Stephen Clarkson, Subodh Kant Chaturvedi, Sun Xiao Jian, Sylvain Marty, Thomas Thornton, Théophile Helleboid, Tiernay, Tim Loes, Timo Wilhelm, Tomasz Gawęda, Tommy Karlsson, https://github.com/blcksrx Hossein[Torabi], Tudor Plugaru, V K, Vadzim Ramanenka, Vaibhav Kushwaha, Vincenzo Santonastaso, Vincenzo Santonastaso, Vojtěch Juránek, Wu Zhenhua, Xianming Zhou, Xiaojian Sun, Xinbin Huang, Xuan Shen, Yang Wu, Yanjie Wang, Yashashree Chopada, Yohei Yoshimuta, Zheng Wang, Zhongqiang Gong, baabgai, david remy, einar-rt, ibnubay, ismail simsek, leoloel, martin, ming luo, moyq5, ruslan, sean, tison, tony joseph, tooptoop4, yohei yoshimuta, حمود سمبول, and 蔡灿材!
What’s next?
For the remainder of the 2024 calendar year, the team will continue to deliver maintenance and bugfix releases for Debezium 3. These will continue at our normal cadence, approximately every 2-3 weeks, barring issues reported and those fixed. However, what will be changing this quarter is that there won’t be a minor release in December like years past. This decision is to allow the team to focus on a number of low-hanging fruit activities that don’t necessarily or directly tie to development tasks.
In addition, as we get closer to the end of the year, we’ll be putting out our vision of the 2025 roadmap and what that includes. Now is a perfect time to think about what you’d like to see on that roadmap and get involved. You can always reach us on our mailing list or in our Zulip chat!
About Debezium
Debezium is an open source distributed platform that turns your existing databases into event streams, so applications can see and respond almost instantly to each committed row-level change in the databases. Debezium is built on top of Kafka and provides Kafka Connect compatible connectors that monitor specific database management systems. Debezium records the history of data changes in Kafka logs, so your application can be stopped and restarted at any time and can easily consume all of the events it missed while it was not running, ensuring that all events are processed correctly and completely. Debezium is open source under the Apache License, Version 2.0.
Get involved
We hope you find Debezium interesting and useful, and want to give it a try. Follow us on Twitter @debezium, chat with us on Zulip, or join our mailing list to talk with the community. All of the code is open source on GitHub, so build the code locally and help us improve ours existing connectors and add even more connectors. If you find problems or have ideas how we can improve Debezium, please let us know or log an issue.