The Debezium team has been extremely busy this past quarter as we prepared for this summer release, and we’re excited to announce the immediate availability of Debezium 3.2.0.Final. This release includes a slew of features including integration with OpenLineage, a new Quarkus DevService/GraalVM extension, Qdrant vector database sink support, improvements to Debezium Platform and AI, and much more!

In this post, we’re going to take a deep dive into all the changes in Debezium 3.2, discussing new features, and explaining all the possible changes that could have any impact to your upgrade process. As always, we recommend you read the release notes to learn about all the bugs that were fixed, update procedures, and more.

Breaking changes

With any new release of software, there is often several breaking changes. This release is no exception, so let’s discuss the major changes you should be aware of before upgrading to Debezium 3.2.0.Final.

Debezium Core

Debezium AI

Debezium for MySQL

Debezium for Oracle

Debezium for IBMi

Debezium core

Kafka 4.0 support

Debezium is now built and tested using Apache Kafka 4.0 (DBZ-8875). As Kafka introduced several API changes between 3.5 and 4.0, we strongly suggest users check Debezium’s support matrix for Kafka compatibility.

Snapshot deployments have moved

Debezium has moved from the RHOSS Sonatype infrastructure to Maven Central (DBZ-9025). To make sure that you’re fetching the latest snapshot builds, please use the updated links, available on our nightly documentation page, found here. Please note, due to current API limitations, these links are updated once daily, even though we push multiple snapshot builds to Maven Central throughout the day.

ExtractNewRecordState deprecated configuration option removed

Debezium introduced the configuration option delete.tombstone.handling.mode in Debezium 2.5 as part of DBZ-6907, marking drop.tombstones and delete.handling.mode as deprecated. We felt that sufficient time has past for the legacy configuration options to be removed, and so Debezium 3.2 no longer uses drop.tombstones nor delete.handling.mode. If your connector configurations have continued to make use of the deprecated options, please adjust your configuration to use delete.tombstone.handling.mode instead (DBZ-6068).

Schema history no longer stores certain DDL statements

Certain DDL events such as TRUNCATE and REPLACE were stored in the internal Debezium schema history topic. Debezium does not require these types of statements to represent schema evolution, and will no longer be captured in the internal schema history topic moving forward (DBZ-9085).

Debezium AI

FieldToEmbedding transformation configuration options changed

The new FieldToEmbedding transformation available as part of the Debezium AI module used a prefix string in the names of its configuration options. This prefix was superfluous, and therefore we’ve decided to remove the prefix (DBZ-9056).

Debezium for MySQL

Improved missing log position validation

When the Debezium MySQL connector was configured with any snapshot.mode that was not when_needed and the binary log position was not available, the connector logged a warning indicating that the connector would resume from the last available position in the logs. However, when the connector transitioned into the streaming phase, it immediately failed reporting that the binary log position could not be located. We have improved this behavior so that we throw the error during the validation phase rather so that it aligns with the behavior of other connectors (DBZ-9118).

Debezium for Oracle

TLS using JKS

The Debezium for Oracle connector not only supports TLS connections with Oracle Wallet, but you can also use JKS as well. To use JKS, special connector configurations are required to provide the right information to the Oracle JDBC driver. This is now properly documented for the community (DBZ-8075).

Debezium for IBMi

No longer uses Avro schema naming by default

The Debezium IBMi connector was explicitly coded to use the Avro schema naming adjuster (DBZ-9183). This is no longer the case and the option is now configurable, and uses the same behavior as other connectors.

Journal file data loss

In prior versions of Debezium for IBMi, if the journal file needed by the connector wasn’t available, it would resume reading from the next available file. This could lead to potential silent data loss. The behavior has changed and the connector will react like other connectors by reporting an error when a required journal file doesn’t exist (DBZ-8898).

New features and improvements

The following describes all noteworthy new features and improvements in Debezium 3.2.0.Final. For a complete list, be sure to read the release notes for more details.

