As the winter chill settles in, and we transition to the festive holiday season, our dedicated team has been busy preparing a special gift for the Debezium community. I am excited to share the immediate release of Debezium 2.5.0.Final, just in time for the holiday celebrations. Get ready to unwrap this latest minor version, filled with holiday cheer and exciting new features!

Let’s take a moment and review all the new features, changes, and improvements that are included in Debezium 2.5, which includes 181 issues resolved by 37 unique contributors.

Breaking changes

While we try to avoid any potential breaking changes between minor releases, such changes are sometimes inevitable. The upgrade to Debezium 2.5 includes a total of 10 unique breaking changes:

MySQL
  • The MySQL community announced that MySQL 5.7 would enter its End of Life cycle at the end of October 2023 In accordance with this upstream community news, Debezium too is making adjustments, like other vendors, around this recent news. To that end, starting with Debezium 2.5, we will no longer be testing nor supporting MySQL 5.7 if full capacity, thus MySQL 5.7 enters what we call "best-effort" support. (DBZ-6874)

  • MySQL BIT datatype did not have an implicit length if any was not set. This is incorrect as the default length if none is provided is 1 (DBZ-7230).

MongoDB
  • In earlier versions of Debezium, a user could use the MongoDB connector and perform a snapshot of a collection on a specific shard when using a MongoDB sharded deployment. Unfortunately, while we understand that users may have taken advantage of this behavior, it was unintended and not supported. Debezium 2.5 removes this ability entirely, and it’s a step in the forward goal to deprecate and remove the replica-set streaming mode in MongoDB completely (DBZ-7139).

  • Default connection mode is changed from replica_set to sharded as a preparation step for its complete removal. The change will lead to invalidation of existing offsets and a silent re-execution of the initial snapshot. To prevent this a check was added that will fail the connector upon start if this would happen. The user can either set replica_set connection mode explicitly or can remove the existing offsets (DBZ-7272).

JDBC
  • There was a corner case reported where the JDBC sink connector was mistakenly writing fields that had null values to the target database with the default value instead of a value of NULL. This has been fixed in Debezium 2.5 (DBZ-7191).

Core
  • If you are presently using the CloudEvents converter to emit events that conform to the CloudEvents format, it’s important to note that the configuration option metadata.location was renamed to metadata.source. You will need to be sure to update your connector configurations to reflect this change with Debezium 2.5 and onward. (DBZ-7060)

  • Part of the team’s focus in Debezium 2.5 was to improve the Debezium Embedded Engine’s experience. With that goal in mind, we took this preview release as an opportunity to clean-up the embedded engine’s API.

  • If your usage of the Debezium Embedded Engine utilized any of the previously deprecated APIs on EmbeddedEngine, you will find those methods have since been removed. (DBZ-7100) The recommended path forward is to make sure that you’re using the DebeziumEngine interface provided by the debezium-api artifact.

  • The ComputePartition was an SMT that enabled users to compute the desired partition for which an event would be written in a Kafka topic. This single message transformation (SMT) was deprecated some time ago in favor of the new PartitionRouting single message transformation. Debezium 2.5 removes the old, deprecated ComputePartition, and users who may still be relying on this SMT will need to migrate to the new PartitionRouting SMT if you have not already done so (DBZ-7141).

  • The schema name prefix and letter casing for Cloud Event headers was not consistent with payload name. The schema name was aligned so both headers and payload share th same namespace and follow the same rules for letter casing (DBZ-7216).

Improvements and changes

In this section, we’re going to take a tour of all the new features and improvements in Debezium 2.5.

Core

Redis Schema History Retries now Limited

Debezium 2.5 introduces a new configuration option, schema.history.internal.redis.max.attempts designed to limit the number of retry attempts while connecting to a Redis database when it becomes unavailable, previously it simply retried forever. This new option defaults to 10 but is user configurable. (DBZ-7120)

Additional notifications for initial snapshots

Debezium’s notification subsystem provides a simple way to integrate external processes and applications with certain Debezium tasks, such as taking a snapshot. In prior versions, the notifications for initial snapshots were quite basic and provided details such as when the snapshot started, when each table started and concluded, and finally when the snapshot ended.

