I am excited to announce the second maintenance release for the Debezium 3 release stream, 3.0.2.Final. This maintenance release introduces a number of features, let’s take a moment and dive into the highlights.
New features and improvements
Debezium 3.0.2.Final introduces a couple of improvements and features, lets take a look at each individually.
Core
Perform blocking snapshots with never snapshot mode
The Debezium blocking snapshot process is designed to execute the initial snapshot based on the signal provided, selectively emitting the historical data for one or more tables. When this was paired with the never snapshot mode
, this lead to unexpected behavior.
In this release, we modified the connector offsets to track the configured snapshot.mode
, allowing the blocking snapshot to succeed and perform the initial snapshot when signaled, even if the snapshot.mode
is configured to never perform a snapshot. This allows users to safely use this feature with this configuration (DBZ-7903).
Due to the connector offset storage change, once the connector is upgraded to 3.0.2.Final or later, the connector cannot be downgraded to 3.0.1.Final or earlier. |
MongoDB
RowsChanged JMX metric type changed
In previous builds of the MongoDB connector, the RowsChanged
JMX metric is exposed as a java.util.Map
, which contradicts the same JMX metric exposed on relational connectors, which is TabularData
. This has been fixed in 3.0.2.Final, the JMX metric uses TabularData
across all connector implementations for uniformity (DBZ-8359).
Any existing MongoDB JMX pipelines may need to be adjusted if you were previously capturing |
Oracle
Higher precision with timestamps
Debezium for Oracle has traditionally emitted column timestamp values with millisecond precision that is controlled by the NLS session properties set on the mining session connection. The precision is improved and provides nanosecond-based (aka FF9
) values (DBZ-8379).
The emitted field type is based on the column’s data type, so field emitted data types remain unchanged. What will change is cases where columns have micro or nanosecond-based values, where these were previously zero, they’ll now have non-zero values. |
Warn or skip DML exceptions
The event.processing.failure.handling.mode
can be configured to fail, warn, or skip specific connector error conditions to improve connector reliability to various data issues. THis configuration is historically used to control how the Oracle connector behaves when a DDL failure is observed.
In this release, the event.processing.failure.handling.mode
is also used to control failures for DML-based events. If there was an issue with the Oracle connector parsing your insert, update, or delete operations, you can safely configure the connector to fail, warn, or skip the DML event based on your needs (DBZ-8208).
The default behavior is to always fail when an event is not safely handled by the connector. By adjusting this to warn or skip, while the connector will safely continue past the failed event, you will introduce data loss and will needs to be addressed manually. |
Vitess
Performance improvements
In earlier builds of the Debezium for Vitess connector, the connector used a regular expression-based filter system that matches all tables based on a prefix with varying suffixes, and later exclusions would be applied based on configuration. This has the potential to waste CPU and create hotspots due to creating intermediate objects for the event that would later to be filtered and garbage collected.
In this release, we’ve improved the way the Vitess connector processes this use case by applying the filtration earlier in the event processing chain. This should reduce the number of intermediate objects created and improve the overall performance of the connector. For key spaces that have the same prefix and differing suffixes, this should provide better overall performance than older builds (DBZ-8354).
Sink connectors
Debezium 0.x introduced a common source-connector framework that has become the foundation for source connectors provided by the project, including our community-led connectors such as Spanner, Vitess, and others. With the introduction of the MongoDB sink connector recently, our long-term goal is to approach sink connectors in a similar way, providing a common sink-connector framework to ease the creation of Debezium-based sink connectors.
Over Debezium 3.x lifecycle, you will see incremental steps to streamline the source across the JDBC and MongoDB sink connectors. We will minimize disruptions in maintenance releases as you have come to expect, but expect future major and minor releases to introduce deprecations and changes to support this endeavor.
In this first round of changes, we’ve introduced a new Debezium module: debezium-sink
. This module acts as the foundation for all sink connectors and is home to a variety of common classes, including the SinkConnectorConfig
class, naming strategy implementations, and the common representation of a sink record, DebeziumSinkRecord
.
As we continue to streamline the MongoDB and JDBC sink connectors, additional common behavior will be added.
JDBC sink connector changes
With the sink module using the naming convention of collection rather than table, several configuration properties have been deprecated and replaced. The old properties will continue to work in Debezium 3.0.x builds; however will be removed in Debezium 3.1.
-
The
table.name.format
property is replaced bycollection.name.format
. -
The
table.naming.strategy
property is replaced bycollection.naming.strategy
.
In addition, the contract for io.debezium.connector.jdbc.naming.TableNamingStrategy
specified by the table.naming.strategy
property is deprecated. A new io.debezium.sink.naming.CollectionNamingStrategy
has been introduced with a slightly different signature.
/**
* Resolves the logical table name from the sink record.
*
* @param config sink connector configuration, should not be {@code null}
* @param record Kafka sink record, should not be {@code null}
* @return the resolved logical table name; if {@code null} the record should not be processed
*/
String resolveTableName(JdbcSinkConnectorConfig config, SinkRecord record);
/**
* Resolves the logical collection name from the Debezium sink record.
*
* @param record Debezium sink record, should not be {@code null}
* @param collectionNameFormat the format string for the collection name (mapped from the topic name)
* @return the resolved logical collection name; if {@code null} the record should not be processed
*/
String resolveCollectionName(DebeziumSinkRecord record, String collectionNameFormat);
The main differences include the new DebeziumSinkRecord
which replaces SinkRecord
and explicitly passing the collection naming format rather than the configuration class.
If you implement a custom |
Debezium Operator
Enabling Debezium Server REST endpoint
The Debezium Server API REST-ful endpoint can now be enabled automatically through a Debezium Server deployment on Kubernetes using the Debezium Operator. In the spec
section of the deployment descriptor, you can include the runtime.api.enabled
property to toggle the API endpoint (DBZ-8234), as shown below.
apiVersion: debezium.io/v1alpha1
kind: DebeziumServer
metadata:
name: my-debezium
spec:
image: quay.io/debezium/server:3.0.2.Final
quarkus:
config:
log.console.json: false
kubernetes-config.enabled: true
kubernetes-config.secrets: postgresql-credentials
runtime:
api:
enabled: true
sink:
type: kafka
config:
producer.bootstrap.servers: dbz-kafka-kafka-bootstrap:9092
producer.key.serializer: org.apache.kafka.common.serialization.StringSerializer
producer.value.serializer: org.apache.kafka.common.serialization.StringSerializer
source:
class: io.debezium.connector.postgresql.PostgresConnector
offset:
memory: { }
schemaHistory:
memory: { }
config:
database.hostname: postgresql
database.port: 5432
database.user: ${POSTGRES_USER}
database.password: ${POSTGRES_PASSWORD}
database.dbname: ${POSTGRES_DB}
topic.prefix: inventory
schema.include.list: inventory
By default, the Debezium Server API endpoint is disabled, but can be enabled by setting the spec.runtime.api.enabled
with a value of true
, as shown above.
Other fixes
In total there were 46 issues resolved in Debezium 3.0.2.Final. The list of changes can also be found in our release notes.
Here are some noteworthy changes:
-
Clarify signal data collection should be unique per connector DBZ-6837
-
Race condition in stop-snapshot signal DBZ-8303
-
Debezium shifts binlog offset despite RabbitMQ Timeout and unconfirmed messages DBZ-8307
-
Use DebeziumSinkRecord instead of Kafka Connect’s SinkRecord inside Debezium sink connectors DBZ-8346
-
Implement new config map offset store in DS DBZ-8351
-
Debezium server with eventhubs sink type and eventhubs emulator connection string fails DBZ-8357
-
Filter for snapshot using signal doesn’t seem to work DBZ-8358
-
JDBC storage module does not use quay.io images DBZ-8362
-
Failure on offset store call to configure/start is logged at DEBUG level DBZ-8364
-
Object name is not in the list of S3 schema history fields DBZ-8366
-
Faulty "Failed to load mandatory config" error message DBZ-8367
-
Upgrade protobuf dependencies to avoid potential vulnerability DBZ-8371
-
Add transform page to provide a single place to list the already configured transform plus UI to add a new transform DBZ-8374
-
Upgrade Kafka to 3.8.1 DBZ-8385
-
Tests in IncrementalSnapshotIT may fail randomly DBZ-8386
-
Add Transform Edit and delete support. DBZ-8388
-
Log SCN existence check may throw ORA-01291 if a recent checkpoint occurred DBZ-8389
-
ExtractNewRecordState transform: NPE when processing non-envelope records DBZ-8393
-
Oracle LogMiner metric OldestScnAgeInMilliseconds can be negative DBZ-8395
-
SqlServerConnectorIT.restartInTheMiddleOfTxAfterCompletedTx fails randomly DBZ-8396
-
ExtractNewDocumentStateTestIT fails randomly DBZ-8397
-
BlockingSnapshotIT fails on Oracle DBZ-8398
-
Oracle OBJECT_ID lookup and cause high CPU and latency in Hybrid mining mode DBZ-8399
-
Upgrade Kafka to 3.9.0 DBZ-8400
-
Protobuf plugin does not compile for PostgreSQL 17 on Debian DBZ-8403
-
Update Quarkus Outbox Extension to Quarkus 3.16.3 DBZ-8409
A big thank you to all the contributors from the community who worked diligently on this release: Anisha Mohanty, dario, Chris Cranford, Enzo Cappa, Jakub Cechacek, Jiri Pechanec, Kavya Ramaiah, Lars M. Johansson, Mario Fiore Vitale, Martin Vlk, P. Aum, René Kerner, Stanislav Deviatov, Stefan Miklosovic, Thomas Thornton, Vojtech Juranek, and Yevhenii Lopatenko!
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.