It’s my pleasure to announce the release of Debezium 1.1.0.CR1!

This release brings a brand-new API module, including a facility for overriding the schema and value conversion of specific columns. The Postgres connector gained the ability to reconnect to the database after a connection loss, and the MongoDB connector supports the metrics known from other connectors now.

The Debezium API Module

In most cases Debezium users don’t interact with Debezium programmatically, but rather configure and deploy the different connectors via Kafka Connect. There are some exceptions, though, most notably the Debezium embedded engine. Also we’re expanding Debezium by adding SPIs like the one for customizing schemas and values (see below). This raises the question, which packages and classes within Debezium are meant for public access by users, and which ones not.

To clarify that split between Debezium’s public Java API and its internal implementation packages, a new API module has been introduced. For compilation, user code should only depend on this module, whereas the debezium-embedded and debezium-core artifacts should be runtime dependencies.

E.g. the following set-up should be used with Apache Maven:

<dependency>
    <groupId>io.debezium</groupId>
    <artifactId>debezium-api</artifactId>
    <version>${version.debezium}</version>
</dependency>
<dependency>
    <groupId>io.debezium</groupId>
    <artifactId>debezium-embedded</artifactId>
    <version>${version.debezium}</version>
    <scope>runtime</scope>
</dependency>

While backwards compatible evolution will be ensured for the API module, breaking changes in the implementation modules may be done at any time. So you should make sure your application code doesn’t rely on these.

With the introduction of the API module, also the bootstrap API for the embedded engine has been reworked.

DebeziumEngine<SourceRecord> engine = DebeziumEngine.create(Connect.class)
    .using(config)
    .notifying(record -> {
        System.out.println(record);
    })
    .build();

Event handlers still receive the Kafka Connect SourceRecord type for now, but going forward we’ll also support event handlers working with other representations such as JSON or Avro.

Schema and Value Customization SPI

A question that has come repeatedly in the chat and on the mailing list is how to customize the schema and value of specific change event fields. For instance you might want to export temporal column values as a specifically formatted string instead of the default representation (a long value representing milli-seconds since epoch). The answer to this has been to implement a custom message transformation (SMT) for Kafka Connect. As they can add some overhead in terms of through-put though (in particular when dealing with more complex schemas), we’ve added a new extension point to Debezium itself, which allows you to override the schema and value conversion for specific columns of captured tables.

As an example, the following converter could be implemented to export Postgres ISBN column values with a specific logical field type "isbn" instead of the default "string" schema:

public class IsbnConverter implements CustomConverter<SchemaBuilder, RelationalColumn> {

    private SchemaBuilder isbnSchema;

    // could evaluate config here
    @Override
    public void configure(Properties props) {
        isbnSchema = SchemaBuilder.string().name("isbn");
    }

    @Override
    public void converterFor(RelationalColumn column,
            ConverterRegistration<SchemaBuilder> registration) {

        // register custom schema and value conversion for "isbn" columns
        if ("isbn".equals(column.typeName())) {
            registration.register(isbnSchema, x -> x.toString());
        }
    }
}

In order to use such custom converter, compile it and add the JAR to the plug-in directory of the connector. Configure it like so:

converters=isbn
isbn.type=com.example.IsbnConverter
# custom options as needed
isbn.foo=bar

Other Features and Changes

The MongoDB connector has been migrated to the common CDC connector framework we’ve been started to develop for the SQL Server and Oracle connectors. This is going to significantly simplify the maintenance of the code base for us, as many cross-cutting features can be implemented in one central place. As an example, the MongoDB connector now supports the metrics you already can use with the relational connectors (DBZ-845), allowing you to monitor that connector in production.

The Postgres connector supports automatic reconnects now in case the database connection has been lost (DBZ-1723). Again this has been largely implemented as a generic facility, the Postgres connector is the first one to make use of this. We’ve seen increased reports of Postgres disconnects in specific environments, which is why we decided this to be the first connector to support reconnects.

When streaming change events to other relational databases, the column.propagate.source.type connector option comes in handy for propagating the exact column definition as a schema header. So far, this was to be configured per each individual column; as of this release, this option can be set globally for column types, drastically reducing the need for configuration when working with many columns of a specific type whose schema information should be exported (DBZ-1830).

Another improvement for MongoDB users is the new support for exporting information about the sharding key in update and delete events (DBZ-1781).

As always, a good number of bugs was fixed, too. Overall, 44 issues were addressed for Debezium 1.1.0.CR1. Please refer to the release notes for more details.

An open-source project like Debezium would be nothing without its community of contributors. The following people have contributed to this release: Alan Zhangzf, Fabio Cantarini, Hossein Torabi, JanHendrikDolling, John Graf, Raúl Tovar and Ruslan Gibaiev. Thank you to each and eveyone of you!

With the CR1 release being done, it won’t be much longer until Debezium 1.1 Final. Depending on issues found with this release candidate, we might do a CR2 release, followed by the Final shortly thereafter. For plans for future versions please refer to the roadmap and let us know about your requirements. Our general plan is to adopt a cadence of quarterly minor releases, i.e. you can expect Debezium 1.2 in about three months from now, 1.3 in Q3 and so on.

Gunnar Morling

Gunnar is a software engineer at Decodable and an open-source enthusiast by heart. He has been the project lead of Debezium over many years. Gunnar has created open-source projects like kcctl, JfrUnit, and MapStruct, and is the spec lead for Bean Validation 2.0 (JSR 380). He’s based in Hamburg, Germany.

   


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.