You are viewing documentation for an unreleased version of Debezium.
If you want to view the latest stable version of this page, please go here.

Content-based routing

By default, Debezium streams all of the change events that it reads from a table to a single static topic. However, there might be situations in which you might want to reroute selected events to other topics, based on the event content. The process of routing messages based on their content is described in the Content-based routing messaging pattern. To apply this pattern in Debezium, you use the content-based routing single message transform (SMT) to write expressions that are evaluated for each event. Depending how an event is evaluated, the SMT either routes the event message to the original destination topic, or reroutes it to the topic that you specify in the expression.

While it is possible to use Java to create a custom SMT to encode routing logic, using a custom-coded SMT has its drawbacks. For example:

  • It is necessary to compile the transformation up front and deploy it to Kafka Connect.

  • Every change needs code recompilation and redeployment, leading to inflexible operations.

The content-based routing SMT supports scripting languages that integrate with JSR 223 (Scripting for the Java™ Platform).

Debezium does not come with any implementations of the JSR 223 API. To use an expression language with Debezium, you must download the JSR 223 script engine implementation for the language. For example, for Groovy 3, you can download its JSR 223 implementation from https://groovy-lang.org/. The JSR223 implementation for GraalVM JavaScript is available at https://github.com/graalvm/graaljs. After you obtain the script engine files, you add them to your Debezium connector plug-in directories, along any other JAR files used by the language implementation.

Set up

For security reasons, the content-based routing SMT is not included with the Debezium connector archives. Instead, it is provided in a separate artifact, debezium-scripting-3.0.5.Final.tar.gz.

To use the content-based routing SMT with a Debezium connector plug-in, you must explicitly add the SMT artifact to your Kafka Connect environment. IMPORTANT: After the routing SMT is present in a Kafka Connect instance, any user who is allowed to add a connector to the instance can run scripting expressions. To ensure that scripting expressions can be run only by authorized users, be sure to secure the Kafka Connect instance and its configuration interface before you add the routing SMT.

With Zookeeper, Kafka, Kafka Connect, and one or more Debezium connectors installed, the remaining tasks to install the filter SMT are:

  1. Download the scripting SMT archive

  2. Extract the contents of the archive into the Debezium plug-in directories of your Kafka Connect environment.

  3. Obtain a JSR-223 script engine implementation and add its contents to the Debezium plug-in directories of your Kafka Connect environment.

  4. Restart your Kafka Connect process to pick up the new JAR files.

The Groovy language needs the following libraries on the classpath:

  • groovy

  • groovy-json (optional)

  • groovy-jsr223

The JavaScript language needs the following libraries on the classpath:

  • graalvm.js

  • graalvm.js.scriptengine

Example: Basic configuration

To configure a Debezium connector to route change event records based on the event content, you configure the ContentBasedRouter SMT in the Kafka Connect configuration for the connector.

Configuration of the content-based routing SMT requires you to specify a regular expression that defines the filtering criteria. In the configuration, you create a regular expression that defines routing criteria. The expression defines a pattern for evaluating event records. It also specifies the name of a destination topic where events that match the pattern are routed. The pattern that you specify might designate an event type, such as a table insert, update, or delete operation. You might also define a pattern that matches a value in a specific column or row.

For example, to reroute all update (u) records to an updates topic, you might add the following configuration to your connector configuration:

...
transforms=route
transforms.route.type=io.debezium.transforms.ContentBasedRouter
transforms.route.language=jsr223.groovy
transforms.route.topic.expression=value.op == 'u' ? 'updates' : null
...

The preceding example specifies the use of the Groovy expression language.

Records that do not match the pattern are routed to the default topic.

Customizing the configuration

The preceding example shows a simple SMT configuration that is designed to process only DML events, which contain an op field. Other types of messages that a connector might emit (heartbeat messages, tombstone messages, or metadata messages about transactions or schema changes) do not contain this field. To avoid processing failures, you can define an SMT predicate statement that selectively applies the transformation to specific events only.

