As we begin to approach the final stretch for Debezium 2.5, I am pleased to announce that Debezium 2.5.0.Beta1 is now available. This release includes a number of improvements like support for MariaDB GTID, partitioning for Debezium Server EventHub’s sink, native RabbitMQ streams Debezium Server sink, streaming from PostgreSQL 16 stand-by databases, MySQL high-precision source timestamps, field inclusion/exclusion with JDBC sink, some additional notifications for initial snapshots, and service account support for Debezium Operator CRDs. Additionally, this release includes a variety of bug fixes and several breaking changes.

Let’s take a closer look at all these changes and improvements included in Debezium 2.5.0.Beta1; as always, you can find the complete list of changes for this release in the release notes. Please remember to take special note to any breaking changes that could affect your upgrade path.

Breaking changes

While we strive to avoid breaking changes, sometimes those changes are inevitable to evolve the right direction. This release includes several breaking changes.

MongoDB snapshots in sharded deployments

In earlier versions of Debezium, a user could use the MongoDB connector and perform a snapshot of a collection on a specific shard when using a MongoDB sharded deployment. Unfortunately, while we understand that users may have taken advantage of this behavior, it was unintended and not supported.

Debezium 2.5 removes this ability entirely, and it’s a step in the forward goal to deprecate and remove the replica-set streaming mode in MongoDB completely (DBZ-7139).

ComputePartition SMT removed

The ComputePartition was an SMT that enabled users to compute the desired partition for which an event would be written in a Kafka topic. This single message transformation (SMT) was deprecated some time ago in favor of the new PartitionRouting single message transformation.

Debezium 2.5 removes the old, deprecated ComputePartition, and users who may still be relying on this SMT will need to migrate to the new PartitionRouting SMT if you have not already done so (DBZ-7141).

JDBC sink value serialization changes

There was a corner case reported where the JDBC sink connector was mistakenly writing fields that had null values to the target database with the default value instead of a value of NULL. This has been fixed in Debezium 2.5 (DBZ-7191).

New features and improvements

Debezium 2.5 also introduces more improvements and features, lets take a look at each individually.

Additional notifications for initial snapshots

Debezium’s notification subsystem provides a simple way to integrate external processes and applications with certain Debezium tasks, such as taking a snapshot. In prior versions, the notifications for initial snapshots were quite basic and provided details such as when the snapshot started, when each table started and concluded, and finally when the snapshot ended.

A recent contribution that was part of DBZ-6878 extends on this base functionality to provide enhanced details about the snapshot. For example, the IN_PROGRESS notification will provide additional details about what tables are being captured and which table is currently in-progress, shown below:

{
   "id":"6d82a3ec-ba86-4b36-9168-7423b0dd5c1d",
   "aggregate_type":"Initial Snapshot",
   "type":"IN_PROGRESS",
   "additional_data":{
      "connector_name":"my-connector",
      "data_collections":"table1, table2",
      "current_collection_in_progress":"table1"
   },
   "timestamp": "1695817046353"
}

Additionally, another initial snapshot notification called TABLE_SCAN_COMPLETED also provides similar details, as shown below:

{
   "id":"6d82a3ec-ba86-4b36-9168-7423b0dd5c1d",
   "aggregate_type":"Initial Snapshot",
   "type":"TABLE_SCAN_COMPLETED",
   "additional_data":{
      "connector_name":"my-connector",
      "data_collection":"table1, table2",
      "scanned_collection":"table1",
      "total_rows_scanned":"100",
      "status":"SUCCEEDED"
   },
   "timestamp": "1695817046353"
}
Several fields shown above such as data_collection, these are not currently available for MongoDB snapshots and are only available for SQL-based relational connectors.

For more details, please see the Notification documentation.

MySQL high-precision source timestamps

Several new high-precision timestamp fields have been included in several MySQL replication events. For example, in MySQL 8.0.1, there are to microsecond-resolution timestamps added to GTID events that specify the timestamp when the transaction was committed on the immediate primary and when committed on the original primary.

Debezium 2.5 now takes advantage of these values and will use these for the ts_ms fields if they’re available, falling back to the second-based precision if they are not or if you’re using a version of MySQL prior to 8.0.1 (DBZ-7183).

