Outbox Event Router
Example
In order to understand the configuration and terms used in this SMT, let’s look into its parts with the given expected outbox message:
# Kafka Topic: outbox.event.order
# Kafka Message key: "1"
# Kafka Message Headers: "id=4d47e190-0402-4048-bc2c-89dd54343cdc"
# Kafka Message Timestamp: 1556890294484
{
"eventType": "OrderCreated",
"payload": "{\"id\": 1, \"lineItems\": [{\"id\": 1, \"item\": \"Debezium in Action\", \"status\": \"ENTERED\", \"quantity\": 2, \"totalPrice\": 39.98}, {\"id\": 2, \"item\": \"Debezium for Dummies\", \"status\": \"ENTERED\", \"quantity\": 1, \"totalPrice\": 29.99}], \"orderDate\": \"2019-01-31T12:13:01\", \"customerId\": 123}"
}
This message is generated by transforming a Debezium raw message, which looks like:
# Kafka Message key: "406c07f3-26f0-4eea-a50c-109940064b8f"
# Kafka Message Headers: ""
# Kafka Message Timestamp: 1556890294484
{
"before": null,
"after": {
"id": "406c07f3-26f0-4eea-a50c-109940064b8f",
"aggregateid": "1",
"aggregatetype": "Order",
"payload": "{\"id\": 1, \"lineItems\": [{\"id\": 1, \"item\": \"Debezium in Action\", \"status\": \"ENTERED\", \"quantity\": 2, \"totalPrice\": 39.98}, {\"id\": 2, \"item\": \"Debezium for Dummies\", \"status\": \"ENTERED\", \"quantity\": 1, \"totalPrice\": 29.99}], \"orderDate\": \"2019-01-31T12:13:01\", \"customerId\": 123}",
"timestamp": 1556890294344,
"type": "OrderCreated"
},
"source": {
"version": "0.9.3.Final",
"connector": "postgresql",
"name": "dbserver1-bare",
"db": "orderdb",
"ts_usec": 1556890294448870,
"txId": 584,
"lsn": 24064704,
"schema": "inventory",
"table": "outboxevent",
"snapshot": false,
"last_snapshot_record": null,
"xmin": null
},
"op": "c",
"ts_ms": 1556890294484
}
This result was achieved with the default configuration which assumes a table structure and event routing based on aggregates. In case you want a custom behavior, the SMT is fully configurable, check the available configuration options.
Configuration
Configuration options
Property | Default | Group | Description |
| Table | The column which contains the event ID within the outbox table | |
| Table | The column which contains the event key within the outbox table; when this is set the value of this column will be used as a Kafka message key | |
| Table | The column which contains the event type within the outbox table | |
Table | Optionally you can override the Kafka message timestamp with a value from a chosen field, otherwise it is the Debezium event processed timestamp. | ||
| Table | The column which contains the event payload within the outbox table | |
| Table | The column which contains the payload ID within the outbox table | |
Table, Envelope | Extra fields can be added as part of the event envelope or as message header; the format is a list of colon-delimited pairs or trios when you desire to have aliases, e.g. | ||
Table, Schema | When set, it is used as schema version as in the Kafka Connect Schema javadoc | ||
| Router | The column which determines how the events will be routed, the value will become part of the topic name | |
| Router | The default regex to use within the RegexRouter, the default capture will allow to replace the routed field into a new topic name defined in | |
| Router | The name of the topic in which the events will be routed, a replacement | |
| Router | Whether or not an empty or | |
| Debezium | While Debezium is monitoring the table, it is not expecting to see ‘update’ row events, in case it happens, this transform can log it as warning, error or stop the process. Options are |
Default table columns
Column | Type | Modifiers
--------------+------------------------+-----------
id | uuid | not null
aggregatetype | character varying(255) | not null
aggregateid | character varying(255) | not null
type | character varying(255) | not null
payload | jsonb |
Default configuration values explained
After observing all those pieces we can see what the default configuration does:
Table Column | Effect |
| The |
| Is the default field for the routing, it gets append to the topic name (check configuration route.topic.replacement) |
| Becomes the Kafka message key, which is important for keeping ordering within Kafka partitions |
| The |
| The JSON representation of the event itself, becomes either part of the message as |
Basic configuration
transforms=outbox,...
transforms.outbox.type=io.debezium.transforms.outbox.EventRouter
Using Avro as the payload format
The outbox routing SMT supports arbitrary payload formats, as the payload column value is passed on transparently. As an alternative to working with JSON as shown above it is therefore also possible to use Avro. This can be beneficial for the purposes of message format governance and making sure outbox event schemas evolve in a backwards-compatible way.
How a source application produces Avro messages as an outbox event payload is out of the scope of this documentation. One possibility could be to leverage the KafkaAvroSerializer
class and use it to serialize GenericRecord
instances. In order to ensure that the Kafka message value is the exact Avro binary data, apply the following configuration to the connector:
transforms=outbox,...
transforms.outbox.type=io.debezium.transforms.outbox.EventRouter
transforms.outbox.table.fields.additional.placement=type:header:eventType
value.converter=io.debezium.converters.ByteBufferConverter
This moves the eventType
value into a Kafka message header, leaving only the payload
column value (the Avro data) as the sole message value. Using ByteBufferConverter
as the value converter will propagate that value as-is into the Kafka message value.