It’s my pleasure to announce the release of Debezium 1.1.0.Beta1!
This release adds support for transaction marker events, an incubating connector for the IBM Db2 database as well as a wide range of bug fixes. As the 1.1 release still is under active development, we’ve backported an asorted set of bug fixes to the 1.0 branch and released Debezium 1.0.1.Final, too.
At the time of writing this, not all connector archives have been synched to Maven Central yet; this should be the case within the next few others.
Transaction Markers
A common requirements for Debezium users is to materialize some kind of "aggregate" made up of change events from multiple change data topics.
Consider for instance an e-commerce application which manages purchase orders.
In a relational database, the information representing such order will typically be spread across multiple tables, e.g. PURCHASE_ORDER
, ORDER_LINE
and SHIPMENT_ADDRESS
.
To examine the complete state of a purchase order in the database itself, you’d run a JOIN query against the three tables.
Things get more challenging when looking at the change event topics produced by CDC connectors such as Debezium. For instance Kafka Streams could be used to join the change event streams for the three tables and materialize an aggregate view.
We’ve explored a possible implementation of this some time ago here on this blog. This approach was prone to exposing intermediary aggregates, though. The reason being that the joining logic had no insight into the transaction boundaries that applied in the source database. So when for instance inserting a purchase order with 10 order lines, it could happen that an aggregate gets materialized which contains the order and the first five lines, shortly followed thereafter by the complete aggregate view with all 10 order lines. Depending on your use case exposing such intermediary aggregates to downstream consumers (e.g. a search index) might not be desirable.
In Debezium 1.1 this situation can be addressed by leveraging the new transaction metadata topic supported by most connectors. When enabled, a separate topic will be published which contains events indicating the begin and end of transactions:
{
"status": "BEGIN",
"id": "571",
"event_count": null,
"data_collections": null
}
...
{
"status": "END",
"id": "571",
"event_count": "11",
"data_collections": [
{
"data_collection": "inventory.purchaseorder",
"event_count": "1"
},
{
"data_collection": "inventory.orderline",
"event_count": "10"
}
]
}
END
events contain the total number of change events originating from this transaction as well as the number of event per affected table.
Actual data change events in the topics for the purchase order and order line tables contain the transaction id, too.
These two things together enable a stream processing application to buffer all the change events originating from one transaction.
Only once it has received all the events of a transaction, it may produce the final aggregate view and publish it to downstream consumers,
avoiding the issue of exposing intermediary aggregate views.
Stay tuned for an in-depth example of such an implementation coming soon!
IBM Db2 Connector
Support for the IBM Db2 database has been on the wishlist for many Debezium users for quite some time. That’s why we were very excited when a group of IBM engineers reached out to us a while ago, offering to implement this connector under the Debezium umbrella. This connector is released in "incubating" state in Debezium 1.1.
Note that at this point a license for the IBM IIDR product is required in order to use the "ASN Capture" API leveraged by the connector. A post with more details around this connector should follow soon; in the mean time please refer to the connector reference documentation to learn more.
Further Changes and Bugfixes
Besides the transaction metadata topic and the Db2 connector, a few more improvements and fixes have been completed for Debezium 1.1 Beta1:
-
The message transformation for extracting the
after
state from change events allows to route change events to a specific topic based on a configurable record field (DBZ-1715) -
The
ExtractNewDocumentState
SMT to be used with the Debezium MongoDB connector will convertDate
andTimestamp
fields now into theorg.apache.kafka.connect.data.Timestamp
logical type, clarifying its semantics (DBZ-1717) -
The MySQL connector won’t log the dabase password in DEBUG level any longer (DBZ-1748)
-
The Antlr DDL parser of the MySQL connector handles the
TRANSACTIONAL
keyword of MariaDB (DBZ-1733) as well as theGET DIAGNOSTICS
statement (DBZ-1740) -
The Postgres connector can be used with proxied connections (DBZ-1738)
Overall, 27 issues were fixed for this release. 16 bugfixes from 1.1 Alpha1 and Beta1 were backported to 1.0.1.Final. Please make sure to read the upgrade notes when upgrading the Postgres connector and the accompanying decoderbufs logical decoding plugin to 1.1 Beta1, as a specific order of upgrading the two is needed.
Many thanks to Andrea Cosentino, Alexander Schwartz, Bingqin Zhou, Igor Gabaydulin, Jan Uyttenhove, Jos Huiting, Luis Garcés-Erice, Marc Zbyszynski, Peter Urbanetz and Sean Rooney for their contributions to this release!
On our road towards Debezium 1.1, we’ll likely do another Beta release before going to the candidate release phase in a few weeks from now. To see what’s coming, take a look at the roadmap, or get in touch to tell us about your specific feature requirements!
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.