Debezium 3.3.0.Alpha1 introduces an exciting wave of innovation, including exactly-once semantics for core connectors and a brand new CockroachDB connector led by the community. Featuring support for emerging data types, deeper Quarkus integration, and enhanced tooling, this release raises the bar for modern change data capture solutions. Ready to see what’s new? Let’s dive in.

Breaking changes

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

Deprecated snapshot modes removed

In Debezium 2.6, we aligned the names of snapshot modes with their intended functionality, which included:

  • Deprecated schema_only_recovery and replaced with recovery.

  • Deprecated schema_only and replaced with no_data.

Starting with Debezium 3.3, both schema_only_recovery and schema_only have been removed (DBZ-8171). Connector configurations that still rely on the deprecated modes will cause invalid configuration errors after upgrading, so it’s important to adjust your connector configurations accordingly.

New features and improvements

The following describes all noteworthy new features and improvements in Debezium 3.3.0.Alpha1. For a complete list, be sure to read the release notes for more details.

New CockroachDB connector

The CockroachDB connector is a new connector, lead by the community in conjunction with Cockroach Labs (DBZ-9289). This connector is built on top of CockroachDB’s change-feed subsystem, to capture row-level changes, and emit those as change events.

This new connector requires CockroachDB 25.2+ with the rangefeed option enabled and Java 21+. This connector is currently incubating and is under heavy development by the community.

If you’d like to get started with the connector, you can get the connector with these coordinates:

<dependency>
  <groupId>io.debezium</groupId>
  <artifactId>debezium-connector-cockroachdb</artifactId>
  <version>3.3.0.Alpha1</version>
</dependency>

The documentation for this connector is still under development, but you can find information about its configuration options and how to use it in the repository’s README.

Exactly-Once Semantics support

There has been numerous requests from the community about Debezium with exactly-once semantics support. Exactly-once semantics in layman’s terms means that an event should only ever be delivered and written to a Kafka topic once, therefore avoiding the possibility of duplicates.

We are pleased to share that Debezium 3.3 introduces support for exactly-once semantics for all core connectors, including MariaDB, MongoDB, MySQL, Oracle, PostgreSQL, and SQL Server (DBZ-9177).

Disabling context headers

In Debezium 3.2, we introduced several new headers to provide context for OpenLineage that are automatically added to every emitted event. Some users reported concerns with these being added with no way to disable them, as some environments had the need to keep event payloads as lean as possible.

Based on community feedback, we have introduced a new configuration option, extended.headers.enabled, that can be set to false, disabling the addition of these context event headers (DBZ-9248).

MariaDB 11.7+ vector data type support

MariaDB 11.7 introduced the VECTOR(n) data type, allowing the storage of values in the relational database as if it were a vector database. Vector values that are generated by an AI model can be stored and searched in MariaDB using this new data type.

Debezium 3.3 introduces MariaDB vector data type support in the MariaDB source connector and the JDBC sink connector when writing to a MariaDB target database (DBZ-8582).

MariaDB and MySQL schema history improvements

The MariaDB and MySQL schema history topic is used by the connector to store a variety of DDL operations captured from the binary logs, allowing the connector to build a relational model from the chronological history of schema changes for tables.

In Debezium 3.3, the schema history topic’s filters were updated to avoid storing specific statements in the topic that are not relevant to Debezium’s function, such as procedure, function, view, trigger, and grant/revoke statements (DBZ-9186).

For new connectors, such statements will not be stored. For existing connectors, only newly captured DDL statements that match the improved filters will be omitted, existing topic events are not removed.

Oracle legacy decimal handling mode behavior

In Debezium 2.7, we introduced a bug fix that aligned the behavior of decimal.handling.mode when using the double or string modes in the Oracle connector to match other relational connectors. This bug fix introduced a change in the event’s schema and created numerous problems for those upgrading who were not expecting such changes.

We looked at a variety of options, whether to introduce legacy support via a CustomConverter or a Transformation, however, we concluded that the legacy support needed to be baked into the connector to provide a sufficient solution.

In Debezium 3.3, a new configuration property legacy.decimal.handling.strategy was added allowing users to restore the use of the broken behavior, where numeric values with zero-scale would not be automatically converted to double or string and instead would be emitted as integer whole values as long as their length was 18 or less (DBZ-9166).

If you are upgrading from a version prior to Debezium 2.7, it’s recommended you enable this feature at least initially to minimize the upgrade complexities to Debezium 3.3 or later.

If you have already upgraded to Debezium 2.7 through 3.2, it’s not recommended to enable this legacy behavior as it will introduce the same schema change complexities but the inverse.

Oracle LogMiner experimental CTE query support

We have introduced a new internal and experimental feature for Oracle LogMiner that leverages a concept called a CTE query, or a Common Table Expression query. A CTE query is a SQL construct that allows for the definition of a temporary, named result set within the execution of another SQL operation, which can be useful for a variety of reasons.