MariaDB GTID support

Both MySQL and MariaDB support what is called Global Transaction Identifiers or GTIDs. These are used in replication to uniquely identify transaction(s) uniquely across a cluster. The implementation details between MySQL and MariaDB differ significantly and in earlier versions of Debezium, we only supported GTID with MySQL.

With Debezium 2.5, we are taking another step forward by introducing GTID support for MariaDB as part of the MySQL connector offering. In order to take advantage of this behavior, you will need to use the MariaDB driver rather than the MySQL driver by using a JDBC connection prefixed as jdbc:mariadb rather than jdbc:mysql. By doing this, you can now take full advantage of working with MariaDB and GTID just like MySQL (DBZ-1482).

It is planned to introduce a new, standalone connector in Debezium 2.6 for MariaDB. A lot of what we have done in 2.5 will carry over to this new connector in 2.6, so any feedback we can obtain earlier will only benefit that transition in early 2024!

Streaming from PostgreSQL 16 stand-bys

In PostgreSQL 16, you can now define replication slots on a stand-by instance. This enables a plethora of new options, including the ability to perform change data capture from a replica rather than your production system for load distribution, particularly in a very active database.

Debezium 2.5 now supports connecting to a stand-by PostgreSQL 16 server and streaming changes (DBZ-7181).

JDBC sink field inclusion/exclusion

Debezium 2.5 introduces a new JBDC sink feature where users can now specify which fields from the event payload are to be included or excluded from the target database write operation. This feature works just like any other include/exclude combination in the Debezium framework where these two properties are mutually exclusive.

As an example, if we have a simple event payload with the following fields in topic customers:

{
  "id": 12345,
  "name": "Acme",
  "address": "123 Main Street"
}

If we want to avoid writing the address field to the target database and only write the id and name fields to the target table, we can use this new feature to accomplish this. This can be done adding either a field.include.list or field.exclude.list property.

Example that prevents writing the address field to target
{
  "field.exclude.list": "customers:address"
}

The format of the include/exclude for fields is [<topic-name>:]<field-name>, where the topic-name is optionally and can be omitted if you want to avoid writing the address field for all events. Please see the JDBC sink connector configuration documentation for more details.

Debezium Server - EventHubs partitioning

In earlier versions of Debezium Server, users could specify a fixed partition-id to stream all changes to a single partition or provide a static partition-key that will be set on all batch operations, which ultimately lends itself to streaming all changes to the same target partition. There are situations where this may be helpful, but it more often leads to a performance concerns for downstream processing.

Debezium 2.5 adjusts this behavior in order to improve performance. By default, when neither a partitionid or partitionkey is defined, the EventHub sink will send events using a round-robin technique to all available partitions. Events can be forced into a single, fixed partition by specifying a partitionid. Alternatively, the partitionkey can be provided to supply a fixed partition key that will be used to route all events to a specific partition.

If additional partition routing requirements are necessary, you can now combine the PartitionRouting SMT accomplish such tasks. For more details, please see the Event Hubs documentation.

Debezium Server - RabbitMQ Streams sink

RabbitMQ introduced Streams in version 3.9, which utilizes a fast and efficient protocol that can be combined with AMQP 0.9.1 to support large fan-outs, replay and time travel, and large data sets with extremely high throughput. Debezium 2.5 takes advantage of this new Streams implementation by introducing a new native Streams implementation (DBZ-6703). In order to get started with this new implementation, configure the Debezium Server sink as follows:

debezium.sink.type=rabbitmqstream
debezium.sink.rabbitmqstream.connection.host=<hostname of RabbitMQ>
debezium.sink.rabbitmqstream.connection.port=<port of RabbitMQ>

Additionally, if you need to pass any other connection parameters to the RabbitMQ connection, you can do so by adding those to the configuration with the prefix debezium.sink.rabbitmqstream.connection. to pass through any config properties.

Please see the Debezium Server RabbitMQ documentation for more details.

Debezium Operator - Service Account for CRDs

