As the team leaps into Q3, we’re happy to announce the fruits of our Q2 work, Debezium 2.7.0.Final is now generally available. This release includes changes for 140 issues with contributions from over 51 contributors. Let’s take a moment and review all the changes.
Breaking changes
The upgrade to Debezium 2.7.0.Final includes a total of 5 unique breaking changes:
- Core
-
-
Debezium snapshot artifacts were originally deployed to
oss.sontatype.org
, the now legacy Sonatype infrastructure. This has changed and artifact snapshots are now found ats01.oss.sonatype.org
, the new Sonatype infrastructure (DBZ-7641). -
Under certain circumstances, JDBC queries were observed in a persistent hung state, such as database communication errors. A configurable timeout property called
query.timeout.ms
was introduced as a way to mitigate the problem for users who experienced this issue. This option defaults to600000
milliseconds (600 seconds), but can be changed to a value of0
to disable timeout handling (DBZ-7616).
-
- Oracle
-
-
Tables that use
NUMERIC
data types with zero scale were ignored by thedecimal.handling.mode
when set todouble
orstring
. This has been fixed and such columns will now be correctly emitted according to the configured decimal handling mode. This can cause issues with upgrades for deployments using strict schema registry compatibility rules (DBZ-7882).
-
- PostgreSQL
-
-
PostgreSQL 10 and 11 have been in end-of-life (EoL) mode for some time now. Support for these versions are now considered best-effort, meaning we no longer explicitly test Debezium against these database versions. Any regressions will not be proactively fixed by the core team; however, community contributions will continue to be accepted for regressions (DBZ-7128).
-
- SQL Server
-
-
In older versions of Debezium, the SQL Server connector would process all transactions available during a poll iteration. This could lead to memory concerns with large traffic. The
max.iteration.transactions
configuration property already exists to solve this corner case for users, but it defaulted to0
, meaning that the connector would process all transactions by default. This configuration’s default value has changed and is now500
to provide more seamless integration for the default configuration use cases (DBZ-7750).
-
New features and improvements
The upgrade to Debezium 2.7.0.Final introduces many new features and improvements in several components:
Core
Transaction metadata encoded ordering
In some pipelines, ordering is critical for consuming applications. There are certain scenarios that can impact this aspect of your data pipeline, such as when Kafka re-partition occur. This leads to problems that can be error-prone trying to reconstruct the ordering after-the-fact.
Now when Transaction Metadata is enabled, these metadata events will also encode their transaction order, so that in the event that a Kafka re-partition or other scenarios occur that alter the ordering semantics, consumers can simply use the new encoded ordering field instead for deterministic ordering of transactions (DBZ-7698).
Blocking incremental snapshot improvements
There are some use cases where incremental snapshot signals require escaping certain characters in the fully-qualified table name. This caused some problems with blocking snapshots because the process to resolve what tables to snapshot used a slightly different mechanism. In Debezium 2.7, we’ve unified this approach, and you can now use escaped table names with blocking snapshots where applicable (DBZ-7718).
Optional delay between snapshot & streaming
Debezium 2.7 ships with a new global configuration option, streaming.delay.ms
. This new option causes the connector to perform a delay before it starts the streaming phase (DBZ-7902).
For some deployment use cases, you may want to guarantee that at least one offset flush interval has happened before the streaming phase begins. In such use cases, uses should guarantee that both properties, streaming.delay.ms
and offset.flush.interval.ms
are aligned.
By default, Debezium will not perform a delay and immediately transitions to the streaming phase to remain consistent with prior version behavior.
Truncating array fields
The column.truncate.to.length.chars
configuration property is improved, supporting a combination of string and now, array field types (DBZ-7925).
Db2
Support Db2 on z/OS
Debezium 2.7 introduces incubating support for using the Db2 connector on the z/OS platform. For the Db2 connector to work with z/OS, several configuration options are required to toggle the connector to work with the z/OS platform database (DBZ-4812).
{
...,
"db2.platform": "ZOS",
"cdc.control.schema": "ASNCDC",
"cdc.change.tables.schema": "ASNCDC"
}
The primary toggle to run in z/OS mode is done using the db2.platform
, which defaults to LUW
for running on Linux, Unix, and Windows. Setting this configuration option to ZOS
enables z/OS.
In addition, we have also added the cdc.control.schema
and cdc.change.tables.schema
connector configuration properties. These were previously hardcoded as ASNCDC
, and while this remains the default, if your installation uses different schemas, this is now possible.
For existing Db2 connectors for Linux, Unix, and Windows, no configuration changes are required when upgrading. The |
JDBC
MariaDB dialect support
While MariaDB and MySQL often share a lot of similar syntax, unfortunately there are corner cases where these two vary slightly that introduces incompatibilities. One such incompatibility was found with now the Debezium JDBC sink constructed upsert statements, which could not be executed when using a MariaDB target database.
Debezium 2.7 officially introduces MariaDB dialect support for the JDBC sink connector, enabling users to configure the JDBC sink to write changes from Kafka topics to a MariaDB target (DBZ-7874). In general, no special configuration should be necessary as both Hibernate and Debezium should detect the target is MariaDB and use the correct dialects.
If you find the dialect resolution does not resolve to MariaDB, you can enforce its use by setting the connector configuration |
MariaDB
New MariaDB standalone connector
Debezium 2.5 introduced official support for MariaDB as part of the existing MySQL connector. The next step in that evolution is here, with a new standalone connector implementation for MariaDB (DBZ-7693).
There are few things worth noting here:
-
MariaDB and MySQL both have a common shared dependency on a new abstract connector called
debezium-connector-binlog
, which provides the common framework for both binlog-based connectors. -
Each standalone connector now specifically is tailored only to its target database, so MySQL users should use MySQL and MariaDB users should use MariaDB. As a result, the
connection.adapter
configuration option has been removed, and thejdbc.protocol
configuration option is now only specific to certain MySQL use cases and not used by MariaDB.
The documentation for this connector is still a work-in-progress and will be added in the future. For the moment, you can refer to the MySQL connector documentation for most things related to MariaDB.
Snapshot row count estimates can be disabled
In some circumstances, users may find that the query that generates the row count estimates for MySQL and MariaDB can have some performance impacts in some environments.
If you determine this query performs poorly, or the calculation is of no interest, it can be safely disabled by setting the log level for io.debezium.connector.binlog.BinlogSnapshotChangeEventSource.RowEstimate
to WARN
(DBZ-7640).
If you are deploying on Kafka Connect, be sure to adjust the Kafka Connect log4j configuration. If you are deploying using Debezium Server, be sure to adjust the log configuration in application.properties
based on the Quarkus documentation.
MongoDB
Support predicate conditions for MongoDB incremental snapshots
The incremental snapshot process is an instrumental part in various recovery situations to collect whole or part of the data set from a source table or collection. Relational connectors have long supported the idea of supplying an additional-conditions
value on the incremental snapshot signal to restrict the data set, providing for targeted resynchronization of specific rows of data.
We’re happy to announce that this is now possible with MongoDB (DBZ-7138). Unlike relational databases, the additional-conditions
should be supplied in JSON format. It will be applied to the specified collection using the find
operation to obtain the subset list of documents that are to be incrementally snapshotted.
ExtractNewDocumentState includes document id for MongoDB deletes
In prior release of the MongoDB ExtractNewDocumentState
single message transformation, a delete event did not provide the identifier as part of the payload. This reduced the meaningfulness of delete events as consumers were supplied with insufficient data to act on these events. This behavior has been improved, and the delete event now includes an _id
attribute in the payload (DBZ-7695).
Collection-scoped change streams
In previous iterations of the Debezium MongoDB connector, change streams could be opened against the deployment and database scopes, which was not always ideal for restrictive permission environments. Debezium 2.7 introduces a new change stream mode where the connector can operate on a single collection scope, allowing for such granular permissive configurations (DBZ-7760).
A new capture scope value has been added called collection
, which can be set using capture.scope
. This is useful if the connector is deployed to only capture changes for a single collection in MongoDB.
Please see the documentation about limitations of this new, incubating feature.
MySQL
Snapshot row count estimates can be disabled
In some circumstances, users may find that the query that generates the row count estimates for MySQL and MariaDB can have some performance impacts in some environments.
If you determine this query performs poorly, or the calculation is of no interest, it can be safely disabled by setting the log level for io.debezium.connector.binlog.BinlogSnapshotChangeEventSource.RowEstimate
to WARN
(DBZ-7640).
If you are deploying on Kafka Connect, be sure to adjust the Kafka Connect log4j configuration. If you are deploying using Debezium Server, be sure to adjust the log configuration in application.properties
based on the Quarkus documentation.
Oracle
New Oracle "RawToString" custom converter
While Oracle recommends that users avoid using RAW
-based columns, these columns are still widely used in standard Oracle tables for backward compatibility reasons. But there are also business use cases where it makes sense to continue to use RAW
columns rather than other data types.
Debezium 2.7 introduces a new custom converter specifically for Oracle called RawToStringConverter
(DBZ-7753). This custom converter is designed to allow you to quickly convert the byte-array contents of the RAW
column to a string-based field using a STRING
schema type. This can be useful for situations where you use a RAW
column to store character data that doesn’t require the collation overhead of VARCHAR2
, but you still have the need for this field to be sent to consumers as string-based data.
To get started with this custom converter, please see the documentation for more details.
Improved NLS character-set support for Oracle
When installing the Debezium 2.7 Oracle connector, you may notice a new dependency, orai18n.jar
. This dependency is being automatically distributed to provide extended character-set support for certain dialects (DBZ-7761).
Oracle ROW_ID included in change events
While ROW_ID
is not unique across all rows of a table for the table’s lifespan, it can be used in certain situations when the lifecycle of the table and rows are managed in a very strict way. At the community’s request, we’ve added a new row_id
field to the Oracle connector’s change event source information block (DBZ-4332). This new field will be populated with the ROW_ID
value under the following conditions:
-
Only populated from streaming events for inserts, updates, and deletes.
-
Snapshot evnets will not contain a
row_id
value. -
Only provided by the LogMiner and XStream streaming adapters, OpenLogReplicator is not supported.
Any event that does not match the criteria will not include a row_id
field as its marked as optional.
Oracle flush table with custom schema names
In prior versions of Debezium, the Oracle connector was strictly designed to create the LogMiner flush table in the default tablespace of the connector user account. This wasn’t always useful in situations where the user’s default tablespace may not be the ideal destination and the DBA would prefer that table to exist in a separate tablespace.
Previously, users would need to modify the user account or use a new user with the correct tablespace to have the table created in the right tablespace location. With Debezium 2.7, this is no longer required, and you can safely include the name of the target schema/tablespace in the configuration (DBZ-7819).
log.mining.flush.table.name=THE_OTHER_SCHEMA.LOG_MINING_FLUSH_TABLE
The schema name is optional and if not supplied, the connector continues to use the same legacy behavior of creating the flush table and checking for its existence in the user’s default tablespace.
Oracle query filter with large numbers of tables
The Debezium Oracle connector can support thousands of tables in a single connector deployment with ease; however, you may have found you wanted to customize the query filter using the IN
mode. This mode is used in situations where you may have a high volume of changes for other tables and you want to filter that dataset out at the database level before the changes are passed to Debezium for processing.
In earlier versions, users may have noticed that setting log.mining.query.filter.mode
with a value of in
and where your table include list contained more than 1000 elements generated a SQL error. Oracle does not permit more than 1000 elements within an in-clause; however, Debezium 2.7 addresses this limitation by using a disjunction between multiple buckets of 1000 item in-clause lists (DBZ-7847).
PostgreSQL
PostreSQL Arrays with the JDBC sink
The JDBC sink connector supports the use of mapping source columns to Kafka ARRAY-based payload field types. With Debezium 2.7, you can now serialize ARRAY-based fields to a target PostgreSQL database, with no change in configuration. The new support should be completely transparent (DBZ-7752).
Read-only incremental snapshots
An incremental snapshot is feature of Debezium to capture all or a subset of historical data from one or more tables in your source database using an ad-hoc signal to start the snapshot. This process normally requires writing to the signal database table to maintain open/close watermarks in the transaction log for deduplication with the change stream that overlaps with the incremental snapshot stream.
Debezium has supported read-only incremental snapshots with other database vendors such as MySQL and MariaDB; however, Debezium 2.7 introduces support for read-only incremental snapshots with PostgreSQL. If you would like information, please check out the design proposal.
This process works by using the pg_current_snapshot
function to obtain information about what transactions are currently active in the database, which is only available on PostgreSQL 13. This means that to utilize read-only incremental snapshots, you must be using PostgreSQL 13 or later.
In order to activate read-only incremental snapshots on PostgreSQL 13 or later, you simply need to add the read.only
connector configuration property set to true
in the connector configuration. The incremental snapshot implementation will choose to use the read-only implementation when this is set to true
, similar to the same behavior for MySQL and MariaDB (DBZ-7917).
SQL Server
Heartbeat action query now supported
The heartbeat.action.query
connector configuration property enables the connector to perform a write operation to the source database on an interval defined by heartbeat.interval.ms
. The write operation is meant to produce a change event that is captured by the connector, and is sent to Kafka or the target system.
In an active database that is capturing changes regularly, you do not need to worry setting heartbeat.action.query
, as the constant stream of changes is sufficient to keep the offsets synchronized with the read position in the transaction logs. However, if the connector is capturing changes from a source that has a higher volume of changes to non-captured tables than to captured tables, this can be useful to keep the read position in the offsets synchronized with the lower capture activity.
In Debezium 2.7, we’ve added support for this property for SQL Server (DBZ-7801). For additional details, please see the SQL Server documentation.
Cassandra
Cassandra performance improvement
The Cassandra connector also saw some changes in Debezium 2.7, specifically to performance optimizations. The implementation of the KafkaRecordEmitter
relied on a thread-synchronization block that reduced the throughput. In addition, the implementation also performed some unnecessary flushing which also impacted performance. This code has been rewritten to improve both throughput and reduce the unnecessary flush calls (DBZ-7722).
Vitess
Improved temporal support in Vitess
Debezium relational connectors rely on a configuration option, time.precision.mode
, to control how temporal values are added to change events. In some cases, you may want to use modes that align with Kafka types, using the connect
mode. In other cases, you may prefer to avoid precision loss by using the default, adaptive_milliseconds
mode.
The Debezium for Vitess connector has traditionally not followed this model, and instead has emitted temporal values as string-based types. While this helps avoid the loss of precision problem when using the connect
mode, this adds unnecessary overhead on consumers to parse and manipulate these values.
In Debezium 2.7, Vitess aligns this behavior with other relational connectors, using the time.precision.mode
to control how temporal values are sent (DBZ-7773). By default, it will use the adaptive_milliseconds
mode, but you can customize this to use connect
mode if you prefer. The emission of string-based temporal values has been removed.
Support heartbeat events
Debezium provides a mechanism to write to the database periodically to emit synchronizing events for offsets using the heartbeat.action.query
connector configuration property. For Vitess, this is unnecessary since the Vitess/VStream provides this out of the box with the HeartbeatInterval flag.
Debezium 2.7 uses the HeartbeatInterval
VStream flag, only requiring users to set the heartbeat.interval.ms
in the connector configuration. As the heartbeat flag is observed in the VStream, a heartbeat event will be emitted based on that interval (DBZ-7962)
Debezium Server
NATS authentication with JWT/seed
The Debezium Server NATs streaming sink adapter was improved, supporting JWT/seed based authentication (DBZ-7829). To get started using the JWT/seed-based authentication, supply the following necessary values in the configuration:
debezium.sink.nats-jetstream.auth.jwt=<your_jwt_token>
debezium.sink.nats-jetstream.auth.seed=<your_nkey_seed>
For this and more, please see the NATS documentation for details about JWT and NKey seed based authentication.
NATS JetStream sink authentication improvements
The Debezium Server NATS JetStream sink also includes improvements to support authentication and encryption in Debezium 2.7. Several new configuration properties are now supported to pass keystore details to the sink adapter (DBZ-7922).
...
debezium.sink.nats-jetstream.auth.tls.keystore=<path-to-keystore-file>
debezium.sink.nats-jetstream.auth.tls.keystore.password=secret-password
debezium.sink.nats-jetstream.auth.tls.password=<tls-password>
To get started using the new authentication and encryption features, simply include the above three configurations into your Debezium Server configuration with appropriate values.
JMX Exporter added
The JMX Exporter agent has been added as part of the Debezium Server container image. This should enable users to easily fetch connector metrics while running Debezium Server out of the box (DBZ-7913).
To enable the JMX Exporter, simply specify the JMX_EXPORTER_PORT
environment variable when creating a container based on debezium/debezium-server:2.7
or later, and making sure that the agent’s port is accessible to the outside world on the container.
The JMX Exporter uses the configuration in |
Kubernetes Operator
Install Debezium Operator with Helm Chart
To improve the deployment of the Debezium Operator, it can be installed with a helm chart at https://charts.debezium.io. This avoids the overly complicated deployment model of installing the operator into separate namespaces, minimizing the complexities for managing multiple Debezium Server deployments on Kubernetes.
Enabling JMX Exporter with Debezium Operator
If you are deploying Debezium Server on Kubernetes using the Debezium Operator, the new JMX Exporter feature in Debezium Server can be enabled directly through the operator custom resource (DBZ-7914). To get started using the exporter with the operator, new configuration operations were added:
runtime:
metrics:
jmxExporter:
enabled: true
configFrom:
key1: value1
key2: value2
In the custom resource, the jmxExporter.enabled
toggles the exporter on or off. Additionally, the metrics configuration can be supplied using key/value pairs in the jmxExporter.configFrom
section.
Stopping Debezium Server when scaled to zero
Debezium Server is stopped when scaling the replicas to zero using the annotation, debezium.io/stop=true
(DBZ-7953).
Other changes
In total, 140 issues were resolved in Debezium 2.7. The list of changes can also be found in our release notes. A big thank you to all the contributors from the community who worked diligently on this release: Ashwin Murali Krishnan, Alexandru Cristu, Amirmohammad Sadat Shokouhi, Andrey Pustovetov, Anil Dasari, Anisha Mohanty, Ankur Gupta, Aristofanis Lekkos, Bertrand Paquet, Choi Jang Ho, Chris Cranford, Chris Recalis, Chungeun Choi, Daan Gerits, Duc Le Tu, Gaurav Miglani, Ilyas Ahsan, Ismail Simsek, Jakub Cechacek, Jiri Novotny, Jiri Pechanec, Jochen Schalanda, Kevin Rothenberger, Lars M. Johansson, Lourens Naudé, Maithem, Mario Fiore Vitale, Mark Banierink, Martin Medek, Michal Augustýn, Michal Pioun, Ondrej Babec, Piotr Piastucki, Pradeep Nain, Rajendra Dangwal, Robert Roldan, Robin Moffatt, Roman Kudryashov, Sean C. Sullivan, Selman Genç, Seongjoon Jeong, Indra Shukla, Sylvain Marty, Thomas Thornton, Tomasz Gawęda, Vadzim Ramanenka, Vojtech Juranek, Vojtěch Juránek, einar-rt, ismail simsek, leoloel, and Mathijs van den Worm!
What’s next & Outlook
With the release of Debezium 2.7, the team is now switching gears and heavily focusing on the next major milestone, Debezium 3.0. This next major release includes a variety of changes, including but not limited to:
-
Java 17 as baseline
-
Kafka 3.1+ as baseline
-
New off-heap Oracle cache implementations based on EhCache & Hazelcast
-
Exactly-once semantics support for other relational connectors
-
Sink connector for MongoDB
-
and more
This list represents a quick glance view at just what’s at the top of our queue, and is subject to change. If you’d like to get involved in the conversation about Debezium 3.0 and the next evolution of the project, contact us on the mailing list or Zulip chat. As always, please review our road map for more details.
As summer is in full swing and holiday plans are starting for many, be safe. Until next time…
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.