Debezium Core

Debezium for PostgreSQL

Debezium for Oracle

Debezium JDBC sink

Debezium for Informix

Debezium for IBMi

Debezium for Vitess

Debezium Embedded

Debezium Server

Debezium Platform

Debezium AI

Debezium OpenLineage

Debezium Quarkus Extension

Debezium core

Schema history recovery moved coordinator thread

In previous versions of Debezium, schema history recovery was performed in the connector’s main startup thread. When the schema history was extraordinarily large, this recovery could take a while to complete.

For environments like Kafka Connect, this could lead to issues with Kafka Connect misrepresenting the connector’s current status. For example, if a connector had previously failed and was in the process of restarting, Kafka Connect would continue to report the connector in FAILED status, although the connector is up, running but is currently recovering the schema history topic. This subtle but important detail would be hard to isolate without looking at the connector’s logs.

To address the concern, Debezium delays the recovery of the schema history topic until the start of Debezium coordinator thread. This allows the Kafka Connect runtime’s startup to return quicker and update the runtime’s connector status to accurately represent the connector’s status without a long schema history recovery having an impact (DBZ-8562).

Kafka schema history recovery performance improvements

A performance optimization opportunity was discovered by a community member in the Kafka schema history implementation. There were some steps during the recovery process that were being executed more frequently than necessary, which lead to higher CPU utilization. To optimize this, several of those steps are not called only once per schema history recovery. This reduces CPU utilization and network communications with the Kafka broker during the schema history recovery process (DBZ-9098).

Snapshot skipped metric

In previous releases, Debezium would indicate the snapshot completed by setting the SnapshotCompleted metric, even in cases where the snapshot was skipped due to configuration or offsets being available. This was misleading for some users, and a new SnapshotSkipped metric was introduced to clarify when the snapshot is performed and completes versus when the snapshot was skipped because it was not needed (DBZ-8610).

ExtractNewRecordState can now convert deletes to tombstones

Debezium automatically injects tombstone events into the change event stream immediately following a delete if configured. However, if the connector restarts and only the delete event is written to the Kafka topic, the connector won’t re-emit the tombstone on restart because tombstones are synthetic.

From the Kafka broker point of view, there’s no functional impact if this happens. However, this can be problematic for consumers that rely on the existence of tombstones for specific behaviors.

Because consumers often rely on a flattened event structure rather than Debezium’s hierarchical event payload, the ExtractNewRecordState transformation includes a new choice for the delete.tombstone.handling.mode configuration property called delete-to-tombstone. This option forces the transformation to convert the delete event into a tombstone. This allows you to disable tombstones and use the transformation to convert deletes to tombstones, ensuring that there is a tombstone both in the topic for compaction, but more importantly so the consumer can perform its behavior that relies on tombstones (DBZ-9022).

Regression with logging performance

In Debezium 3.1, a change was introduced to centralize the logging of sensitive information. Unfortunately, this change introduced a regression, leading to lower performance across a variety of code paths.

This change has been reverted and replaced with an implementation that retains the centralized logging intent while restoring the prior performance (DBZ-8879).

Reset certain streaming metrics through JMX

During periods of idle activity, a Debezium connector will continue to report LagBehindSource JMX metric as the last computed value, as this value is only updated as new changes are received. For some environments, this is less than desirable or not intuitive if you are unaware of the idle or low activity window.

Debezium 3.2 introduces a new option that can be triggered through JConsole or other JMX integrations to reset the current LagBehindSource metric by calling the new function resetLagBehindSource (DBZ-8885).

Log JMX MBean name when registration fails

When the JMX metric MBean fails registration, it’s most often due to the fact that an existing MBean is already registered with such name. This can happen for a variety of reasons, including a prior task not stopping gracefully or misconfiguration between two connectors sharing the same topic.prefix.

Unfortunately the log message does not provide adequate information to know which MBean and connector the failure is about. To address this, the JMX MBean name will be logged when the registration fails so that the exact connector deployment that’s affected can be determined more easily (DBZ-8862).

