Did you know January 16th is National Nothing Day? It’s the one day in the year without celebrating, observing or honoring anything.
Well, normally, that is. Because we couldn’t stop ourselves from sharing the news of the Debezium 1.1.0.Alpha1 release with you! It’s the first release after Debezium 1.0, and there are some really useful features coming with it. Let’s take a closer look.
Quarkus Outbox Pattern Extension
The outbox pattern is a great way for letting services update their own database (e.g. to persist a new purchase order) as well as emitting corresponding events to other services in a reliable and consistent way. The pattern avoids unsafe dual writes, but also doesn’t require distributed transactions.
Debezium already supports this pattern via its outbox event router, which can be used to stream events from one outbox table into different topics in Apache Kafka. As we saw a growing adoption of the pattern and usage of this router, we wanted to take things to the next level and also provide a component to simplify the creation of outbox events within your application itself.
For that purpose there’s now a brand-new extension for Quarkus, a "Kubernetes Native Java stack tailored for OpenJDK HotSpot and GraalVM, crafted from the best of breed Java libraries and standards". Using this extension it becomes as simple as firing plain CDI events to produce outbox events:
@ApplicationScoped
public class OrderService {
@Inject
Event<ExportedEvent<?, ?>> event;
@Transactional
public PurchaseOrder addOrder(PurchaseOrder order) {
// process and persist the order...
// create a corresponding outbox event to notify other services
event.fire(OrderCreatedEvent.of(order));
return order;
}
}
Once the event has been persisted in the outbox table and the transaction has been committed, Debezium will capture it and propagate it to downstream consumers via Kafka.
We’re planning to publish a detailed post with a complete end-to-end example for implementing the outbox pattern with this Debezium and this Quarkus extension next week, so stay tuned. To read up on the outbox pattern in the meantime, please refer to the blog post linked above or to any of these great posts from the community.
CloudEvents Support
CloudEvents is a "specification for describing event data in a common way", aiming at providing "interoperability across services, platforms and systems". By means of a new Kafka Connect message converter (io.debezium.converters.CloudEventsConverter
), Debezium now can emit change events that adhere to the CloudEvents specification:
{
"id" : "name:test_server;lsn:29274832;txId:565",
"source" : "/debezium/postgresql/test_server",
"specversion" : "1.0",
"type" : "io.debezium.postgresql.datachangeevent",
"time" : "2020-01-13T13:55:39.738Z",
"datacontenttype" : "application/json",
"iodebeziumop" : "r",
"iodebeziumversion" : "1.1.0.Alpha1",
"iodebeziumconnector" : "postgresql",
"iodebeziumname" : "test_server",
"iodebeziumtsms" : "1578923739738",
"iodebeziumsnapshot" : "true",
"iodebeziumdb" : "postgres",
"iodebeziumschema" : "s1",
"iodebeziumtable" : "a",
"iodebeziumtxId" : "565",
"iodebeziumlsn" : "29274832",
"iodebeziumxmin" : null,
"data" : {
"before" : null,
"after" : {
"pk" : 1,
"name" : "Bob"
}
}
}
With CloudEvents, each event contains a few defined attributes such as id
, source
and type
. The actual event payload can be found in the data
attribute, which in the case of Debezium is the structure of old and new state of the affected database record. Most of the other Debezium change events attributes (op
, timestamp, source metadata) are mapped to custom attributes using the iodebezium
prefix.
The data
attribute as well as the entire event can be encoded using JSON or Avro. Initially, only the "structured mode" of CloudEvents is supported, i.e. all the attributes are part of the event structure, which is the Kafka record value in this case. In a future release we’ll also add support for the CloudEvents "binary mode", where only the data
attribute is part of the event structure, while all other attributes will be mapped the (Kafka) header attributes.
CloudEvents support is under active development, so details around the format likely will change in future versions as this feature matures. We’d love to get your feedback on this and learn from your insights and experiences with CloudEvents.
Further Changes
Besides these two larger features, a number of smaller improvements and fixes has been done for Debezium 1.1 Alpha1:
Altogether, 17 issues were fixed for this release.
Many thanks to Andrea Cosentino, Vasily Ulianko, Vedit Firat Arig, Yongjun Du and Yuchao Wang for their contributions to this release!
Going forward, we’ll continue with further Debezium 1.1 preview releases every two to three weeks. Take a look at the roadmap to see what’s coming up, or get in touch to tell us about your specific feature requirements!
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.