A recent contribution that was part of DBZ-6878 extends on this base functionality to provide enhanced details about the snapshot. For example, the IN_PROGRESS notification will provide additional details about what tables are being captured and which table is currently in-progress, shown below:

{
   "id":"6d82a3ec-ba86-4b36-9168-7423b0dd5c1d",
   "aggregate_type":"Initial Snapshot",
   "type":"IN_PROGRESS",
   "additional_data":{
      "connector_name":"my-connector",
      "data_collections":"table1, table2",
      "current_collection_in_progress":"table1"
   },
   "timestamp": "1695817046353"
}

Additionally, another initial snapshot notification called TABLE_SCAN_COMPLETED also provides similar details, as shown below:

{
   "id":"6d82a3ec-ba86-4b36-9168-7423b0dd5c1d",
   "aggregate_type":"Initial Snapshot",
   "type":"TABLE_SCAN_COMPLETED",
   "additional_data":{
      "connector_name":"my-connector",
      "data_collection":"table1, table2",
      "scanned_collection":"table1",
      "total_rows_scanned":"100",
      "status":"SUCCEEDED"
   },
   "timestamp": "1695817046353"
}
Several fields shown above such as data_collection, these are not currently available for MongoDB snapshots and are only available for SQL-based relational connectors.

For more details, please see the Notification documentation.

Re-select columns

In some cases, because of the way that certain source databases function, when a Debezium connector emits a change event, the event might exclude values for specific column types. For example, values for TOAST columns in PostgreSQL, LOB columns in Oracle, or Extended String columns in Oracle Exadata, might all be excluded.

Debezium 2.5 introduces the ReselectColumnsPostProcessor providing a way to re-select one or more columns from a database table and fetch the current state. You can configure the post processor to re-select the following column types:

  • null columns.

  • columns that contain the unavailable.value.placeholder sentinel value.

Configuring a PostProcessor is similar to configuring a CustomConverter or Transformation, except that it works on the mutable payload’s Struct rather than the SourceRecord.

INSERT/DELETE semantics for incremental snapshot watermarking

The property incremental.snapshot.watermarking.strategy has been introduced to let users choose the watermarking strategy to use during an incremental snapshot.

The insert_insert (old behavior) approach lets Debezium creating two entries in the signaling data collection for each chunk during the snapshot to signal the opening of the snapshot window and another to mark its closure.

On the other hand, with the insert_delete option, a single entry is written in the signaling data collection for each chunk at the beginning of the window. After completion, this entry is removed, and no corresponding entry is added to signify the closure of the snapshot window. This method aids in more efficient management of the signaling data collection.

For more details, please see the Connector properties section of the connector of your interest.

MongoDB

Seamless large document handling

Debezium has introduced several changes around large document processing in recent releases; however, those changes primarily focused on handling that use case with MongoDB 4 and 5. While these improvements certainly help for those older versions, the MongoDB community has introduced a way in MongoDB 6 to seamlessly deal with this at the database pipeline level.

Debezium 2.5’s MongoDB connector now uses the $changeStreamSplitLargeEvent aggregation feature, introduced as part of MongoDB 6.0.9. This avoids the BSONObjectTooLarge exception when working with documents that would exceed the 16MB document size limit of MongoDB. This new feature is controlled by the oversize.handling.mode option, which defaults to fail. Please adjust this configuration if you would like to take advantage of this new, opt-in feature. (DBZ-6726)

Debezium is simply utilizing an underlying feature of the MongoDB database. As such, the database still has some limitations discussed in the MongoDB documentation that could still lead to exceptions with large documents that don’t adhere to MongoDB’s split rules.

MySQL

8.2 support

The MySQL community recently released a new innovation release, MySQL 8.2.0 at the end of October 2023. This new release has been tested with Debezium, and we’re happy to announce that we officially support this new innovation release. (DBZ-6873)

High-precision source timestamps