Debezium for PostgreSQL

Control partition table publishing by configuration

For PostgreSQL users, when capturing changes for partitioned tables, changes are emitted using the partition name rather than the base table. If you want to recombine changes across partitions back into a singular topic, this would normally require the use of the ByLogicalTableRouter to reroute events from all the partitions to a singular topic or to manually adjust the publication at the database level.

Debezium 3.2 aims to simplify this problem through configuration with the new publish.via.partition.root property. By default, this option is false, indicating that partition names are used for event topics rather than the base table unless you manually adjust the publication. However, you can set this property to true and Debezium will make sure the publication is created using the publish_via_partition_root feature so that events from the various partition tables are emitted using the single base table name (DBZ-9158).

Avoid write operations in read-only mode

A regression was identified when using the Debezium for PostgreSQL connector in read-only mode. Unfortunately, the connector would attempt to CREATE or ALTER the connector’s publication despite being configured for read-only mode, causing a SQL exception. We have corrected this behavior and Debezium will no longer attempt to create or alter the publication when operating in read-only mode (DBZ-8743).

Debezium for Oracle

Unbuffered LogMiner adapter using committed data only

Debezium for Oracle introduces a new adapter implementation that buffers transactions on the relational database using the database’s memory buffers rather than relying on the JVM heap or off-heap cache implementations (DBZ-8924).

To get started with the new adapter, the connector configuration needs to be adjusted, as shown here:

{
  "database.connection.adapter": "logminer_unbuffered"
}

While most of the log.mining.* connector configurations are honored by this new adapter, any configuration that relates to buffer, cache, or transaction management are not used.

This new adapter implementation is based on Oracle LogMiner’s COMMITTED_DATA_ONLY mode. This mode forces Oracle LogMiner to use the database’s available memory to buffer transactions and to only supply Debezium with committed changes. Because Debezium only receives committed changes, the connector can immediately dispatch changes as they’re read which avoids complex memory sizing requirements along with the need to handle partial rollbacks due to save points or constraint violations.

Because Oracle LogMiner is now responsible for buffering transactions until the commit is observed, Oracle LogMiner will rely on the database’s memory to handle this staging requirement. This directly means that this staging is only as powerful as the PGA_AGGREGATE_LIMIT database administrator’s configured Oracle database instance parameter. When a transaction’s memory requirement exceeds this limit, Oracle will refuse to buffer the transaction and will throw a connector error. This can be resolved by raising the configured limit, removing the limit entirely, or resizing your transactions so they fit within the boundary of this Oracle limit.

This feature is currently in incubating state, i.e. exact semantics, configuration options etc. may change in future revisions, based on the feedback we receive. Please let us know if you encounter any problems while using this extension.

Filter Oracle LogMiner results by client id

The Oracle LogMiner adapters provide a myriad of ways to exclude transactions by explicitly passing filters to the database query to excluding or only including transactions performed by specific database usernames. In this release, we’ve added another interesting filter criteria based on the Oracle LogMiner field CLIENT_ID, where you can elect to include or exclude changes based on this field’s value (DBZ-8904).

The following configuration properties can be used:

log.mining.clientid.include.list

Specifies a comma-separated list of values to match against the CLIENT_ID field for capture.

log.mining.clientid.exclude.list

Specifies a comma-separated list of values to match against the CLIENT_ID field to exclude from capture.

Just like any of the other include/exclude configuration properties, these are mutually exclusive.

Reduced CPU utilization under specific scenarios

In Debezium 3.1, we introduced a change as part of DBZ-8665 to restore the same performance from 2.7.0.Final when processing constraint violations or save point rollback operations. While this change was successful at reducing the latency caused by processing such events in Debezium 3.0 through 3.0.7, we found that even the performance from Debezium 2.7 was overall suboptimal.