For the Debezium Oracle connector, the new internal.log.mining.use.cte.query feature toggles a special pre-pass that examines all transactions and performs a pre-filter pass (DBZ-9272). This filter pass is designed to only send START, COMMIT, and ROLLBACK events for transactions that actually contain at least one DML operation for a captured table. In other words, if someone performs 100 transactions and only one of those transactions modifies your captured table, then not only do we not receive the DML events for the other 99 transactions, but the transaction markers are also omitted, too.

This feature is not without caveats, more notably the fact it requires two passes over the transaction logs. For systems where there is a disproportionately higher volume of changes in non-captured versus captured tables, the extra read pass may be worth it to minimize the network and connector processing overhead of extraneous transaction marker events.

Postgres text-search vector support

In PostgreSQL 13+, full text search data types were added. The tsvector data type represents a document in a form optimized for text searches, providing a sorted list of distinct lexemes.

With Debezium 3.3, you can now capture changes made to columns using the tsvector data type (DBZ-8470). These column types are emitted using the logical type called io.debezium.data.Tsvector and is represented as a string data type in the event.

Outbox extension supports latest Quarkus

The Debezium Quarkus Outbox extension relies on Hibernate for persistence like most Quarkus-based extensions, and in the latest Quarkus 3.23.x, the version of Hibernate was updated to 7.x. This new version of Hibernate introduced a variety of breaking changes that lead to the extension being incompatible.

With Debezium 3.3 (and backported to the upcoming Debezium 3.2.1), the Debezium Quarkus Outbox extension is compatible with the latest builds of Quarkus working with Hibernate 7+ (DBZ-9219).

Due to the compatibility changes, when using Debezium 3.2.0.Final or earlier, the Quarkus application must be based on Quarkus 3.22.x or earlier. The use of Quarkus 3.24.x or later is only possible when using Debezium 3.2.1 / 3.3.x or later.

Built-in hashing partition keys for Azure Event Hubs

Azure Event Hub’s partition keys are limited to a maximum of 128 characters. In the event that an event’s key results in a partition key that exceeds that limit, Debezium Server’s Azure Event Hubs sink adapter would throw a exception and the connector would terminate. This is less than desirable behavior and introduces constraints on source tables where it may not be possible to address at the source.

To address this concern, the Azure Event Hubs sink can be configured to use one of several hashing algorithms to adhere to the maximum partition key size while working with source data where the partition key would otherwise exceed the maximum allowed size (DBZ-9245).

To enable the Azure Event Hubs sink to use the built-in hashing for partition keys, the sink must be configured using the debezium.sink.eventhubs.hashmessagekey configuration property set to true (the default is false). When enabled, the debezium.sink.eventhubs.hashmessagekeyfunction can be set to java (the default), md5, sha1, or sha256 to determine the type of hashing algorithm that’s used.

Debezium Platform introduces a Smart Editor

Managing similar Debezium connectors across different runtimes presents its own unique sets of maintenance challenges since each runtime uses different formats, i.e. Kafka Connect with JSON versus Debezium Server with key/value properties files. The Debezium Platform also uses its own JSON-based format that differs slightly from Kafka Connect, which adds another layer of complexity.

To streamline the user experience with Debezium Platform, a new Smart Editor feature is available, which allows you to write or paste configurations from Kafka Connect or Debezium Server, and convert those to the Platform’s format, automatically (DBZ-8873 DBZ-8888).

Quarkus extension improvements

One of the most recent additions to Debezium’s portfolio is the Debezium Quarkus extension, which enables developers to utilize Debezium’s CDC capabilities directly within a Quarkus-based application in either JVM or Native builds. We’ve added several new improvements, so let’s cover each of those separately.

Heartbeat listeners

Debezium can be configured with heartbeat.interval.ms to make Debezium emit a heartbeat event into the event stream periodically. Heartbeat events can be used for a variety of reasons, but their main use is to make sure that offsets remain aligned with the current read state on the source database, even during periods of low/no activity for captured tables.

A Quarkus application using the Debezium extension can also perform additional actions when a heartbeat is emitted by observing the CDI event DebeziumHeartbeat. This allows for application-specific code to be executed for each heartbeat (DBZ-8960).

In the following example, the listener merely writes to the console when a heartbeat is observed.

@ApplicationScoped
public class HeartbeatListener {
    public void consoleHeartbeat(@Observes DebeziumHeartbeat event) {
        System.out.println("Debezium emitted a heartbeat event!");
    }
}

Easily implement custom converters

A Debezium CustomConverter is a tool that can be used to change how Debezium emits the value for a given column type. This can be an extremely powerful for consumers that require data in a specific format.

Within your Quarkus application, you can use the traditional way to define a CustomConverter by specifying it as part of the connector configuration in your application.properties file; however, Debezium 3.3 adds a new annotation-based approach that simplifies configuration and makes iterative development much easier and faster (DBZ-8966).

As an example, the following showcases defining a converter that takes a given raw field and return’s its value converted as a String in the event using the Java toString() function.

@ApplicationScoped
class RawToStringConverter {
    @CustomConverter
    public ConverterDefinition<SchemaBuilder> bind(ConvertedField field) {
        return new ConverterDefinition<>(SchemaBuilder.string(), Object::toString);
    }
}

