
When I started working on Debezium, two questions came to mind: Is it possible to build a native version of Debezium? Can I receive change data capture (CDC) events directly inside my microservice without relying on additional infrastructure?
This led us to work on a new Debezium stream: I’m excited to announce the first release of Debezium Extensions for Quarkus!
Introduction
Change Data Capture (CDC) has become a key pattern in modern microservices architectures, in some cases one of the best way to evolve them. However, adopting CDC often comes with infrastructure overhead especially in fast-moving environments.
The Debezium extension for Quarkus aims to change that by allowing developers to consume CDC events directly inside their Quarkus applications, like anything else. Just add the extension, configure your data source, and listen to changes as they happen.
Supported Connectors
The Postgres connector is the only supported connector at this time.
When should you use it?
-
when infrastructure is a business concern
-
when low memory footprint is important, thanks to Quarkus Native support
-
lightweight use cases like proof of concepts, developer environments, or monolith to microservices migration
-
organization is initially evaluating CDC
What it isn’t?
-
a replacement for Debezium Server or Kafka Connect that guarantee production-grade CDC
-
an out-of-the-box sink. You must manage what happens after the change events reach your application
-
Coupling CDC logic into the application may reduce flexibility and lead to technical debt
Let’s start: build a CDC microservice for Postgres
In this section, we will go through the creation of a Debezium listener that captures changes within a Quarkus application!
Prerequisite
-
JDK 21+ installed with
JAVA_HOME
configured -
Apache maven 3.9.9
-
Quarkus version 3.25.0
-
Docker or Podman
-
Optionally Mandrel or GraalVM installed and configured if you want to build a native executable
Setup a Quarkus Project
If you already have a Quarkus application, you can skip this step. Otherwise, the easiest way to create a new Quarkus Project is to open a terminal and run the following command:
mvn io.quarkus.platform:quarkus-maven-plugin:3.25.0:create \
-DprojectGroupId=org.acme \
-DprojectArtifactId=debezium-starter
it generates a maven project with some Quarkus boilerplate in ./debezium-starter
directory.
Add the extension to the project
In the project, add the Debezium extension dependency to your application’s pom.xml
in the dependencies
section. In this walkthrough, we’re going to add the Postgres-specific connector implementation, but there will be other database vendor-specific implementations in the future.
<dependency>
<groupId>io.debezium</groupId>
<artifactId>quarkus-debezium-postgres</artifactId>
<version>${version.debezium}</version>
</dependency>
Don’t forget to set the |
Configuring the extension
The Debezium extension is shipped with a Connector (in this case Postgres) that listen CDC events from the datasource but a minimal configuration is mandatory. The configuration parameters available for the connector are the same as those used in the Debezium Documentation with a prefix quarkus.debezium.*
.
The following includes the minimum configuration necessary for the extension when working with Postgres. These should be added to your src/main/resources/application.properties
:
# Debezium CDC configuration
quarkus.debezium.offset.storage=org.apache.kafka.connect.storage.MemoryOffsetBackingStore
quarkus.debezium.name=native
quarkus.debezium.topic.prefix=dbz
quarkus.debezium.plugin.name=pgoutput
quarkus.debezium.snapshot.mode=initial
In this guide, we will use the DevService support with the official Debezium image that includes CDC setup. This avoids the need to configure a datasource to the target database.
Example data
Indeed, it’s possible to add some sql script to the docker image for development purpose, so create a new file in src/main/resources/init.sql
with the following:
CREATE SCHEMA inventory;
CREATE TABLE inventory.products (
id INT PRIMARY KEY,
name VARCHAR(100),
description TEXT,
weight BIGINT
);
INSERT INTO inventory.products (id, name, description, weight) VALUES
(1, 'Laptop', 'High-performance ultrabook', 1250),
(2, 'Smartphone', 'Latest model with AMOLED display', 180),
(3, 'Coffee Mug', 'Ceramic mug with lid', 350);
in a new dev configuration file like src/main/resources/application-dev.properties
, you can configure Quarkus dev-services to use it:
quarkus.datasource.devservices.init-script-path=init.sql
Receiving change data capture events from Debezium
Let’s start by creating a Quarkus application bean that listens for change events called InventoryListener
.
In the following class, we have annotated a method with the @Capturing
annotation. This annotation is used without specifying a destination, so the method is called for each event that Debezium produces. This can be useful when you want to derive the routing or handling logic programmatically within a single change event handler method.
package org.acme;
import io.debezium.runtime.Capturing;
import io.debezium.runtime.CapturingEvent;
import jakarta.enterprise.context.ApplicationScoped;
import jakarta.inject.Inject;
import org.apache.kafka.connect.source.SourceRecord;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ApplicationScoped
public class InventoryListener {
private final Logger logger = LoggerFactory.getLogger(InventoryListener.class);
@Capturing()
public void products(CapturingEvent<SourceRecord> event) {
logger.info("capturing event for destination {} with data {}", event.destination(), event.record());
}
}
If you specify a destination
value in the @Capturing
annotation, Debezium will only route events to that method handler that matches the destination. In the example below, the method is only called if the destination topic for the event is dbz.inventory.products
.
@Capturing(destination = "dbz.inventory.products")
public void orders(CapturingEvent<SourceRecord> event) {
logger.info("capturing products for destination {} with data {}", event.destination(), event.record());
}
You can also map the change event to a given target object type. For example, the following ObjectMapperSerializer
implementation uses Jackson
to convert the change event into a Product
object type.
package org.acme;
import io.quarkus.debezium.engine.deserializer.ObjectMapperDeserializer;
public class ProductDeserializer extends ObjectMapperDeserializer<Product> {
public ProductDeserializer() {
super(Product.class);
}
}
public record Product(Integer id, String name, String description, BigDecimal weight) { }
The mapping for the deserializer must be defined in application.properties in this way:
# Deserializer mapping for Product
quarkus.debezium.capturing.product.destination=dbz.inventory.products
quarkus.debezium.capturing.product.deserializer=org.acme.ProductDeserializer
The deserializer mapping can be used in conjunction with the @Capturing
annotation. As shown here, the method will be called using the newly created mapped object instance rather than the SourceRecord that is traditionally passed to the capturing method handler.
@Capturing(destination = "dbz.inventory.products")
public void orders(CapturingEvent<Product> event) {
logger.info("capturing products for destination {} with data {}", event.destination(), event.record());
}
This pattern can be beneficial when you need to communicate with external services using domain types or even if you want to implement the outbox pattern as a sidecar.
Running the application
So if we run our application in development mode:
mvn quarkus:dev
you will see the snapshot events captured, taken from the example table.
You can insert/update a row in the database and check it in the log:
-
get the
container id
container_id=$(docker ps --filter "ancestor=quay.io/debezium/postgres:15" --format "{{.ID}}") && echo $container_id
-
update the product name from Laptop to Notebook
docker exec "$container_id" bash -c "PGPASSWORD=quarkus psql -U quarkus quarkus -c \"UPDATE inventory.products SET name = 'Notebook' where id = 1\""
Build a Native Application
If you have configured your environment correctly, you can generate a native application using:
mvn clean install -Dnative
but don’t forget to add in the application.properties
or in application-prod.properties
the configuration of your datasource. An example is shown below:
quarkus.datasource.db-kind=postgresql
quarkus.datasource.username=native
quarkus.datasource.password=native
quarkus.datasource.jdbc.url=jdbc:postgresql://localhost:5432/native
quarkus.datasource.jdbc.max-size=16
Going further
This quick guide shows how you can use Debezium in Quarkus to build data streaming applications with ease.
For the exhaustive list of features and configuration options, check the Reference documentation for Debezium Extension.
You can find a working example in the Debezium example repository.
Giovanni Panice
Core team member of Debezium, with experience building resilient, distributed systems in different domains. His focus is Debezium’s Server and Engine, including native builds and Quarkus integration. Outside of work, he co-organizes the Ticino Software Craft Meetup to promote knowledge sharing and continuous learning.

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.