Several new high-precision timestamp fields have been included in several MySQL replication events. For example, in MySQL 8.0.1, there are to microsecond-resolution timestamps added to GTID events that specify the timestamp when the transaction was committed on the immediate primary and when committed on the original primary.

Debezium 2.5 now takes advantage of these values and will use these for the ts_ms fields if they’re available, falling back to the second-based precision if they are not or if you’re using a version of MySQL prior to 8.0.1 (DBZ-7183).

PostgreSQL

Streaming from PostgreSQL 16 stand-bys

In PostgreSQL 16, you can now define replication slots on a stand-by instance. This enables a plethora of new options, including the ability to perform change data capture from a replica rather than your production system for load distribution, particularly in a very active database.

Debezium 2.5 now supports connecting to a stand-by PostgreSQL 16 server and streaming changes (DBZ-7181).

Oracle

Streaming Metrics Changes

In previous builds of Debezium, there was a single Oracle streaming metrics bean that exposed all metrics options that spanned across all three streaming adapters. This often lead to some confusion about what metrics are applicable to which streaming adapter so we wanted to define a clear distinction in this case.

With Debezium 2.5, the Oracle streaming metrics beans have been split into three different implementations, one for each adapter type. For observability stacks, this change should be completely transparent unless you were previously gathering a metric for one adapter type while using another. In this case you’ll find that metric is no longer available.

Specifically for LogMiner users, several metrics have been renamed and the old metrics have been deprecated. While you will still be able to use the old metric names in Debezium 2.5, these are scheduled for removal in a future 2.7+ build. The metrics that were deprecated and renamed are as follows:

Old/Deprecated Metric New Metric

CurrentRedoLogFileName

CurrentLogFileNames

RedoLogStatus

RedoLogStatuses

SwitchCounter

LogSwitchCounter

FetchingQueryCount

FetchQueryCount

HoursToKeepTransactionInBuffer

MillisecondsToKeepTransactionsInBuffer

TotalProcessingTimeInMilliseconds

TotalBatchProcessingTimeInMilliseconds

RegisteredDmlCount

TotalChangesCount

MillisecondsToSleepBetweenMiningQuery

SleepTimeInMilliseconds

NetworkConnectionProblemsCounter

No replacement

LOB behavior

Debezium 2.5 aligns LOB behavior in snapshot and streaming. When lob.enabled is set to false, the unavailable value placeholder will be explicitly included during snapshot to match the behavior of streaming.

SQL Server

Notification Improvements

Debezium for SQL Server works by reading the changes captured by the database in what are called capture instances. These instances can come and go based on a user’s needs, and it can be difficult to know if Debezium has concluded its own capture process for a given capture instance.

Debezium 2.5 remedies this problem by emitting a new notification aggregate called Capture Instance, allowing any observer to realize when a capture instance is no longer in use by Debezium. This new notification includes a variety of connector details including the connector’s name along with the start, stop, and commit LSN values. (DBZ-7043)

Driver Updates

SQL Serer 2019 introduced the ability to specify column-specific sensitivity classifications to provide better visibility and protections for sensitive data. Unfortunately, the current driver shipped with Debezium 2.4 and earlier does not support this feature. Debezium 2.5 introduces the latest 12.4.2 SQL Server driver so that users can now take advantage of this feature out of the box. (DBZ-7109)

JDBC sink

Batch Support

Debezium first introduced the JDBC sink connector in March 2023 as a part of Debezium 2.2. Over the last several months, this connector has seen numerous iterations to improve its stability, feature set, and capabilities. Debezium 2.5 builds atop of those efforts, introducing batch-writes. (DBZ-6317)

In previous versions, the connector worked on each topic event separately; however, the new batch-write support mode will collect the events into buckets and write those changes to the target system using the fewest possible transaction boundaries as possible. This change increases the connector’s throughput capabilities and makes the interactions with the target database far more efficient.

Field inclusion/exclusion

Debezium 2.5 introduces a new JBDC sink feature where users can now specify which fields from the event payload are to be included or excluded from the target database write operation. This feature works just like any other include/exclude combination in the Debezium framework where these two properties are mutually exclusive.

As an example, if we have a simple event payload with the following fields in topic customers:

{
  "id": 12345,
  "name": "Acme",
  "address": "123 Main Street"
}

If we want to avoid writing the address field to the target database and only write the id and name fields to the target table, we can use this new feature to accomplish this. This can be done adding either a field.include.list or field.exclude.list property.

Example that prevents writing the address field to target
{
  "field.exclude.list": "customers:address"
}

The format of the include/exclude for fields is [<topic-name>:]<field-name>, where the topic-name is optionally and can be omitted if you want to avoid writing the address field for all events. Please see the JDBC sink connector configuration documentation for more details.

Debezium Server

Operator

The Debezium Server Operator for Kubernetes has been actively improved in this preview release of Debezium 2.5. Several improvements include:

  • Ability to set image pull secrets in the CRDs DBZ-6962

  • Ability to set resource limits in the CRDs DBZ-7052

  • Published OLM bundle scripts to Maven Central DBZ-6995

  • Support OKD/OpenShift catalog in OperatorHub release script DBZ-7010

  • Display name and descriptions metadata available in OLM bundle DBZ-7011

  • New metrics endpoint for gathering metrics DBZ-7053

As we continue to improve the Debezium Server Operator for Kubernetes, we’d love to get your feedback.

Service Account for CRDs

In previous versions of Debezium, it was not possible to use a service account named differently than the predefined one. This made the process a tad bit cumbersome for users because while you could grant roles and authorization to this predefined account separately, it meant you needed to use this predefined service account rather than one that you may already wish to use.

Debezium 2.5 simplifies this process, allowing you to now use your own, custom service account (DBZ-7111).

Kinesis Sink Improvements

Debezium Server Kinesis users will be happy to note that there has been some reliability improvements with the sink adapter with Debezium 2.5. The Kinesis Sink will now automatically retry the delivery of a failed record up to a maximum of 5 attempts before the adapter triggers a failure. This should improve the sink adapter’s delivery reliability and help situations where a batch of changes may overload the sink’s endpoint. (DBZ-7032)

EventHubs partitioning

In earlier versions of Debezium Server, users could specify a fixed partition-id to stream all changes to a single partition or provide a static partition-key that will be set on all batch operations, which ultimately lends itself to streaming all changes to the same target partition. There are situations where this may be helpful, but it more often leads to a performance concerns for downstream processing.

Debezium 2.5 adjusts this behavior in order to improve performance. By default, when neither a partitionid or partitionkey is defined, the EventHub sink will send events using a round-robin technique to all available partitions. Events can be forced into a single, fixed partition by specifying a partitionid. Alternatively, the partitionkey can be provided to supply a fixed partition key that will be used to route all events to a specific partition.

If additional partition routing requirements are necessary, you can now combine the PartitionRouting SMT accomplish such tasks. For more details, please see the Event Hubs documentation.

RabbitMQ Streams sink

RabbitMQ introduced Streams in version 3.9, which utilizes a fast and efficient protocol that can be combined with AMQP 0.9.1 to support large fan-outs, replay and time travel, and large data sets with extremely high throughput. Debezium 2.5 takes advantage of this new Streams implementation by introducing a new native Streams implementation (DBZ-6703). In order to get started with this new implementation, configure the Debezium Server sink as follows:

debezium.sink.type=rabbitmqstream
debezium.sink.rabbitmqstream.connection.host=<hostname of RabbitMQ>
debezium.sink.rabbitmqstream.connection.port=<port of RabbitMQ>

Additionally, if you need to pass any other connection parameters to the RabbitMQ connection, you can do so by adding those to the configuration with the prefix debezium.sink.rabbitmqstream.connection. to pass through any config properties.

Please see the Debezium Server RabbitMQ documentation for more details.

StreamNameMapper for Apache Kafka sink

The Kafka sink behaviour can now be modified by a custom logic providing alternative implementations for specific functionalities. When the alternative implementations are not available then the default ones are used.

For more details, please see the Apache Kafka Injection points.

AWS SQS sink

