With the new year comes a new development cycle, and I’m pleased to announce our first pre-release of Debezium 3.1, 3.1.0.Alpha1. This release includes a variety of new features including vector data type support for PostgreSQL and MySQL in the JDBC sink, new adaptive temporal precision modes, MariaDB SSL support, and more. Without further ado, lets jump right into these new features and improvements.

Breaking changes

With any new major release of software, there is often several breaking changes. The Debezium 3.1.0.Alpha1 release is no exception, so let’s discuss the major changes you should be aware about.

Event source block now versioned

Debezium’s change events contain a source information block that depicts attributes about the origin of the change event. The source information block is a Kafka Struct data type, and can be versioned; however, in older versions of Debezium the version attribute was left empty.

With Debezium 3.1 and later, the source information block is now versioned, and will be set to a version of 1 (DBZ-8499). As future changes are implemented, the version will be incremented accordingly.

For those using schema registry, this change may likely introduce schema compatibility issues.

Sparse vector logical type renamed

The PostgreSQL extension vector (aka pgvector) provides an implementation of a variety of vector data types, including one called sparsevec. A sparse vector is one that stores only the populated key/value entries within the vector, excluding those that aren’t set to minimize the space used to represent the data set.

Debezium 3.0 introduced the SparseVector logical type named io.debezium.data.SparseVector. As we began to evaluate implementations for other relational databases, we determined that the logical name was insufficient when we began to implement other sparse vector types (DBZ-8585).

With Debezium 3.1, we have repackaged the io.debezium.data.SparseVector class from the PostgreSQL connector into Debezium’s core package, we’ve also renamed the class to SparseDoubleVector, and changed the logical name to io.debezium.data.SparseDoubleVector to align with the class name changes.

For those who may have been working with SparseVector logical types, you may need to adjust your code to recognize the new logical type name.

New features and improvements

The upgrade to Debezium 3.1.0.Alpha1 introduces several new features and improvements in several components:

JDBC sink supports MySQL/PostgreSQL vector data types

We introduced a variety of vector data types as part of Debezium 3.0 in late 2024, which included vector for MySQL/PostgreSQL and halfvec/sparsevec for PostgreSQL. With Debezium 3.1, we’ve extended support for these data types to the JDBC sink connector (DBZ-8571).

This new mapping includes several rules:

  • MySQL to MySQL or MySQL to PostgreSQL, the vector data type is mapped automatically.

  • PostgreSQL vector to PostgreSQL or MySQL, the vector data type is mapped automatically.

  • Replication of halfvec and sparsevec are mapped automatically if the target is PostgreSQL only.

For target databases that do not have a native mapping for vector data types or has no support for such types, the field cannot be natively written to the target system. For such use cases, you can use the io.debezium.transforms.VectorToJsonConverter transformation to alter the event payload in-flight to a JSON representation, which most databases universally support. The target column type in the database will then either be json, clob, or a text-based column type depending on the database vendor.

As more source database vector types are supported, we’ll continue to expand this in the future.

New adaptive temporal precision mode types

Debezium has long supported a variety of time.precision.mode types like adaptive and connect. With Debezium 3.1, three new modes have been added to provide even more customization and choice for temporal-based columns (DBZ-6387).

Mode Description

isostring

Configures the connector to map temporal values as ISO-8601 formatted strings in UTC.

microseconds

Configures the connector to map temporal values with microsecond precision, if available.

nanoseconds

Configures the connector to map temporal values with nanosecond precision, if available.

When using microseconds or nanoseconds based precision modes, the connector will opt to use different semantic types based on whether the field is a DATE, TIME, or TIMESTAMP-based. Please review your specific connector documentation for details on how this is interpreted by a connector.

MariaDB SSL support

Debezium 3.1 introduces several new MariaDB-specific modes designed to allow the MariaDB connector to connect using SSL that are aligned and compatible with the MariaDB driver (DBZ-8482). The following table describes the modes and the MySQL equivalents if you’re moving from an older MySQL connector deployment to the new standalone MariaDB connector.

Mode Description

disable

Disables the use of SSL/TLS connectivity. All connections are insecure. This is the equivalent to MySQL’s disabled mode.

trust