In previous versions of Debezium, it was not possible to use a service account named differently than the predefined one. This made the process a tad bit cumbersome for users because while you could grant roles and authorization to this predefined account separately, it meant you needed to use this predefined service account rather than one that you may already wish to use.

Debezium 2.5 simplifies this process, allowing you to now use your own, custom service account (DBZ-7111).

Other fixes

In addition, there were quite a number of stability and bug fixes that made it into this release. These include the following:

  • Generate sundrio fluent builders for operator model DBZ-6550

  • Convert operator source into multi module project DBZ-6551

  • Implement "validate filters" endpoint in connector-specific Connect REST extensions DBZ-6762

  • Test Avro adjustment for MongoDb connector and ExtractNewDocumentState SMT DBZ-6809

  • Implement IT tests against Cloud Spanner emulator in main repo. DBZ-6906

  • The DefaultDeleteHandlingStrategy couldn’t add the rewrite "__deleted" field to a non-struct value DBZ-7066

  • Implement strategy pattern for MariaDB and MySQL differences DBZ-7083

  • Debezium server has no default for offset.flush.interval.ms DBZ-7099

  • Failed to authenticate to the MySQL database after snapshot DBZ-7132

  • Run MySQL CI builds in parallel DBZ-7135

  • Failure reading CURRENT_TIMESTAMP on Informix 12.10 DBZ-7137

  • Debezium-ddl-parser crashes on parsing MySQL DDL statement (specific UNION) DBZ-7140

  • outbox.EventRouter SMT throws NullPointerException when there is a whitespace in fields.additional.placement value DBZ-7142

  • Debezium-ddl-parser crashes on parsing MySQL DDL statement (specific UPDATE) DBZ-7152

  • Add matrix strategy to workflows DBZ-7154

  • Add Unit Tests for ServiceAccountDependent Class in Debezium Operator Repository DBZ-7155

  • JsonSerialisation is unable to process changes from sharded collections with composite sharding key DBZ-7157

  • Log sequence check should treat each redo thread independently DBZ-7158

  • Fail fast during deserialization if a value is not a CloudEvent DBZ-7159

  • Correctly calculate Max LSN DBZ-7175

  • Upgrade to Infinispan 14.0.20 DBZ-7187

  • Upgrade Outbox Extension to Quarkus 3.5.3 DBZ-7188

  • Fix DebeziumMySqlConnectorResource not using the new MySQL adatper structure to support different MySQL flavors DBZ-7179

  • Parsing MySQL indexes for JSON field fails, when casting is used with types double and float DBZ-7189

  • Unchanged toasted array columns are substituted with unavailable.value.placeholder, even when REPLICA IDENTITY FULL is configured. DBZ-7193

  • Enable ability to stream changes against Oracle 23c for LogMiner DBZ-7194

  • Add modify range_partitions to modify_table_partition rule in parsing PL/SQL DBZ-7196

  • MongoDB streaming pauses for Blocking Snapshot only when there is no event DBZ-7206

  • Handle Drop Tablespace in PL/SQL DBZ-7208

  • Upgrade logback to 1.2.12 DBZ-7209

  • NPE on AbstractInfinispanLogMinerEventProcessor.logCacheStats DBZ-7211

What’s next?

We have about three weeks before the team takes a break for the holidays, and so this will be our one and only beta release for Debezium 2.5. We intend to release our release candidate in the middle of next week and with the final due out the week before the holiday break.

The team is also working on a maintenance release of Debezium 2.4, due out later this week. This update to Debezium 2.4 will bring a host of bug fixes and stability improvements already in Debezium 2.5 to the 2.4 release stream.

The team is also finalizing our roadmap for 2024. I will have details later in the week posted on our website as well as a complete overview of Debezium 2.5 and our 2024 roadmap in detail as part of our Debezium Community Event, Tuesday December 12th at 10am ET. All are welcomed to attend on Google Meet.

As always, please be sure to get in touch with us on the mailing list or Zulip chat if you have questions or feedback. Until next time, stay warm out there!

Chris Cranford

Chris is a software engineer at Red Hat. He previously was a member of the Hibernate ORM team and now works on Debezium. He lives in North Carolina just a few hours from Red Hat towers.

   


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.