It’s my pleasure to announce the first release of the Debezium 1.8 series, 1.8.0.Alpha1!
With the colors of Autumn upon us, the team has been hard at work painting lines of code for this release. With Debezium 1.8.0.Alpha1 comes quite a number of improvements but most notably is the new native MongoDB 4.0 change streams support!
MongoDB Change Streams Support
MonogoDB change streams allow an application or client access to real-time change data capture events without the complexity of tailing the oplog. This functionality was first introduced by the MongoDB engine in version 3.6; however the functionality was limited. Starting with MongoDB 4.0, change streams now captures changes across a database, replica set, or even a sharded cluster.
Debezium added change streams support to:
-
Enable compatibility with MongoDB 5 (not yet tested, see future work below).
-
Provide full document output in update events (see below).
-
Abstract from internal (and potentially changing) specifics of the oplog format, making this new implementation a potential replacement for oplog reading in the future.
In order to use change stream support with Debezium, you must use MongoDB 4.0 or later.
Enablement
Debezium for MongoDB exposes a new configuration property called capture.mode
. The capture mode specifies how the connector should obtain change events from the MongoDB database. The valid options are:
oplog
-
Specifies that changes should be captured by tailing the oplog; this is the legacy behavior.
change_streams
-
Specifies that changes should be captured by using MongoDB change streams. Updates will not contain the full message; only changed fields are part of the event.
change_streams_update_full
-
Specifies that changes should be captured by using MongoDB change streams. Updates will contain a full snapshot of the current record in the event. This is the new default for the connector.
The new |
Event changes
Using our tutorial from our examples repository, lets take a look at the differences in these capture modes.
First, lets add a new record to our customers
collection. Using the MongoDB shell, this can be done by running the following command:
db.customers.insert([
{
_id : NumberLong("1005"),
first_name : 'Bob',
last_name : 'Hopper',
email : 'thebob@example.com',
unique_id : UUID() }
]);
This will generate a change event but as you’ll see if you inspect the topic, the contents of the event are not all that different in this release. Since the event is an insert, all field values provided in the emitted event.
During updates, this is where we can see the capture mode differences in action. Now modify the customer’s first and last name using the MongoDB shell with this command:
db.customers.update(
{ _id:NumberLong("1005") },
{ $set: {
first_name: "Bobby",
last_name: "Copper"
} });
This modifies the first and last name of the customer with id 1005
. The following sections show what each event will look like for the given capture mode.
Oplog
The following shows a snippet of an update event when using the oplog
capture mode.
{
"schema": {
...
},
"payload": {
"after": null,
"patch": "{\"$v\": 1, \"$set\": { \"first_name\": \"Bobby\", \"last_name\": \"Copper\"}}",
"filter": "{\"_id\": {\"$numberLong\": \"1005\"}}",
"updateDescription": null,
"source": {
"version": "1.8.0.Alpha1",
"connector": "mongodb",
"name": "dbserver1",
"ts_ms": 1635291250000,
"snapshot": "false",
"db": "inventory",
"sequence": null,
"rs": "rs0",
"collection": "customers",
"ord": 1,
"h": 3510217852938498600,
"tord": null,
"stxnid": null
},
"op": "u",
"ts_ms": 1635291250803,
"transaction": null
}
}
The emitted event’s after field has no value. Instead, the event provides values for patch and filter that describe limited details about what changed in the source document. Since the event only provides details about what fields have changed and not the values for unchanged fields, this may not be ideal for certain consumers that require knowledge of the full document.
Change Streams
The following shows a snippet of an update event when using the change_streams
capture mode.
{
"schema": {
...
},
"payload": {
"after": null,
"patch": null,
"filter": null,
"updateDescription": {
"removedFields": null,
"updatedFields": "{\"first_name\": \"Bobby\", \"last_name\": \"Copper\"}",
"truncatedArrays": null
},
"source": {
"version": "1.8.0.Alpha1",
"connector": "mongodb",
"name": "dbserver1",
"ts_ms": 1635292448000,
"snapshot": "false",
"db": "inventory",
"sequence": null,
"rs": "rs0",
"collection": "customers",
"ord": 1,
"h": null,
"tord": null,
"stxnid": null
},
"op": "u",
"ts_ms": 1635292448736,
"transaction": null
}
}
The emitted event has a slightly different set of values than the legacy oplog output. As shown above, the event does not have a value in the after, patch, or filter fields. Instead, the event relies on describing the changes to the document’s fields in the updateDescription
structure. While this provides a bit more detail about values that may have been set and even unset due to an update, this may still not be ideal for some consumers that need values for all fields of the source document.
Change Streams Full Document
The following shows a snippet of an update event when using the change_streams_update_full
capture mode.
{
"schema": {
...
},
"payload": {
"after": "{\"_id\": {\"$numberLong\": \"1005\"},\"first_name\": \"Bobby\",\"last_name\": \"Copper\",\"email\": \"thebob@example.com\",\"unique_id\": {\"$binary\": \"KRywzYp5RneNu8DUmhQHAQ==\",\"$type\": \"04\"}}",
"patch": null,
"filter": null,
"updateDescription": {
"removedFields": null,
"updatedFields": "{\"first_name\": \"Bobby\", \"last_name\": \"Copper\"}",
"truncatedArrays": null
},
"source": {
"version": "1.8.0.Alpha1",
"connector": "mongodb",
"name": "dbserver1",
"ts_ms": 1635292878000,
"snapshot": "false",
"db": "inventory",
"sequence": null,
"rs": "rs0",
"collection": "customers",
"ord": 1,
"h": null,
"tord": null,
"stxnid": null
},
"op": "u",
"ts_ms": 1635292878244,
"transaction": null
}
}
This capture mode is nearly identical to the change_streams
mode except with one critical difference, the after
field is populated with a complete snapshot of document. This mode is great for consumers that rely on having all fields in the source document.
Please see the MongoDB documentation for more details on full document mode semantics.
The full document mode is based on a re-selection of the source document when MongoDB provides the change event over the change stream to the connector. In cases where multiple changes to the same document happen within close proximity of one another, each event may have the same full document representation. |
Future work
In conjunction to the work already done with MongoDB change streams, we recognize there is much work that remains which includes testing the new change streams implementations against MongoDB 5 and updating the connector documentation to reflect these new changes. You can expect this and much more as a part of the next preview release. As per the updated Debezium 1.8 roadmap, we’re also planning to add support for incremental snapshots to the Debezium connector for MongoDB, as well as a variant of the outbox event router which supports the event format of this connector.
Other Fixes
There were quite a number of bugfixes and stability changes in this release, some noteworthy are:
-
Row hashing in LogMiner Query not able to differentiate between rows of a statement (DBZ-3834)
-
The chunk select statement is incorrect for combined primary key in incremental snapshot (DBZ-3860)
-
column.the mask.hash.hashAlgorithm.with…. data corruption occurs when using this feature (DBZ-4033)
-
Infinispan SPI throws NPE with more than one connector configured to the same Oracle database (DBZ-4064)
Altogether, 82 issues were fixed for this release. A big thank you to all the contributors from the community who worked on this release: Christopher Burch, Claus Guttesen, Fabian Martinez, Guy Korland, Harvey Yue, Hossein Torabi, Juan Fiallo, Judah Rand, Laurent Broudoux, Plugaru Tudor, Sergei Morozov, Sungho Hwang, Ünal Sürmeli, Vivek Wassan, Willie Zhu, Zongwen Li, and 陆杰.
Outlook
As the end of the year is just around the corner, we intend to press forward with the same vigor. We have started an open discussion regarding Debezium 2.0 on the mailing list. Your feedback is invaluable so let us know what you’d like to see added, changed, or improved! In the meantime, you can also expect a minor bugfix release to the Debezium 1.7 series next week, as well as another preview release of the Debezium 1.8 series in a couple more weeks. Happy Streaming!
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.