It’s my pleasure to announce the first release of the Debezium 2.1 series, 2.1.0.Alpha1!

The Debezium 2.1.0.Alpha1 release includes quite a number of bug fixes but also some noteworthy improvements and new features including but not limited to:

  • Support for PostgreSQL 15

  • Single Message Transformation (SMT) predicate support in Debezium engine

  • Capturing TRUNCATE as change event in MySQL table topics

  • Oracle LogMiner performance improvements

  • New Redis-based storage module

Let’s take a few moments and dive into some of these in more detail!

SMT predicate support in Debezium engine

Single Message Transformations (SMTs) are a critical part of a change event’s lifecycle and they can apply any number of messaging patterns to the emitted change event. For example, a database table may have a specific column that gets emitted as a part of Debezium’s change events, but you want this field to be excluded so that the field isn’t present in the persisted event inside Kafka. This can be done using a single message transformation (SMT).

However, Debezium emits a number of different event types such as heartbeat, schema change, and data change events. Each of these events have their own event structure and there may come a point where a specific SMT should only be applied to a specific event type or if a specific event has a certain criteria. One way to evaluate whether the SMT should be applied was to do this evaluation inside the SMT itself, checking its event type or all the criteria to see whether the SMT should be applied or if the SMT should return the event unchanged.

Kafka Connect later introduced a concept called predicates, which is where a set of external rules can be specified in the connector configuration and must be evaluated to determine whether the SMT should be fired for an event or whether the SMT is skipped. This has enormous benefits because it allows developers to write very specific transformations that focus on a singular mutation and its entirely up to the user to determine whether that SMT should be applied or not using predicates.

Starting in Debezium 2.1, the power of Single Message Transformation (SMT) predicates can be harnessed when using the Debezium Engine or Debezium Server. An example configuration might like this:

# Define the filter transformation, linking it to the IsFoo predicate/rule
debezium.transforms=Filter
debezium.transforms.Filter.type=org.apache.kafka.connect.transforms.Filter
debezium.transforms.Filter.predicate=IsFoo

# Define the IsFoo predicate/rule
debezium.predicates=IsFoo
debezium.predicates.IsFoo.type=org.apache.kafka.connect.transforms.predicates.TopicNameMatches
debezium.predicates.IsFoo.pattern=foo

With these additional debezium.predicates.* configuration properties, it is possible to define a set of rules that must be evaluated to determine whether the Filter SMT will be fired or skipped in the transformation chain. In the example above, the predicate checks to see whether the event’s topic name matches foo and if it does, the Filter transformation will be fired. If the topic name does not match foo, the Filter transformation is skipped.

To read more about applying Single Message Transformations (SMTs) selectively using predicates, see:

Capture MySQL TRUNCATE as change event

Debezium has supported the concept of emitting a change event to signal a TRUNCATE TABLE scenario for PostgreSQL and Oracle for quite a while. Starting with Debezium 2.1, this behavior has been extended to the MySQL connector.

By default, the connector configuration option, skipped.operations, automatically skips TRUNCATE events if they’re detected. This means that by default, there will not be anything emitted when the connector detects this pattern. In order to support emission of such events, the skipped.operations configuration property must be specified with a value of none or other operation types that do not include the t (truncate) type.

Once the connector is configured to emit events for TRUNCATE operations, a new data change event type will be emitted to the table topics. These event types signal that the table or collection has been truncated. The event’s payload will looking like this:

"payload": {
  "source": {
    ...
  },
  "op": "t",
  "ts_ms": 1465581029523
}

The most notable point here is that truncate events do not contain a before or after state.

New Redis-based storage module

Debezium recently modularized parts of its codebase around persisting offsets and schema history into a set of modules supporting File and Kafka -based implementation. In Debezium 2.1, a new module was introduced to support persisting to Redis data stores.

The following fully-qualified class names can be used to persist offsets or schema history to Redis data stores:

  • io.debezium.storage.redis.offset.RedisOffsetBackingStore

  • io.debezium.storage.redis.history.RedisSchemaHistory

If you have manually installed Debezium, be sure to include the debezium-storage-redis artifact on your classpath if it does not exist in order to gain access to these new implementations.

For information about what options can be configured with this new implementation, please see the source configuration section of the Debezium Server documentation and look for configuration options prefixed with:

  • debezium.source.offset.storage.redis.*

  • debezium.source.schema.history.internal.redis.*

Other fixes

There were quite a number of bugfixes and stability changes in this release, some noteworthy are:

  • Missing snapshot pending transactions DBZ-5482

  • Using snapshot.mode ALWAYS uses SCN from offsets DBZ-5626

  • MongoDB multiple tasks monitor misalignment DBZ-5629

  • UNIQUE INDEX with NULL value throws exception when lob.enabled is true DBZ-5682

  • Columns are not excluded when doing incremental snapshots DBZ-5727

  • NullPointerException thrown during snapshot of tables in Oracle source connector DBZ-5738

  • Hostname not available for load balanced ocp services in ARO DBZ-5753

  • Exclude Oracle Compression Advisor tables from capture to avoid infinite loop DBZ-5756

  • Message with LSN 'LSN{XYZ}' not present among LSNs seen in the location phase DBZ-5792

What’s Next?

So as we continue to work on Debezium 2.1, we’ve been able to include a number of the expected changes in today’s release, but we still do intend to deliver on a new Single Message Transformation (SMT) for generating change event deltas before the end of the year. There is also some much anticipated changes for Debezium UI, such as supporting editing of connector configurations and much more.

You can find this information and what else to expect as a part of Debezium in 2023 in our recently updated road map. We have quite a lot of new features planned for next year, and we would love to hear your feedback or suggestions on things that may not be on the roadmap you’d like to see. Be sure to get in touch with us on the mailing list if there is.

Until next time…​

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.