As the summer nears, I’m excited to announce the release of Debezium 1.9.3.Final!
This release primarily focuses on bugfixes and stability; however, there are some notable feature enhancements. Lets take a moment to cool off and "dive" into these new features in a bit of detail :).
Decoding binary payloads with MongoDB and Outbox
Databases have had support for storing binary data since the beginning, and there are a number of reasons that applications favor using a database for this over other alternatives. Binary data doesn’t always have to be information like images, thumbnails, or binary documents like PDFs, it can also include serialized objects too.
When using the outbox pattern with MongoDB and Avro serialization, the outbox’s payload field is the only value that is emitted in the message’s value. When the payload consists of binary data, as shown below, it is emitted as-is and can lead to some serialization issues with consumers of the topic.
{
"payload": {
"$binary": ""RGViZXppdW0gaXMgdGhlIGJlc3Qu",
"$type": "0"
}
}
In this release, we’ve deprecated the ByteBufferConverter
and we’ve introduced two new converters in its place:
-
ByteArrayConverter
: converts payload to be delivered as it is, abyte[]
. -
BinaryDataConverter
: converts payload to be delivered as either aByteBuffer
orByte[]
.
For this use case, we’re going to make use of the new ByteArrayConverter
as we want to emit this payload’s value as the raw byte[]
. In the connector configuration, the value.converter
configuration option must be set to handle this use case.
transforms=outbox,...
transforms.outbox.type=io.debezium.connector.monogdb.transforms.outbox.MongoEventRouter
value.converter=io.debezium.converters.ByteArrayConverter
Now the data will be emitted to the broker as-is, as a byte-array that can be safely consumed by consumers. Thanks to Nathan Bradshaw for this excellent contribution!
Heartbeat action queries with Oracle
The heartbeat action query is a feature that allows a Debezium connector to write records to the source database and to capture those records during the event processing loop. This was first introduced for PostgreSQL to deal with situations when captured tables change less frequently than other non-captured tables, causing unintended WAL growth.
For Oracle using the LogMiner implementation, a similar problem occurs that impacts the connector’s ability to restart due to the offset SCN not being advanced on a regular interval due to changes being made to other tables that are not captured, whether they are part of the same pluggable database or another.
To enable heartbeat action queries, the connector must be configured with:
heartbeat.interval.ms=1000
heartbeat.action.query=INSERT INTO heartbeat (id) values (SYSDATE)
table.include.list=MYSCHEMA.HEARTBEAT,...
The heartbeat functionality must first be enabled by specifying the heartbeat.interval.ms
. This controls how often the connector generates heartbeat events. If this value is not greater-than 0
, then heartbeats are disabled.
Next, to specifically use the action query feature, the heartbeat.action.query
option must be given. This specifies a SQL statement that will be executed on each heartbeat interval. This statement can be either an INSERT
or an UPDATE
, as long as the resulting SQL operation generates a row change.
Finally, the action query must operate on a table that is included in the connector’s filter configuration. Like any other captured table, the table must also be configured with the correct supplemental logging so that the event is captured.
With this configuration in place, and assuming no long-running transaction, the offset SCN will advance on each heartbeat.
Oracle LogMiner session duration is now controllable
The Debezium Oracle connector’s LogMiner session duration has always been based on how often the redo log switches. Generally, this behavior has worked well for most environments; however, in low traffic environments or during off-peak hours, this has the potential to re-use the same LogMiner session for a longer period of time that can lead to ORA-04030 exceptions when the PGA memory is exhausted.
A new configuration option has been added, log.mining.session.max.ms
, allowing full control over the maximum duration of an Oracle LogMiner session. When set to a value greater-than 0
, the connector will automatically close and restart the mining session if the maximum duration is reached or a log switch occurs, whichever comes first. Oracle environments with low volume, particularly during off-peak hours, should no longer notice any PGA memory concerns when enabling this new option. Coordinate with your database administrator team to determine the best value based on your environment’s configuration and activity.
Further Changes
But that’s not all, there were also a number of bugfixes that are noteworthy, including but not limited to:
-
MySQL connector increment snapshot failed parse datetime column lenth when connector set "snapshot.fetch.size": 20000 DBZ-4939
-
InstanceAlreadyExistsException during MongoDb connector metrics registration DBZ-5011
-
DateTimeParseException: Text 'infinity' could not be parsed in Postgres connector DBZ-5014
-
4 Connections per connector (postgres) DBZ-5074
-
Oracle Logminer: records missed during switch from snapshot to streaming mode DBZ-5085
-
Cannot Set debezium.sink.kafka.producer.ssl.endpoint.identification.algorithm to empty value DBZ-5105
-
MilliSecondsBehindSource is not reported by SQL Server connector DBZ-5137
-
ExtractNewRecordState SMT Replaces Null Value with Column’s Default Value DBZ-5166
-
Oracle connector metrics tracking of rollback and abandoned transactions may cause high memory usage DBZ-5179
We strongly recommend upgrading to 1.9.3.Final to get the latest improvements both in performance and stability.
Overall, 47 issues were fixed in this release. Please refer to the release notes to learn more about all fixed bugs, update procedures, etc.
Many thanks to the following individuals from the community which contributed to Debezium 1.9.3.Final: Alexey Miroshnikov, Anisha Mohanty, Bob Roldan, Chris Cranford, Connor Szczepaniak, César Martínez, Harvey Yue, Himanshu Mishra, Jakub Cechacek, Jiri Pechanec, Mark Allanson, Mark Bereznitsky, Nathan Bradshaw, Sergei Morozov, Stefan Miklosovic, Vadzim Ramanenka, and Vojtech Juranek!
Outlook
The Debezium 1.9 release stream will remain the current long-running version for the next three months. During this time, we will continue to evaluate user reports and do micro-releases to address bugs and regressions depending on severity.
Also, quite a lot of work has gone into Debezium 2.0. We intend to release Debezium 2.0.0.Alpha2 next week, will releases to follow about every three weeks thereafter.
Stay tuned for more in the coming weeks and stay cool out there!
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.