We have implemented a complete rework of the transaction buffering solution to handle constraint violations and save point rollbacks more efficiently (DBZ-8860).

When using heap-based buffering, we reduced the time needed to process such events by nearly 90% while also reducing the time complexity for off-heap buffering by 97-99%. In addition to the time complexity reduction, we have also reduced the overall CPU usage while handling these events to remain aligned with expectations.

Improved the online_catalog mining strategy performance

Prior to adding the hybrid mining strategy, the Debezium Oracle LogMiner implementation included a specific condition to include events for tables where LogMiner failed to resolve the table name. This use case happens when the object id and version in the redo entry does not match the online data dictionary, which occurs after specific DDL operations are performed.

Including these changes, particularly when users perform bulk operations and LogMiner fails to resolve the table name, this increases latency, connector overhead, only so that the connector can log the unknown table. Given the reduction of performance solely for logging, we have chosen to omit including these events in the data fetch moving forward (DBZ-8926).

Improved the hybrid mining strategy performance

We also identified another performance bottleneck, this time when using the hybrid mining strategy while processing bulk events where LogMiner failed to resolve the table name during object id/version mismatches. The hybrid strategy is designed to handle this use case and fallback to Debezium’s relational model to resolve the table name; however, despite using a cache, the cost overhead for the cache lookups for bulk operations was significantly high.

In order to reduce the cost and improve throughput performance of bulk operations on unknown tables, we have reworked the lookup in a way that increases the throughput by significantly, allowing bulk operations to be handled more efficiently and with less overall CPU utilization (DBZ-8925).

Improve log message when failing to apply a partial rollback

When using the Debezium for Oracle LogMiner buffered adapter, when a partial rollback is observed, the log entry does not capture critical information that could be useful for debugging purposes. Debezium 3.2 now includes the transaction identifier and the system change number associated with the partial rollback redo entry (DBZ-8944).

Username sometimes returned as unknown

When a transaction is mined across two or more log mining steps, any event collected after the first step was returned with an UNKNOWN username. This had the potential to lead to situations where configured username exclusions would not be applied correctly and events were mistakenly emitted.

In Debezium 3.2, we have improved how this works so that the username attribute is collected properly, even if the transaction is mined across multiple steps. This improvement avoids the dispatch of the transaction to your Kafka topics, however the transactions will continue to be buffered for now (DBZ-8884).

Debezium JDBC sink

Configuration available to column/table naming strategies

The Debezium JDBC sink offers a variety of configurable hooks, including the option to define deployment-specific TableNamingStrategy or ColumnNamingStrategy implementations. However, these implementations were unable to obtain the full configuration for the connector deployment, making these hooks far less restrictive than intended.

With Debezium 3.2, these strategy implementations provide a new configure method (DBZ-7051), shown here:

@Override
public void configure(Map<String, Object> config) {
}

To provide backward compatibility, this method is defined as default with no implementation, so no code changes are required where this configuration step is unnecessary.

Avoid reduction buffer performing forced flushes

The Debezium JDBC sink connector can use one of two buffering algorithms.

The default is where for every change that happens in the source, there will be a matching SQL change at the target system. This algorithm relies on maintaining two buckets for each target table and periodically flushing those buckets based on sequence of observed events in that topic.

The second algorithm is called a reduction buffer where all events in a batch are analyzed and reduced so that the connector executes the fewest number of SQL operations across that entire batch. For example, if within a batch there was an insert, an update, and delete all for the same key in a table, the reduction buffer algorithm computes there are no writes necessary for that event’s primary key.

The underlying issue was that even though the reduction buffer should only flush at the end of the batch, flushes were being performed as if the connector was operating with the default algorithm, which leads to suboptimal performance. Using the previous example, the regression caused the reduction buffer to execute 2 SQL operations rather than no SQL operations.

This is fixed in Debezium 3.2 and the reduction buffer should now perform significantly better (DBZ-8982).

Debezium for Informix

Heartbeat action query support