Variables for use in content-based routing expressions

Debezium binds certain variables into the evaluation context for the SMT. When you create expressions to specify conditions to control the routing destination, the SMT can look up and interpret the values of these variables to evaluate conditions in an expression.

The following table lists the variables that Debezium binds into the evaluation context for the content-based routing SMT:

Table 1. Content-based routing expression variables
Name Description Type

key

A key of the message.

org.apache.kafka.connect​.data​.Struct

value

A value of the message.

org.apache.kafka.connect​.data​.Struct

keySchema

Schema of the message key.

org.apache.kafka.connect​.data​.Schema

valueSchema

Schema of the message value.

org.apache.kafka.connect​.data​.Schema

topic

Name of the target topic.

String

headers

A Java map of message headers. The key field is the header name. The headers variable exposes the following properties:

  • value (of type Object)

  • schema (of type org.apache.kafka​.connect​.data​.Schema)

java.util.Map​<String,​ io.debezium​.transforms​.scripting​.RecordHeader>

An expression can invoke arbitrary methods on its variables. Expressions should resolve to a Boolean value that determines how the SMT dispositions the message. When the routing condition in an expression evaluates to true, the message is retained. When the routing condition evaluates to false, the message is removed.

Expressions should not result in any side-effects. That is, they should not modify any variables that they pass.

Options for applying the transformation selectively

In addition to the change event messages that a Debezium connector emits when a database change occurs, the connector also emits other types of messages, including heartbeat messages, and metadata messages about schema changes and transactions. Because the structure of these other messages differs from the structure of the change event messages that the SMT is designed to process, it’s best to configure the connector to selectively apply the SMT, so that it processes only the intended data change messages. You can use one of the following methods to configure the connector to apply the SMT selectively:

Language specifics

The way that you express content-based routing conditions depends on the scripting language that you use. For example, as shown in the basic configuration example, when you use Groovy as the expression language, the following expression reroutes all update (u) records to the updates topic, while routing other records to the default topic:

value.op == 'u' ? 'updates' : null

Other languages use different methods to express the same condition.

The Debezium MongoDB connector emits the after and patch fields as serialized JSON documents rather than as structures.
To use the ContentBasedRouting SMT with the MongoDB connector, you must first unwind the array fields in the JSON into separate documents.
You can do this by applying the MongoDB ExtractNewDocumentState SMT.

You could also take the approach of using a JSON parser within an expression to generate separate output documents for each array item.
For example, if you use Groovy as the expression language, add the groovy-json artifact to the classpath, and then add an expression such as (new groovy.json.JsonSlurper()).parseText(value.after).last_name == 'Kretchmar'.

Javascript

When you use JavaScript as the expression language, you can call the Struct#get() method to specify the content-based routing condition, as in the following example:

value.get('op') == 'u' ? 'updates' : null
Javascript with Graal.js

When you create content-based routing conditions by using JavaScript with Graal.js, you use an approach that is similar to the one use with Groovy. For example:

value.op == 'u' ? 'updates' : null

Configuration options

Property

Default

Description

An optional regular expression that evaluates the name of the destination topic for an event to determine whether to apply the condition logic. If the name of the destination topic matches the value in topic.regex, the transformation applies the condition logic before it passes the event to the topic. If the name of the topic does not match the value in topic.regex, the SMT passes the event to the topic unmodified.

The language in which the expression is written. Must begin with jsr223., for example, jsr223.groovy, or jsr223.graal.js. Debezium supports bootstrapping through the JSR 223 API ("Scripting for the Java ™ Platform") only.

The expression to be evaluated for every message. Must evaluate to a String value where a result of non-null reroutes the message to a new topic, and a null value routes the message to the default topic.

keep

Specifies how the transformation handles null (tombstone) messages. You can specify one of the following options:

keep

(Default) Pass the messages through.

drop

Remove the messages completely.

evaluate

Apply the condition logic to the messages.