We are very happy to announce the release of Debezium 1.5.0.Beta2!

The main features of this release is the new Debezium Signaling Table support, Vitess SET type support, and a continued focus to minor improvements, bugfixes, and polish as we sprint to the finish line for the 1.5 release.

Overall, the community fixed 54 issues since the Beta1 release, some of which we’ll explore more in-depth below.

Signaling Table

The signal table feature is a huge milestone for Debezium. It provides a command pattern based on a source database table to send commands (aka signals) to Debezium so that specific actions may be taken. The framework is extendable, allowing a connector to implement custom commands beyond the common commands provided by Debezium core. There are several situations where this might be applicable, including but not limited to, PostgreSQL primary key column changes, changes to enum value-sets of a column definition, and schema changes with the Oracle connector. This is an incubating feature and therefore behavior is subject to change between releases.

In order to use the signal table feature, the connector option signal.data.collection must be specified in the connector’s configuration. This option specifies the fully qualified name of the table from which signal requests will be sourced. If this option is not specified or empty, the signal table feature will be disabled.

The signal table itself must be created ahead of time and it must adhere to the following convention:

Column Name Data Type

ID

STRING
The unique identifier of the signal such as a UUID.

TYPE

STRING
The unique command to be performed.

DATA

STRING
The payload for the command.

Lastly, the signal table must be explicitly found as part of your connector’s include/exclude-list specifications. In other words, if you’re specifying a list of tables to monitor, this list will need to be adjusted to include the name of the signal table as well.

This new feature has quite a number of use cases that we intend to explore in future releases. Lets discuss what signals are currently supported in this release.

Log Signal

The log signal is detected when the TYPE column in the signal table is log. This signal requests that Debezium write the contents of the DATA column (payload) to the connector logs as-is. This can be useful for a variety of purposes from debugging to tracking progress of database script operations and much more.

As an example:

INSERT INTO DEBEZIUM_SIGNALS (ID, TYPE, DATA) VALUES ('1', 'log', 'Hello World');

Once that insert is committed and written to the database’s transaction logs, Debezium will recognize the signal and will then write Hello World to the connector logs using the INFO log level.

Schema Changes Signal

The schema changes signal is detected when the TYPE column in the signal table is schema-changes. This signal tells Debezium to emit a SchemaChangeEvent to the schema changes topic that is based on the changes supplied in the row’s DATA column (payload). The format of the DATA column must be given in JSON and an example of the format is below. Additionally, this signal will also have Debezium update it’s in-memory representation of the table’s schema structure.

As an example, we have a PostgreSQL table s1.a where we want to add a new column aa:

{
  "database": "postgres",
  "changes": [{
    "type": "ALTER",
    "id": "s1.a",
    "table": {
      "defaultCharsetName": null,
      "primaryKeyColumnNames": [ "pk" ],
      "columns": [{
        "name": "pk",
        "jdbcType": 4,
        "nativeType": 23,
        "typeName": "serial",
        "typeExpression": "serial",
        "charsetName": null,
        "length": 10,
        "scale": 0,
        "position": 1,
        "optional": false,
        "autoIncremented": true,
        "generated": false
      }, {
        "name": "aa",
        "jdbcType": 4,
        "nativeType": 23,
        "typeName": "int4",
        "typeExpression": "int4",
        "charsetName": null,
        "length": 10,
        "scale": 0,
        "position": 2,
        "optional": true,
        "autoIncremented": false,
        "generated": false
      }]
    }
  }]
}

With this JSON payload, the signal would be inserted as:

INSERT INTO DEBEZIUM_SIGNALS (ID, TYPE, DATA) VALUES ('1', 'schema-changes', <json-payload-string>);

Vitess SET support

The Vitess team improved the SET data type support in the VStream API as part of Vitess 9.0. This improvement has lead to the SET data type now being supported by the Debezium Vitess connector. This data type will be emitted as an EnumSet that will now contain all the permissible values of the column’s SET definition.

Other Features and Fixes

Besides the Signal Table and Vitess SET support, a few other improvements and fixes found their way into this release.

  • The Debezium connector for Oracle now uses the LogMiner-based capturing implementation by default. In order to use the XStream-based implementation, the database.connection.adapter option must be explicitly set to xstream (DBZ-3241).

  • In an earlier release of Debezium 1.5, the Oracle connector began to emit NUMBER(1) data types as BOOLEAN. Rather than this conversion be implicitly done by the connector, this behavior has been moved to an OOTB converter, NumberOneToBooleanConverter, that can be used as needed (DBZ-3208).

  • System generated index-organized tables (tables that begin with SYS_IOT_OVER) are ignored by the Oracle connector (DBZ-3036)

  • Debezium Server’s sink for AWS Kinesis can be configured with an endpoint by specifying debezium.sink.kinesis.endpoint (DBZ-3246).

As always, you can find the complete list of all the addressed issues and upgrade procedures in the release notes.

Many thanks to all the community members contributing to this release: Bingqin Zhou, David Seapy, Victar Malinouski, Xiao Fu, Kewei Shang, Martín Pérez, Hoa Le, Vladimir Osin, Martín Pérez, and Meng Qiu!

What’s Next?

Slowly wrapping up the work on the Debezium 1.5 release train, we’ve also taken the opportunity and integrated the Debezium Oracle into the main debezium source code repository. With that, all connectors of the former debezium-incubator respository have either been moved into their own, dedicated repository, or integrated into the main one. The incubator repository has been set to "Archived" mode, allowing to examine its history if needed.

For the remaining time until 1.5 Final, we’re planning to focus on bug fixes, performance improvements, documentation adjustments and other stabilization efforts; barring any unforeseen issues, the LogMiner-based capture implementation will be promoted from Incubating to Stable state for the Final release, too. If things go as planned, there’ll be a CR (candidate release) mid next week, followed by the final release around the end of the month.

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.