The heartbeat.action.query configuration option is a widely used feature in many Debezium connectors. This option is often used when there is a case of high activity on non-captured tables, and it guarantees that there is a periodic event in a captured table to keep offsets from becoming stale. Starting with Debezium 3.2, the Debezium for Informix connector now supports this configuration option (DBZ-9081).

Debezium for IBMi

Add support for BOOLEAN data types

IBM provides support for the BOOLEAN data types on V7R5+ and later V7R4 releases of the IBM iSeries database. In prior versions, if a table contained a boolean data type, this would result in a hard failure with the Debezium for IBMi connector.

With Debezium 3.2, BOOLEAN data types are now captured without throwing an exception (DBZ-7796).

Add decimal handling mode support

The decimal.handling.mode configuration property specifies how the connector should handle floating point values for connector-specific data types. In prior versions of the Debezium for IBMi connector, this configuration property was not honored.

Debezium 3.2 introduces support for decimal.handling.mode, allow you to specify whether floating point values are serialized (DBZ-8301). This configuration property can be configured with one of three values:

precise

Represents values precisely by using java.math.BigDecimal values represented in change events in a binary form.

double

Represents values by using double values. Using double values is easier, but can result in a loss of precision.

string

Encodes values as formatted strings. Using the string option is easier to consume, but results in a loss of semantic information about the real type.

Debezium for Vitess

Customize load balancing connection policy

A new configuration property vitess.grpc.default.load.balancing.policy enables the connector to control how it’s connections are load balanced. For example, the property can be set to pick_first (the default) or round_robin (DBZ-9014).

Debezium Embedded

New polling started/ended callbacks added

The DebeziumEngine interface provides a variety of features, including knowing when the connector or task has started or stopped. However, because the connector operates asynchronously, it may be useful to know when the connector has entered the polling phase for specific user-driven logic.

To address this concern, two new methods have been added to the ConnectorCallback contract (DBZ-8948):

/**
 * Called after all the tasks have been successfully started and engine has moved to polling phase, but before actual polling is started.
 */
default void pollingStarted() {
    // nothing by default
}

/**
 * Called after all the tasks have successfully exited from the polling loop, i.e. the callback is not called when any of the tasks has thrown
 * exception during polling or was interrupted abruptly.
 */
default void pollingStopped() {
    // nothing by default
}

Now when setting up a DebeziumEngine instance, the ConnectorCallback supplied implementation can include logic to call during these lifecycle state changes as needed.

Debezium Server

Milvus allows unwinding of JSON data types

Debezium source connectors are designed to emit JSON data type values as a io.debezium.data.Json semantic type that encodes the JSON value as a string. However, this may not always be the desired outcome when sinking changes to a Milvus sink using Debezium Server.

A new configuration property, debezium.sink.milvus.unwind.json, has been added that can be set to either true or false (the default). When this property is set to true, the JSON string value will be represented as a JsonObject instead (DBZ-8909).

Redis can skip heartbeat messages

Debezium source connectors are often configured with heartbeat events so that at no point during low activity periods that the offset information for the source becomes stale. However, for some sinks like Redis, these heartbeat events aren’t useful to be passed to the sink target.

A new configuration property, debezium.sink.redis.skip.heartbeat.messages, has been added that can be set to either true or false (the default). When this property is set to true, the Redis sink will skip emitting heartbeat events to the Redis target; however, the heartbeat events will continue to influence the management of stale offsets (DBZ-8911).

New Qdrant sink adapter

Qdrant is an open-source, high-performance, low-latency vector database designed to store and search high-dimensional vectors efficiently. It’s also pairs extremely well with embeddings generated from models like OpenAI, Hugging Face, or sentence-transformers. Some use cases for Qdrant include, but are not limited to, chatbots, document searches, image/video similarities, and more.

With Debezium 3.2, we’ve introduced a new sink adapter for Debezium Server, allowing you to consume changes and place those into Qdrant. To get started with the Qdrant sink, configure Qdrant in the application properties as follows:

debezium.sink.type=qdrant

There are a myriad of optional configurations that you may need. Please review the online documentation for more details.

Dynamic messaging keys with Azure Event Hubs

In order to use compacted topics with Azure event hubs, a messaging key is vital to keep topic size reasonable and to enable easier replaying of topic messages. Debezium 3.2 aligns the behavior of the Azure Event Hubs sink to mirror that of others like Kinesis that base the partition selection logic based on the record’s key (DBZ-9195).

Support headers in NATS Jetstream sink

When change events are modified as part of the transformation chain, a transform may elect to add headers to the event. Unfortunately, in prior builds of Debezium, those headers were not replicated into the NATS Jetstream.

Starting with Debezium 3.2, all headers that are added to an event during the transformation chain are now replicated into the NATS Jetstream. No configuration or changes are required, this should happen automatically by updating to Debezium 3.2.0.Final (DBZ-9171).

PubSub sink maximum buffer size reduced

When the PubSub sink consumes a large batch of changes, it is possible for the PubSub sink to exceed to the maximum allowed batch threshold of 10mb while sending the batch data, leading to delivery issues. To address this so that the user experience is streamlined, the batch.request.byte.threshold has been reduced to 9.5mb. This aligns with other connectors that interact with Google PubSub to guarantee batch delivery without exceeding the maximum threshold (DBZ-9144).

Enable hostname verification for Redis sinks

When using the Redis sink, you may want to have the sink verify the hostname as part of the SSL connection process. This can be done by supplying the ssl.hostnae.verification.enabled option as part of the sink configuration so that the host verification is performed. This option is also available for when using the Redis offset and schema history implementations as well (DBZ-9042).

Custom keystore/truststore for Redis sinks

Debezium 3.2 introduces several new configuration options enabling the use of custom key/trust store details for Redis sinks (DBZ-9082). The following shows an example of these new options:

debezium.sink.redis.ssl.truststore.path=/path/to/truststore.p12
debezium.sink.redis.ssl.truststore.password=secret
debezium.sink.redis.ssl.truststore.type=PKCS12
debezium.sink.redis.ssl.keystore.path=/path/to/keystore.p12
debezium.sink.redis.ssl.keystore.password=secret
debezium.sink.redis.ssl.keystore.type=PKCS12

Debezium Platform

Improve navigation and workflow for transformations

We have added several new features to the Debezium Management Platform interface for transformations (DBZ-8328), which includes:

  • A new main navigation menu option called Transforms.

  • Improved the pipeline creation workflow allowing transformations to be added in-flight.

  • Support for modifying and removing existing transformations.

We’d love your feedback on the new navigation workflow and improvements around transformations.

Basic notification channel support

Debezium’s notification subsystem is a powerful way to allow Debezium to inform interested consumers about connector state changes. Debezium 3.2 introduces basic support for notifications when deploying Debezium Server on Kubernetes with Debezium Platform. This basic supports focuses on providing the log channel as a way to consume notifications so that they can be observed in the Debezium Server pod logs (DBZ-9046).

Debezium AI

Introduce timeout for Ollama embedding models

We have added a new configuration property ollama.operation.timeout.ms for the Debezium AI Ollama model integration using the FieldToEmbedding transformation. This configuration property specifies the number of milliseconds that the model operation is allowed to execute for before the request is timed out. By default, the transformation waits 15 seconds, but can be adjusted accordingly (DBZ-8908).

Debezium OpenLineage Integration

OpenLineage is an open standard for metadata and lineage collection for modern data-driven systems. It defines not only a specification, but a set of libraries to track the flow of data through systems like change data capture and transformation pipelines, stream processing, and data warehouses.

For example, lets consider a relatively common CDC and Data Warehousing scenario:

  • Debezium’s PostgreSQL connector captures changes, those are written to Kafka.

  • Flink reads the Kafka topic(s), performs stream processing, and writes the results to S3

  • A dbt job models S3 data and pushes that to Snowflake for warehousing