Uses SSL/TLS for encryption, but does not perform certificate or hostname verification. This is the equivalent to MySQL’s required mode.

verify-ca

Uses SSL/TLS for encryption and performs certificate validation, but not host verification. This is the equivalent to MySQL’s verify_ca mode.

verify-full

Uses SSL/TLS for encryption and performs certificate and host validation. This is the equivalent to MySQL’s verify_identity mode.

For MariaDB, these properties are passed as part of the database.ssl.mode property.

SQL Server streaming memory improvements

The Microsoft SQL Server driver is unable to multiple when multiple selects are executed on the same connection. There are some cases where this may occur, and this often leads to all data buffered to memory, which can be quite inefficient or result in memory issues.

To address this driver limitation, Debezium 3.1 introduces a new configuration option, streaming.fetch.size (DBZ-8557). This configuration option allows specifying the maximum number of rows that should be read in a single fetch from each table while streaming. By default, this is set to 0 so the behavior remains unchanged, but setting this to a positive value results in multiple data round trips to the database to fetch the data in batches of the configured size.

Debezium content-based routing/filtering using WASM

The Debezium scripting module now includes support for running scripts using Chicory, a native JVM runtime for web assemblies (WASM) (DBZ-8658).

For example, given the following Go-based program:

package main

import (
    "github.com/debezium/debezium-smt-go-pdk"
)

func process(proxyPtr uint32) uint32 {
    var topicNamePtr = debezium.Get(proxyPtr, "topic")
    var topicName = debezium.GetString(topicNamePtr)
    return debezium.SetBool(topicName == "theTopic")
}

func main() {}

This Go program can be compiled into a web assembly .wasm file and then used by the ContentBasedRouter or Filter transformations. The following example shows how you would use this with the Filter transformation:

{
  "transforms": "route",
  "transforms.route.type": "io.debezium.transforms.Filter",
  "transforms.route.condition": "<path-to-compiled-wasm-file>",
  "transforms.route.language": "wasm.chicory"
}

In this example, if the event’s topic matches theTopic, the event is passed, otherwise the event is dropped.

For more information, you can see the documentation on the Filter SMT and the Content-based Router SMT.

Explicitly use path-style addressing with S3 storage

The S3 SDK introduced a small behavior change in 2.18+ where the URLs are built using virtual-host style instead of path style, as discussed in the upstream S3 SDK community. While the S3 bucket supports both styles of URL, there may be cases, including test cases where the virtual-host style may not yet be supported.

In Debezium 3.1, a new configuration option schema.history.internal.s3.forcePathStyle has been added, which defaults to false (DBZ-8569). In situations where you may need path style URLs rather than virtual-host style URLs, setting this to true will restore the older URL behavior.

Debezium Server Pulsar key-based batch support

Debezium 3.1 introduces a new and improved throughput option when using Apache Pulsar’s KeyShared subscription. A new configuration option, debezium.sink.pulsar.producer.batchBuilder has been added, which can be set to KEY_BASED, but defaults to DEFAULT (DBZ-8563).

When set to use KEY_BASED, this subscription model delivers messages with the same key to only one consumer in order. More information about Key_Shared subscription model can be found in the Apache Pulsar documentation.

Other changes

  • Exception during commit offsets won’t trigger retry logic. DBZ-2386

  • Fix invalid gtid error on startup when ordered tx metadata enabled DBZ-8541

  • Debezium operator generate wrong offset and schema history properties DBZ-8543

  • A recent log switch may be seen as consistent during log gathering DBZ-8546

  • Content-based routing expression variable headers is singular in code DBZ-8550

  • Upgrade MongoDB driver to 5.2 DBZ-8554

  • Use enum set strings flag available in Vitess v20 for decoding enums/sets DBZ-8561

  • MongoDataConverter does not recognize nested empty array DBZ-8572

  • Fix issues in Transformation UI sections DBZ-8575

  • ORA-65040 occurs on log switches when log.mining.restart.connection is enabled and connection defaults to PDB rather than CDB$ROOT DBZ-8577

  • Move to Quarkus 3.17.7 for the Outbox Extension DBZ-8583

In total, 26 issues were resolved in Debezium 3.1.0.Alpha1. The list of changes can also be found in our release notes.

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.