In addition, custom converters can be selectively applied to fields. Developers may be inclined to implement this behavior in the custom converter annotated method, and while that’s valid, this can be done in a component-driven way using a FieldFilteringStrategy.

As an example, the following showcases the same custom converter, but uses the field filtering strategy:

@ApplicationScoped
class RawToStringConverter {
    @CustomConverter(filter = CustomFieldFilteringStrategy.class)
    public ConverterDefinition<SchemaBuilder> bind(ConvertedField field) {
        return new ConverterDefinition<>(SchemaBuilder.string(), Object::toString);
    }
}

@ApplicationScoped
class CustomFieldFilteringStrategy implements FieldFilteringStrategy {
    @Override
    public boolean filter(ConvertedField field) {
        /* implement your selective logic here */
        return false;
    }
}

CDC events as POJOs

The Debezium Quarkus extension emits change events as structured ChangeEvent<K, V> types. But in some situations, it may be useful to transform the change event into a Java record or a domain-specific POJO. For these cases, Debezium’s Quarkus extension provides a standardized way to deserialize a ChangeEvent into your Java record or POJO.

As an example, let’s assume we have a products table in our source where we have a capturing handler defined. In this handler, we’d like to process Product objects, as seen below.

@ApplicationScoped
class ProductHandler {
    @Capturing(destination = "prefix.inventory.products")
    public void captureProducts(Product product) {
        /* do something with product */
    }
}

For this to work, a Deserializer<T> needs to be implemented to convert the ChangeEvent into our target POJO type Product. The extension provides an implementation to handle this using Jackson, ObjectMapperDeserializer.

public class ProductDeserializer extends ObjectMapperDeserializer<Product> {
    public ProductDeserializer() {
        super(Product.class);
    }
}

The last step involves defining the deserializing mapping in the Quarkus configuration to glue it all together.

quarkus.debezium.capturing.product.destination=prefix.inventory.products
quarkus.debezium.capturing.product.deserializer=<the-java-package>.ProductDeserializer

The extension will use this configuration to map an event destined for the prefix.inventory.products topic to use the product deserializer. This deserializer is mapped to the <the-java-package>.ProductDeserializer class, which uses Jackson to convert the ChangeEvent into a Product object. After the conversion, the @Capturing annotated method is called with the Product object instead of the emitted ChangeEvent.

Other changes

  • Execute Debezium in oracle readonly replica DBZ-8319

  • Fine-grained logging configuration DBZ-8638

  • Debezium Engine Quarkus Extension: introduce PostProcessor handler DBZ-8965

  • During a mining session treat ORA-00310 redo logs being inconsistent DBZ-8870

  • Add JMX metrics/statistics for cached events DBZ-8991

  • Create a showcase example for Openlineage DBZ-9058

  • '||' in ORACLE NVARCHAR data will cause exception DBZ-9132

  • [ORACLE] DDL parsing failed DBZ-9172

  • Remove heartbeat creation from configuration in favor of HeartbeatFactory DBZ-9176

  • When using non-recovery snapshot modes, offsets are not reset DBZ-9208

  • Validation for Log Position for SqlServer can fail DBZ-9212

  • Possible regression with throwing DebeziumException rather than warning DBZ-9217

  • Double event publishing via NATS Jetstream sink DBZ-9221

  • NullPointerException is thrown because DebeziumHeaderProducer is not registered DBZ-9225

  • MongoDB ExtractNewDocumentState SMT crash with nested struct in array in 3.2 DBZ-9231

  • Mongodb incremental snapshot is not honoring additional conditions DBZ-9232

  • WithClause inside an INSERT statement throws DDL parser exception DBZ-9233

  • Oracle snapshot boundary mode does not have a field display name DBZ-9236

  • Request fix for muti-task CREATE TABLE collisions for jdbc postgres target causing task to crash DBZ-9237

  • Oracle split table partition does not support online mode DBZ-9238

  • Exceptionally large mining windows can lead unintended metrics/performance issues DBZ-9241

  • Throw an exception on missing heartbeat table on Debezium-connector-postgres DBZ-9247

  • Allow Oracle heartbeat action query error handler to be resilient to ORA-02396 DBZ-9280

  • OpenLineage output dataset uses the wrong datatype DBZ-9285

  • Uppdate Informix JDBC Driver to 4.50.12 DBZ-9288

  • Debezium platform verify signal data collection fails DBZ-9290

  • Unchecked exception from OffsetStorageWriter.doFlush() in AsyncEmbeddedEngine leaves semaphore in OffsetStorageWriter unreleased and probably causes engine to fail DBZ-9292

  • Reselect post processor does not work with VariableScaleDecimal primary keys DBZ-9293

  • Duplicate key exception when using postgres connector based on pgoutput plugin DBZ-9305

  • Log all rows from LogMiner session logs during LogMiner failures DBZ-9322

In total, 63 issues were resolved in Debezium 3.3.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.

×