I am happy to announce the second maintenance release of the Debezium 2.7 release stream, 2.7.2.Final. This maintenance release focuses on addressing stability and regressions, with some improvements such as support for DECIMAL(p) data types with Informix and fixing a regression with the JsonConverter and the TruncateColumn handler.

Let’s dive into these changes…​

Breaking changes

The team aims to avoid any potential breaking changes between minor releases; however, such changes are sometimes inevitable.

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).

SQL Server

Normal JMX metrics are registered with a taskId attribute because SQL Server supports spawning a unique task per database mapping. Unfortunately, the JMX signal channel did not honor this and lead to some issues starting the JMX signal channel for each task. This has been fixed and the JMX signal channel’s MBean name now uses the taskId in its name to uniquely identify a signal channel for each database task when using a single connector to stream changes from multiple SQL Server databases (DBZ-8137).

New features and improvements

Debezium 2.7.2.Final also introduces a couple improvements and features, lets take a look at each individually.

Informix DECIMAL(p) data type support

The DECIMAL data type has seen a few improvements with Debezium for Informix. The data type can be defined with or without precision. When defined without precision, the connector interprets the column’s type as DECIMAL(16). However, users can also define columns as DECIMAL(p,s), where p is a precision up to 32 and s is the variable scale.

In order to represent the upper-bounds of such decimal numbers using PRECISE decimal handling mode, the Informix connector had to be adjusted to take advantage of the Debezium VariableScaleDecimal semantic type, which enables Debezium to retain full precision of such floating point numbers when serialized in a Kafka message (DBZ-8114).

For more information on how DECIMAL(p,s) data types are serialized, please see the Informix documentation.

Fixes and stability improvements

Debezium 2.7.2.Final introduces several bug fixes and stability improvements.

Column truncation now respected by JsonConverter

In earlier versions of Debezium, the column.truncate.* functionality returned a sliced ByteBuffer based on the truncation configuration. While this worked when using Avro, the truncation was not honored if your connector configuration used the JsonConverter as it operated on the entire underlying array rather than the given slice.

With Debezium 2.7.2.Final, this has been fixed and the column truncation logic now explicitly creates a ByteBuffer based on a new array. This change allows the JsonConverter to respect the truncated column value during the serialization to Kafka (DBZ-8189).

Snapshot mode schema_only_recovery has returned

It was also reported that the schema_only_recovery snapshot mode was not functioning in Debezium 2.7. We have identified the issue and the schema-only recovery snapshot mode has returned as a viable choice for Debezium 2.7 once again.

A gentle reminder that schema_only_recovery and schema_only were deprecated in Debezium 2.7 and have been removed in Debezium 3.0. Please be sure to use the recovery or no_data snapshot modes respectively moving forward when upgrading from Debezium 2.7 to 3.0.

MariaDB missing from Debezium Server

Some of you may have noticed that with the addition of the new official MariaDB connector in Debezium 2.7, that the connector was absent from the Debezium Server distribution. You’ll be happy to know that the Debezium 2.7.2.Final builds will now contain the MariaDB connector in the distribution, so you will no longer need to manually add it. (DBZ-8186).

Oracle VECSYS tablespace marked as built-in

If you’ve attempted to use Debezium with any of the latest Oracle 23ai images, you will have noticed a new tablespace has emerged called VECSYS. This tablespace includes a variety of system tables for storing metadata about the new Oracle 23 vector database and search features.

We have found that the DBMS_METADATA.GET_DDL function used to capture the table’s DDL structure is not compatible with several database objects in the VECSYS tablespace and will lead to connector issues if you don’t explicitly set schema.history.internal.store.only.captured.tables.ddl to true. To address this problem, the tablespace is marked as built-in, meaning it is no longer eligible for capture (DBZ-8198).

Other fixes

  • Incremental snapshots don’t work with CloudEvent converter (DBZ-7601)

  • Incremental snapshot fails with NPE if surrogate key doesn’t exist (DBZ-7797)

  • Postgres connector - null value processing for "money" type column. (DBZ-8027)

  • Using snapshot.include.collection.list with Oracle raises NullPointerException (DBZ-8032)

  • Log additional details about abandoned transactions (DBZ-8044)

  • Performance degradation when reconstructing (log.mining.stragtegy hybrid mode) (DBZ-8071)

  • Documentation for signals provides incorrect data-collection format for some connectors (DBZ-8090)

  • ORA-00600: internal error code, arguments: [krvrdGetUID:2], [18446744073709551614], [], [], [], [], [], [], [], [], [], [] (DBZ-8125)

  • ConvertingFailureIT#shouldFailConversionTimeTypeWithConnectModeWhenFailMode fails randomly (DBZ-8128)

  • Unpredicatable ordering of table rows during insertion causing foreign key error (DBZ-8130)

  • schema_only crashes ibmi Connector (DBZ-8131)

  • Support larger database.server.id values [DBZ-8134](https://issues.redhat.com/browse/DBZ-8134)

  • Add LogMiner start mining session retry attempt counter to logs (DBZ-8143)

  • Open redo thread consistency check can lead to ORA-01291 - missing logfile (DBZ-8144)

  • Reduce logging verbosity of XStream DML event data (DBZ-8148)

  • When stopping the Oracle rac node the Debezium server throws an expectations - ORA-12514: Cannot connect to database and retries (DBZ-8149)

  • Issue with Debezium Snapshot: DateTimeParseException with plugin pgoutput (DBZ-8150)

  • JDBC connector validation fails when using record_value with no primary.key.fields (DBZ-8151)

  • Taking RAC node offline and back online can lead to thread inconsistency (DBZ-8162)

  • Postgres JSONB Fields are not supported with Reselect Post Processor (DBZ-8168)

  • NullPointerException (schemaUpdateCache is null) when restarting Oracle engine (DBZ-8187)

  • XStream may fail to attach on retry if previous attempt failed (DBZ-8188)

  • Reduce log verbosity of OpenLogReplicator SCN confirmation (DBZ-8201)

In total there were 31 issues resolved in Debezium 2.7.2.Final. The list of changes can also be found in our release notes.

A big thank you to all the contributors from the community who worked diligently on this release: Ashish Binu, Chris Cranford, Jiri Pechanec, Jochen Schalanda, Lars M. Johansson, Mario Fiore Vitale, Minh Son Nguyen, Mohamed El Shaer, moyq5, Robert Roldan, Sergei Kazakov, Thomas Thornton, and Vojtech Juranek!

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.