It has been nearly two weeks since our last preview release of the Debezium 2.4 series, and I am thrilled to announcement the next installation of that series, Debezium 2.4.0.Beta2.
While typically beta releases focus on stability and bugs, this release includes quite a number of noteworthy improves and new features including a new ingestion method for Oracle using OpenLogReplicator, a new single message transform to handle timezone conversions, custom authentication support for MongoDB, configurable order for the MongoDB aggregation pipeline, and lastly support for MongoDB 7.
Let’s take a few moments and dive into all these new features, improvements, and changes in more detail.
Oracle ingestion using OpenLogReplicator
The Debezium for Oracle connector has traditionally shipped with two adapters, one for Oracle XStream and another to integrate directly with Oracle LogMiner. While each adapter has its own benefits and is quite mature with features and support for a wide array of data types and use cases, we wanted to explore a completely different way of capturing changes.
Debezium 2.4.0.Beta2 introduces a new, experimental Oracle ingestion adapter based on OpenLogReplicator. The adapter integrates directly with the OpenLogReplicator process in order to create change events in a similar way that the XStream implementation acts as a client to Oracle GoldenGate.
OpenLogReplicator is a standalone process that must either run on the Oracle database server or can run independently of the database server but requires direct communication with the database via TCP/IP and have direct read access to the Oracle redo and archive log files. OpenLogReplicator also does not ship with any pre-built binaries, so the code must either be built directly from source or deployed in a container image that can access the database and its files remotely via file shares.
Once OpenLogReplicator is installed, set up requires the following steps:
-
Configure the OpenLogReplicator’s configuration,
OpenLogReplicator.json
. -
Configure the Oracle connector to use the OpenLogReplicator adapter.
At this time, the Debezium for Oracle connector expects the OpenLogReplicator configuration to use very specific settings so that the data is transferred to the connector using the right serialization. The example configuration shows the critical configuration parameters that must be set for Debezium to ingest the data properly.
When OpenLogReplicator is configured, you should see OpenLogReplicator start with the following:
OpenLogReplicator v1.2.1 (C) 2018-2023 by Adam Leszczynski (aleszczynski@bersler.com), see LICENSE file for licensing information, arch: x86_64, system: Linux, release: 6.4.11-200.fc38.x86_64, build: Debug, modules: OCI Probobuf
adding source: ORACLE (1)
adding target: DBZ-NETWORK (2)
writer is starting with Network:0.0.0.0:9000 (3)
1 | The source alias configured in OpenLogReplicator.json |
2 | The target alias configured in OpenLogReplicator.json |
3 | The host and port the OpenLogReplicator is listening on. |
Lastly to configure the connector, set the following connector configuration options:
{
"database.connection.adapter": "olr",
"openlogreplicator.source": "<source-alias>", (1)
"openlogreplicator.host": "<host>", (2)
"openlogreplicator.port": "<port>" (3)
1 | The source alias defined in the OpenLogReplicator.json configuration that is to be used. |
2 | The host that is running the OpenLogReplicator. |
3 | The port the OpenLogReplicator is listening on. |
When the connector starts and begins to stream, it will connect to the OpenLogReplicator process' network endpoint, negotiate the connection with the serialization process, and then will begin to receive redo log entries.
We will have another blog post that goes over OpenLogReplicator in more detail in the coming weeks leading up to the final release, but in the meantime feel free to experiment with the new ingestion method as we would love to hear your feedback.
As this ingestion method is experimental, there are a few known limitations, please review the connector documentation for details. |
New Timezone Transformation
A common request we have often heard from the community has been to emit temporal columns using other time zones besides UTC. Debezium has supported this by using a CustomConverter
to change the way temporal columns are emitted by default to writing your own single message transformation; however, these approaches may not be for everyone.
Debezium 2.4 now ships with a brand-new time zone transformation that enables you to control, to a granular level, which temporal columns in an emitted event will be converted from UTC into whatever desired time zone your pipeline requires. To get started with this new transformation, add the following basic configuration to your connector:
{
"transforms": "tz",
"transforms.tz.type": "io.debezium.transforms.TimezoneConverter",
"transforms.tz.converted.timezone": "America/New_York"
}
By specifying the above configuration, all temporal columns that are emitted in UTC will be converted from UTC to the America/New_York time zone. But you are not limited to just changing the timezone for all temporal fields, you can also target specific fields using the include.fields
property as shown below:
{
"transforms": "tz",
"transforms.tz.type": "io.debezium.transforms.TimezoneConverter",
"transforms.tz.converted.timezone": "America/New_York",
"transforms.tz.include.fields": "source:customers:created_at,customers:updated_at"
}
In the above example, the first entry will convert the created_at
field where the source table name is customers
whereas the latter will convert the updated_at
field where the topic name is customers
. Additionally, you can also exclude fields from the conversion using exclude.fields
to apply the conversion to all but a subset:
{
"transforms": "tz",
"transforms.tz.type": "io.debezium.transforms.TimezoneConverter",
"transforms.tz.converted.timezone": "America/New_York",
"transforms.tz.exclude.fields": "source:customers:updated_at"
}
In the above example, all temporal fields will be converted to the America/New_York time zone except where the source table name is customers
and the field is updated_at
.
You can find more information about this new transformation in the documentation and we would love to hear your feedback.
MongoDB changes
Debezium 2.4.0.Beta2 also ships with several MongoDB connector changes, lets take a look at those separately.
Breaking changes
The mongodb.hosts
and mongodb.members.autodiscover
configuration properties were removed and no have any influence on the MongoDB connector behavior. If you previously relied on these configuration properties, you must now use the MongoDB connection string configuration property moving forward (DBZ-6892).
Custom Authentication
In specific environments such as AWS, you need to use AWS IAM role-based authentication to connect to the MongoDB cluster; however, this requires setting the property u sing AWS_CREDENTIAL_PROVIDER
. This provider is responsible for creating a session and providing the credentials.
To integrate more seamlessly in such environments, a new configuration property, mongodb.authentication.class
has been added that allows you to define the credential provider class directly in the connector configuration. If you need to use such a provider configuration, you can now add the following to the connector configuration:
{
"mongodb.authentication.class": "<fully-qualified-class-name-to-use>",
"mongodb.user": "username",
"mongodb.password": "password"
}
In addition, if the authentication needs to use another database besides admin
, the connector configuration can also include the mongodb.authsource
property to control what authentication database should be used.
For more information, please see the documentation.
Configurable order of aggregation pipeline
Debezium 2.4 now provides a way to control the aggregation order of the change streams pipeline. This can be critical when specific documents are being aggregated that could lead to pipeline problems such as large documents.
By default, the connector applies the MongoDB internal pipeline filters and then any user-constructed filters; however this could lead to situations where large documents make it into the pipeline and MongoDB could throw an error if the document exceeds the internal 16Mb limit. In such use cases, the connector can now be configured to apply the user stages to the pipeline first defined by cursor.pipeline
to filter out such use cases to avoid the pipeline from failing due to the 16Mb limit.
To accomplish this, simply apply the following configuration to the connector:
{
"cursor.pipeline.order": "user_first",
"cursor.pipeline": "<custom-pipeline-filters>"
}
For more details, please see the documentation.
MongoDB 7 support
MongoDB 7.0 was released just last month and Debezium 2.4 ships with MongoDB 7 support.
If you are looking to upgrade to MongoDB 7 for your environment, you can easily do so as Debezium 2.4+ is fully compatible with the newer version. If you encounter any problems, please let us know.
Other fixes & improvements
There are several bugfixes and stability changes in this release, some noteworthy are:
-
Documentation content section in the debezium.io scroll over to the top header. DBZ-5942
-
Only publish deltas instead of full snapshots to reduce size of sync event messages DBZ-6458
-
Postgres - Incremental snapshot fails on tables with an enum type in the primary key DBZ-6481
-
schema.history.internal.store.only.captured.databases.ddl flag not considered while snapshot schema to history topic DBZ-6712
-
ExtractNewDocumentState for MongoDB ignore previous document state when handling delete event’s with REWRITE DBZ-6725
-
MongoDB New Document State Extraction: original name overriding does not work DBZ-6773
-
Error with propagation source column name DBZ-6831
-
Support truncating large columns DBZ-6844
-
Always reset VStream grpc channel when max size is exceeded DBZ-6852
-
Kafka offset store fails with NPE DBZ-6853
-
JDBC Offset storage - configuration of table name does not work DBZ-6855
-
JDBC sink insert fails with Oracle target database due to semicolon DBZ-6857
-
Oracle test shouldContinueToUpdateOffsetsEvenWhenTableIsNotChanged fails with NPE DBZ-6860
-
Tombstone events causes NPE on JDBC connector DBZ-6862
-
Debezium-MySQL not filtering AWS RDS internal events DBZ-6864
-
Avoid getting NPE when executing the arrived method in ExecuteSnapshot DBZ-6865
-
errors.max.retries = 0 Causes retrievable error to be ignored DBZ-6866
-
Streaming aggregation pipeline broken for combination of database filter and signal collection DBZ-6867
-
ChangeStream aggregation pipeline fails on large documents which should be excluded DBZ-6871
-
Oracle alter table drop constraint fails when cascading index DBZ-6876
Altogether, a total of 36 issues were fixed for this release. A big thank you to all the contributors from the community who worked on this release: Andy Pickler, Anisha Mohanty, Breno Moreira, Chris Cranford, Harvey Yue, Indra Shukla, Jakub Cechacek, Jiri Pechanec, Mario Fiore Vitale, Nancy Xu, Nir Levy, Ondrej Babec, Thomas Thornton, and tison!
Outlook & What’s Next?
Debezium 2.4 is shaping up quite nicely with our second Beta2 preview release which now includes OpenLogReplicator support. We intend to spend the remaining several weeks as we move toward a 2.4 final working on stability and any regressions that are identified. We encourage you to give Debezium 2.4.0.Beta2 a try. I would anticipate a Beta3 likely next week to address any shortcomings with OpenLogReplicator with the hope of a final by end of the month.
Don’t forget about the Debezium Community Event, which I shared with you on the mailing list. The event will be held on Thursday, September 21st at 8:00am EDT (12:00pm UTC) where we’ll discuss Debezium 2.4 and the future. Details are available on the Zulip chat thread, so be sure to join if you are able, we’d love to see you there.
Additionally, if you intend to participate at Current 2023 (formerly Kafka Summit) in San Jose, California, I will be there doing on a presentation on Debezium and data pipelines Wednesday afternoon with my good friend Carles Arnal. There will also be another presentation by my colleague Hans-Peter Grahsl on event-driven design you shouldn’t miss. If you’d like to meet up and have a quick chat about Debezium, your experiences, or even just to say "Hi", I’d love to chat. Please feel free to ping me on Zulip (@Chris Cranford) or send me a notification on Twitter (@crancran77).
As always, if you have any ideas or suggestions, you can also get in touch with us on the mailing list or our chat.
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.