Amazon Simple Queue Service (Amazon SQS) is a distributed message queuing service. It supports programmatic sending of messages via web service applications as a way to communicate over the Internet. SQS is intended to provide a highly scalable hosted message queue that resolves issues arising from the common producer–consumer problem or connectivity between producer and consumer.

Debezium 2.5 offers the possibility to send events to Amazon SQS.

Spanner

  • Support for Cloud Spanner emulator with the Spanner connector DBZ-6845

  • Resumable snapshot support for the Vitess connector DBZ-7050

Informix

IBM Informix Connector

Thanks to the contribution from Lars Johansson, Debezium 2.5 introduces a new connector to its portfolio to gather changes from IBM Informix. IBM Informix is an embeddable, high-performance database for integrating SQL, NoSQL, JSON, time-series, and spatial data in one place. It’s designed for analytics at the edge, in the cloud, or on premise.

The IBM Informix connector is bundled just like any of our community lead connectors, it is available on Maven Central or you can download the plug-in archive from our Debezium 2.5 releases page.

The maven artifact coordinates are:

<dependency>
    <groupId>io.debezium</groupId>
    <artifactId>debezium-connector-informix</artifactId>
    <version>2.5.0.Final</version>
</dependency>

If you would like contribute to the Informix connector, we have added a new repository under the Debezium organization, debezium-connector-informix.

I’d like to thank Lars Johansson for this contribution and his collaboration with the team, kudos!

MariaDB

Preview support

The community has leveraged the MySQL connector as an alternative to capture changes from MariaDB for quite some time now; however that compatibility was primarily best-case effort.

The Debezium 2.5 release stream aims to bring MariaDB to the forefront as a first-class connector by taking a very clear and methodological approach to incrementally check, validate, and eventually support MariaDB at the same capacity that we do MySQL. Our goal and hope is that we can do this within the scope of the MySQL connector proper; however, there is still quite a bit of ongoing investigation around GTID support that may influence the path forward.

This first preview build of Debezium 2.5 has taken the first step, we’ve verified that the code works against a single MariaDB database deployment, the test suite passes and we’ve addressed any changes needed with the Binlog client to support that deployment. Our next steps is to look into GTID support, which MariaDB supports but using an approach that isn’t compatible with MySQL.

Stay tuned for future builds as we continue to expand on this and we certainly welcome any early feedback.

GTID support

Both MySQL and MariaDB support what is called Global Transaction Identifiers or GTIDs. These are used in replication to uniquely identify transaction(s) uniquely across a cluster. The implementation details between MySQL and MariaDB differ significantly and in earlier versions of Debezium, we only supported GTID with MySQL.

With Debezium 2.5, we are taking another step forward by introducing GTID support for MariaDB as part of the MySQL connector offering. In order to take advantage of this behavior, you will need to use the MariaDB driver rather than the MySQL driver by using a JDBC connection prefixed as jdbc:mariadb rather than jdbc:mysql. By doing this, you can now take full advantage of working with MariaDB and GTID just like MySQL (DBZ-1482).

Other changes