By using OpenLineage, the following can happen:

  • Debezium emits metadata about the CDC system

  • Flink can report lineage from Kafka to S3

  • And dbt can report lineage from S3 to Snowflake

And within the lineage UI, you would be able to see end-to-end flow from PostgreSQL to Snowflake. This provides a foundation for traceability for audits, debugging, and compliance, while also identifying things such as impact when a CDC pipeline is interrupted, and so much more.

For more information about this integration, how it works, and why this matters, please take a few moments and read our colleague’s latest blog post about the work we’ve done to integrate with OpenLineage (DBZ-9020).

Debezium Quarkus Extension

The Debezium Quarkus Extension is one of our most recent additions to the Debezium portfolio with a goal of not only providing Debezium as a DevService within the Quarkus ecosystem, but to use Debezium with GraalVM.

This is currently an actively developed feature, with new features being added across minor and micro releases (DBZ-8902). Our current focus at the moment is an MVP based on the PostgreSQL connector, with more connectors being added soon. In the not to distant future, we would like to build Debezium Server based on this work, to bring the power of Debezium and GraalVM to the Debezium Platform.

Several new features we’ve introduced as part of Debezium 3.2 include:

  • Annotate bean methods with @Capturing for change event handlers (DBZ-8961)

  • CDI lifecycle events, i.e. started, polling, stopped, etc. (DBZ-8959)

  • Supporting notification events (DBZ-8964)

For more information about this effort, please read up on the design document. As we continue moving forward throughout Debezium 3.3, expect to see new blogs and features added.

Other fixes

The following are just a subset of some noteworthy additional changes and fixes in Debezium 3.2:

  • Events inserted during snapshot are being duplicated (DBZ-9006)

  • Oracle connector crashes on redo entry without SQL_REDO for temporary tables (DBZ-9199)

  • Upgrade PostgreSQL JDBC driver to 42.7.7 (DBZ-9200)

  • SQL Server connector doesn’t properly handle special characters in schema names (DBZ-9117)

  • The presence of "_" in the ORACLE table name caused the cdc to fail (DBZ-9131)

  • Postgres: log errors from keepalive thread for replication (DBZ-9161)

  • Fix truncate present in history topic if it is a skipped operation (DBZ-9162)

  • MySQL connector doesn’t properly handle special characters in database object names (DBZ-9168)

  • Align configuration for post processors to transforms, predicates (DBZ-9170)

  • Error validating connector with special character $ (DBZ-8660)

  • Error when converting table and column names to uppercase (DBZ-9017)

  • Support signals in UI frontend (DBZ-8422)

  • Replace Postgres txid_current() by pg_current_xact_id() (DBZ-9011)

  • Ingestion issues with Mongodb when empty [] or empty {} appear in the Json feed (DBZ-5920)

  • Incremental snapshot in-progress notification doesn’t contain full composite PK (DBZ-8207)

  • Connector errors.max.retries is ignored (DBZ-8711)

  • debezium/server mongodb org.apache.kafka.connect.errors.DataException: is not a valid field name (DBZ-8972)

  • Connection left in "idle in transaction" state when setting snapshot mode to initial only (DBZ-9003)

  • Default values may be misinterpreted as bind parameters due to nested quotes (DBZ-9040)

  • Blocking snapshot does not always resume streaming thread when task is in shutdown (DBZ-9055)

  • Postgres Reselector fails on serial primary keys (DBZ-9086)

  • Create emdeddings SMT extension for Hugging face (DBZ-8992)

  • Create emdeddings SMT extension for Voyage AI models (DBZ-8993)

In total, 213 issues were resolved in Debezium 3.2. 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:

Chris Cranford

Chris is a software engineer at Red Hat. He previously was a member of the Hibernate ORM team and now works on Debezium. He lives in North Carolina just a few hours from Red Hat towers.

   


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.