With great happiness I’m announcing the release of Debezium 1.2.0.Beta1!
This release brings user-facing schema change topics for the SQL Server, Db2 and Oracle connectors, a new message transformation for content-based change event routing, support for a range of array column types in Postgres and much more. We also upgraded the Debezium container images for Apache Kafka and Kafka Connect to version 2.5.0.
As it’s the answer to all questions in life, the number of issues fixed for this release is exactly 42!
Schema Change Topics
Similarly to the MySQL connector, also the Debezium connectors for SQL Server, Db2 and Oracle optionally expose a public topic with schema change information now. Unlike the connector-internal history topic, the schema change topics are meant for consumption by interested clients and for instance allow to seed the schema of downstream data stores.
Schema information is propagated in a typed structure, sparing consumers from having to parse database specific DDL statements. As an example, here’s a schema event for the creation of the customers
table from the Debezium tutorial, as produced by the SQL Server connector and using the JSON message converter:
{
"source": {
"version": "1.2.0.Beta1",
"connector": "sqlserver",
"name": "server1",
"ts_ms": 1588252618953,
"snapshot": "true",
"db": "testDB",
"schema": "dbo",
"table": "customers",
"change_lsn": null,
"commit_lsn": "00000025:00000d98:00a2",
"event_serial_no": null
},
"databaseName": "testDB",
"schemaName": "dbo",
"ddl": null,
"tableChanges": [
{
"type": "CREATE",
"id": "\"testDB\".\"dbo\".\"customers\"",
"table": {
"defaultCharsetName": null,
"primaryKeyColumnNames": [
"id"
],
"columns": [
{
"name": "id",
"jdbcType": 4,
"nativeType": null,
"typeName": "int identity",
"typeExpression": "int identity",
"charsetName": null,
"length": 10,
"scale": 0,
"position": 1,
"optional": false,
"autoIncremented": false,
"generated": false
},
{
"name": "first_name",
"jdbcType": 12,
"nativeType": null,
"typeName": "varchar",
"typeExpression": "varchar",
"charsetName": null,
"length": 255,
"scale": null,
"position": 2,
"optional": false,
"autoIncremented": false,
"generated": false
},
{
"name": "last_name",
"jdbcType": 12,
"nativeType": null,
"typeName": "varchar",
"typeExpression": "varchar",
"charsetName": null,
"length": 255,
"scale": null,
"position": 3,
"optional": false,
"autoIncremented": false,
"generated": false
},
{
"name": "email",
"jdbcType": 12,
"nativeType": null,
"typeName": "varchar",
"typeExpression": "varchar",
"charsetName": null,
"length": 255,
"scale": null,
"position": 4,
"optional": false,
"autoIncremented": false,
"generated": false
}
]
}
}
]
}
To learn more about schema change topics, please refer to the connector-specific documentation, e.g. for SQL Server. Note that this feature currently is in incubation state, which means that e.g. details of the event format may change in future versions, based on the feedback we receive from the community.
Content-Based Topic Routing
Using the new single message transformation (SMT) for content-based topic routing you can control the topics specific change events get sent to based on their column values. As an example consider a database table purchase_orders
which contains orders of two kinds, B2B and B2C. Their change events should be sent to a distinct topic for each kind in Apache Kafka. For that, simple script expressions in languages such as JavaScript or Groovy can be leveraged:
...
transforms=route
transforms.route.type=io.debezium.transforms.Filter
transforms.route.language=jsr223.graal.js
transforms.route.topic.expression=value.after.ordertype == 'B2C' ? 'b2c_orders' : 'b2b_orders'
...
Further Changes and Bugfixes
Some other features and fixes of this release include:
-
Support for Postgres
JSON
,JSONB
,TIME
andTIMESTAMP
array columns (DBZ-1969, DBZ-1990) -
Column whitelists for the Postgres connector, which comes in handy if you’re interested in only capturing a small subset of table columns (DBZ-1962)
-
MySQL’s
FLUSH TABLE
statement is handled correctly (DBZ-2047) -
Unique namespaces are used in routed outbox events (DBZ-1963)
-
Fixed a potential value overflow in Postgres
BIT VARYING
columns (DBZ-1949) -
Support for the
eventType
field has been removed from the outbox routing SMT (DBZ-2014); if needed, please configure this field explicitly as header or message value attribute; this was done to allow for exporting this field using any custom name which was not easily possible before -
Improved start-up performance for the Postgres connector when using enum columns (DBZ-2038)
Please refer to the release notes for the list of all addressed issues and upgrading procedures.
Many thanks to all the community members contributing to this release: Anders Engström, Anton Kondratev, Bingqin Zhou, Braden Groom, David Jerome, Hossein Torabi, Ivan Klass Jan-Hendrik Dolling, John Psoroulas, Jos Huiting, Joy Gao lyidataminr, and Siddhant Agnihotry!
Gunnar Morling
Gunnar is a software engineer at Decodable and an open-source enthusiast by heart. He has been the project lead of Debezium over many years. Gunnar has created open-source projects like kcctl, JfrUnit, and MapStruct, and is the spec lead for Bean Validation 2.0 (JSR 380). He’s based in Hamburg, Germany.
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.