Altogether, 7 issues were fixed in this release and a total of 169 issues across all the Debezium 2.5 releases.

  • Adding Debezium Server example using MySQL and GCP PubSub DBZ-4471

  • Refactor ElapsedTimeStrategy DBZ-6778

  • Multiple debezium:offsets Redis clients DBZ-6952

  • Wrong case-behavior for non-avro column name in sink connector DBZ-6958

  • Handle properly bytea field for jdbc sink to postgresql DBZ-6967

  • Debezium jdbc sink process truncate event failure DBZ-6970

  • Single quote replication includes escaped quotes for N(CHAR/VARCHAR) columns DBZ-6975

  • Provide configuration option to exclude extension attributes from a CloudEvent DBZ-6982

  • Debezium jdbc sink should throw not supporting schema change topic exception DBZ-6990

  • Debezium doesn’t compile with JDK 21 DBZ-6992

  • OLM bundle version for GA releases is invalid DBZ-6994

  • Further refactoring to correct downstream rendering of incremental snapshots topics DBZ-6997

  • ALTER TABLE fails when adding multiple columns to JDBC sink target DBZ-6999

  • Invalid Link to zulip chat in CSV metadata DBZ-7000

  • Make sure to terminate the task once connectivity is lost to either the rebalance or sync topic DBZ-7001

  • Missing .metadata.annotations.repository field in CSV metadata DBZ-7003

  • Single quote replication and loss of data DBZ-7006

  • Remove deprecated embedded engine code DBZ-7013

  • Enable replication slot advance check DBZ-7015

  • Add configuration option to CloudEventsConverter to retrieve id and type from headers DBZ-7016

  • Oracle connector: Payload size over 76020 bytes are getting truncated DBZ-7018

  • Use optional schema for Timezone Converter tests DBZ-7020

  • DDL statement couldn’t be parsed DBZ-7030

  • Blocking ad-hoc snapshot is not really blocking for MySQL DBZ-7035

  • Fake ROTATE event on connection restart cleans metadata DBZ-7037

  • Consolidate resource labels and annotations DBZ-7064

  • Oracle RAC throws ORA-00310: archive log sequence required DBZ-5350

  • oracle missing CDC data DBZ-5656

  • Missing oracle cdc records DBZ-5750

  • Add (integration) tests for Oracle connector-specific Debezium Connect REST extension DBZ-6763

  • Intermittent failure of MongoDbReplicaSetAuthTest DBZ-6875

  • Connector frequently misses commit operations DBZ-6942

  • Missing events from Oracle 19c DBZ-6963

  • Mongodb tests in RHEL system testsuite are failing with DBZ 2.3.4 DBZ-6996

  • Use DebeziumEngine instead of EmbeddedEngine in the testsuite DBZ-7007

  • Debezium Embedded Infinispan Performs Slowly DBZ-7047

  • Field exclusion does not work with events of removed fields DBZ-7058

  • Update transformation property "delete.tombstone.handling.mode" to debezium doc DBZ-7062

  • JDBC sink connector not working with CloudEvent DBZ-7065

  • JDBC connection leak when error occurs during processing DBZ-7069

  • Some server tests fail due to @com.google.inject.Inject annotation DBZ-7077

  • Add MariaDB driver for testing and distribution DBZ-7085

  • Allow DS JMX to use username-password authentication on k8 DBZ-7087

  • HttpIT fails with "Unrecognized field subEvents" DBZ-7092

  • MySQL parser does not conform to arithmetical operation priorities DBZ-7095

  • VitessConnectorIT.shouldTaskFailIfColumnNameInvalid fails DBZ-7104

  • When RelationalBaseSourceConnector#validateConnection is called with invalid config [inside Connector#validate()] can lead to exceptions DBZ-7105

  • Debezium crashes on parsing MySQL DDL statement (specific INSERT) DBZ-7119

  • Generate sundrio fluent builders for operator model DBZ-6550

  • Convert operator source into multi module project DBZ-6551

  • Implement "validate filters" endpoint in connector-specific Connect REST extensions DBZ-6762

  • Test Avro adjustment for MongoDb connector and ExtractNewDocumentState SMT DBZ-6809

  • Implement IT tests against Cloud Spanner emulator in main repo. DBZ-6906

  • The DefaultDeleteHandlingStrategy couldn’t add the rewrite "__deleted" field to a non-struct value DBZ-7066

  • Implement strategy pattern for MariaDB and MySQL differences DBZ-7083

  • Debezium server has no default for offset.flush.interval.ms DBZ-7099

  • Failed to authenticate to the MySQL database after snapshot DBZ-7132

  • Run MySQL CI builds in parallel DBZ-7135

  • Failure reading CURRENT_TIMESTAMP on Informix 12.10 DBZ-7137

  • Debezium-ddl-parser crashes on parsing MySQL DDL statement (specific UNION) DBZ-7140

  • outbox.EventRouter SMT throws NullPointerException when there is a whitespace in fields.additional.placement value DBZ-7142

  • Debezium-ddl-parser crashes on parsing MySQL DDL statement (specific UPDATE) DBZ-7152

  • Add matrix strategy to workflows DBZ-7154

  • Add Unit Tests for ServiceAccountDependent Class in Debezium Operator Repository DBZ-7155

  • JsonSerialisation is unable to process changes from sharded collections with composite sharding key DBZ-7157

  • Log sequence check should treat each redo thread independently DBZ-7158

  • Fail fast during deserialization if a value is not a CloudEvent DBZ-7159

  • Correctly calculate Max LSN DBZ-7175

  • Upgrade to Infinispan 14.0.20 DBZ-7187

  • Upgrade Outbox Extension to Quarkus 3.5.3 DBZ-7188

  • Fix DebeziumMySqlConnectorResource not using the new MySQL adatper structure to support different MySQL flavors DBZ-7179

  • Parsing MySQL indexes for JSON field fails, when casting is used with types double and float DBZ-7189

  • Unchanged toasted array columns are substituted with unavailable.value.placeholder, even when REPLICA IDENTITY FULL is configured. DBZ-7193

  • Enable ability to stream changes against Oracle 23c for LogMiner DBZ-7194

  • Add modify range_partitions to modify_table_partition rule in parsing PL/SQL DBZ-7196

  • MongoDB streaming pauses for Blocking Snapshot only when there is no event DBZ-7206

  • Handle Drop Tablespace in PL/SQL DBZ-7208

  • Upgrade logback to 1.2.12 DBZ-7209

  • NPE on AbstractInfinispanLogMinerEventProcessor.logCacheStats DBZ-7211

  • Oracle abandoned transaction implementation bug causes OoM DBZ-7236

  • Add Grammar Oracle Truncate Cluster DBZ-7242

  • Length value is not removed when changing a column’s type DBZ-7251

  • MongoDB table/collection snapshot notification contain incorrect offsets DBZ-7252

  • Broken support for multi-namespace watching DBZ-7254

  • Add tracing logs to track execution time for Debezium JDBC connector DBZ-7217

  • Validate & clarify multiple archive log destination requirements for Oracle DBZ-7218

  • Upgrade logback to 1.2.13 DBZ-7232

  • Add configuration option to CloudEventsConverter to customize schema type name DBZ-7235

  • Support persistent history for snapshot requests for the kafka signal topic. DBZ-7164

  • Change metrics endpoint of Connect REST Extensions to use the MBeanServerv directly instead of HTTP calls to the Jolokia endpoint DBZ-7177

  • Metrics endpoint must handle connectors with multiple tasks (SQL Server) DBZ-7178

  • DDL GRANT statement couldn’t be parsed DBZ-7213

  • Debezium Oracle plugin 2.5.0 Beta does not support Oracle 11g DBZ-7257

  • Error during snapshot with multiple snapshot threads will not properly abort snasphostting DBZ-7264

  • MySQL RDS UPDATE queries not ignored DBZ-7271

  • Leaking JDBC connections DBZ-7275

  • IncrementalSnapshotCaseSensitiveIT#insertDeleteWatermarkingStrategy fails DBZ-7276

  • Debezium MySQL could not parse certain grant privileges. DBZ-7277

  • Add PL/SQL Parser for Create Table Memoptimize DBZ-7279

  • Support for Creating EDITIONABLE or NONEDITIONABLE Packages DBZ-7283

  • Add PL/SQL Parser for Alter Table Memoptimize DBZ-7268

  • Move metrics endpoint from UI backend to the Debezium Connect REST extension/s DBZ-6764

  • website-builder image fails with newer bundler DBZ-7269

  • Vitess connector build fails due to invalid GPG key DBZ-7280

Outlook & What’s next?

Debezium 2.5 was a feature packed milestone for the team, so after a few drinks and celebration, the plan is to turn our focus toward what is ahead for the 2.6 release. We already had our second Debezium Community meeting, discussed our road map, and we’re more than eager to get started.

If you have any ideas or suggestions for what you’d like to see included in Debezium 2.6, please provide that feedback on our mailing list or in our Zulip chat.

Merry Christmas and Happy New Year 2024!

Onwards and Upwards!

Fiore Mario Vitale

Mario is a Senior Software Engineer at Red Hat. He lives in Italy.

   


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.