I’m very happy to announce the release of Debezium 0.10.0.CR2!
After the CR1 release we decided to do another candidate release, as there was not only a good number of bug fixes coming in, but also a few very useful feature implementations were provided by the community, which we didn’t want to delay. So we adjusted the original plan a bit and now aim for Debezium 0.10 Final in the course of next week, barring any unforeseen regressions.
As usual, let’s take a closer look at some of the new features and resolved bugs.
Customizable Message Keys
Being able to configure which columns of a table should go into the key of corresponding Kafka messages has been a long-standing feature request (DBZ-1015). To recap, by default the message key of Debezium’s data change events will contain the primary key column(s) of the represented table. That’s a sensible default, but sometimes more flexibility is desirable:
-
Some tables don’t have a primary key, esp. in legacy data models
-
Choosing a different message key than the primary key may facilitate stream processing applications that operate on multiple change data topics
The second case is especially of interest when building Kafka Streams applications that join multiple CDC topics: in general, topic joins can only be done if the message key is the same on both sides of the join.
For instance let’s assume we have two tables, Customer
and CustomerDetails
with different primary keys and a foreign key relationship from CustomerDetails
to Customer
.
By choosing that foreign key column as the message key for customer detail change events,
the two table streams could be joined without the need for re-keying the customer details topic.
To do so, the new message.key.columns
option can be used like so:
message.key.columns=dbserver1.inventory.customerdetails:CustomerId
Customizable message keys are supported for all the relational Debezium connectors.
Pluggable Serializers for Cassandra
From previous announcements you might remember that Debezium’s Cassandra connector is a bit different from the other ones, as it’s not based on the Kafka Connect framework. As such, until now it didn’t support the notion of configurable message serializers; Avro was the only supported message format.
As part of the ongoing efforts to align the Cassandra connector more closely with the other ones, it now allows to configure different serializers, so you also could use JSON, ProtoBuf and other formats (DBZ-1405). The serializer framework from Kafka Connect is reused for that, so you can leverage all the existing serializers and configure them exactly the same way as done for any other connector.
Improved handling of Postgres TOAST Columns
TOAST columns are a mechanism in Postgres for dealing with column values that exceed the page size limit (typically 8 KB). While the usage of TOAST is transparent when interacting with the database itself, this is not the case when obtaining change events via logical decoding. As TOASTed values are not stored within the physical data row itself, logical decoding does not expose the value of unchanged TOAST columns (unless the column is part of the table’s replica identity).
This situation used to be handled in different ways by the the logical decoding plug-ins supported by Debezium (pgoutput
, decoderbufs
and wal2json
),
one approach being the retrieval of such column "out of bands".
Unfortunately, there’s no way for savely doing this when considering concurrent writes to such record.
So we reworked how TOAST columns are handled:
if a TOAST column’s value hasn’t changed and that column isn’t part of the table’s replica identity, its value will not be contained in UPDATE
or DELETE
events.
Instead, a configurable marker value will be exported in this case
(defaulting to __debezium_unavailable_value
).
This avoids the race conditions that were possible before, but of course raises the question how consumers should deal with this marker value. There are multiple possible answers to that:
-
The value could simply be ignored; for instance a consumer that writes change events to a database, may omit that column from the
UPDATE
statement it issues -
When not working with dynamic updates, a trigger may be installed in a sink database, that ignores any updates that would set a column value to the marker, keeping the previous value
-
When actually requiring complete change events including any TOAST column within the Kafka change data topic itself, a stateful Kafka Streams application could be built which hydrates incoming change events with the marker value based on the previous column value persisted in a state store
Thinking about it, the last approach might be an interesting topic for a future blog post :-)
Bugfixes and Other Changes
Besides these feature implementations, this release contains a number of bugfixes, too:
-
When using the
pgoutput
logical decoding plug-in for Postgres, custom publication names are supported (DBZ-1436) -
The Postgres connector will retry for a configurable period of time to obtain a replication slot, which can be helpful when rebalancing existing connectors in a cluster (DBZ-1426)
-
Reserved characters in column names can be replaced when using Avro as message format (DBZ-1044)
-
Default values without the time part for MySQL
DATETIME
columns are supported now (DBZ-1501) -
MySQL
CREATE DATABASE
andCREATE TABLE
statements with default character sets are supported (DBZ-1470)
Testing for MongoDB has been expanded to also cover version 4.2 (DBZ-1389), and the Postgres driver has been updated to the latest and greatest version 42.2.7 (DBZ-1462). We’re also happy to report that going forward, the Debezium container images are also available in the quay.io container registry (DBZ-1178).
Overall, not less than 30 issues were fixed in the 0.10 CR2 release. Please refer to the release notes for the complete list of addressed issues and the upgrading procedure.
This release wouldn’t be possible without all the fantastic people from the Debezium community who contributed: Bingqin Zhou, Ching Tsai, Guillaume Rosauro, Javier Holguera Jaromir Hamala, Josh Arenberg and Taylor Rolison.
Many thanks to all of you!
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.