E.g. the general message structure for an `update` change looks like this:
[source,json,indent=0]
----
{
"op": "u",
"source": {
...
},
"ts_ms" : "...",
"before" : {
"field1" : "oldvalue1",
"field2" : "oldvalue2"
},
"after" : {
"field1" : "newvalue1",
"field2" : "newvalue2"
}
}
----
More details about the message structure are provided in xref:connectors/index.adoc[the documentation for each connector].
This format allows the user to get most information about changes happening in the system.
The downside of using the complex format is that other connectors or other parts of the Kafka ecosystem usually expect the data in a simple message format that can generally be described like so:
[source,json,indent=0]
----
{
"field1" : "newvalue1",
"field2" : "newvalue2"
}
----
Debezium provides https://kafka.apache.org/documentation/#connect_transforms[a single message transformation] that crosses the bridge between the complex and simple formats, the https://github.com/debezium/debezium/blob/master/debezium-core/src/main/java/io/debezium/transforms/ExtractNewRecordState.java[ExtractNewRecordState] SMT.
The SMT provides three main functions.
It
* extracts the `after` field from change events and replaces the original event just with this part
* optionally filters delete and tombstone records, as per the capabilities and requirements of downstream consumers
The SMT can be applied either to a source connector (Debezium) or a sink connector.
We generally recommend to apply the transformation on the sink side as it means that the messages stored in Apache Kafka will contain the whole context.
The final decision depends on use case for each user.
== Configuration
The configuration is a part of source/sink task connector and is expressed in a set of properties:
* (optionally) a record with `null` value and the same key (a "tombstone" message). This record serves as a marker for Apache Kafka that all messages with this key can be removed from the topic during https://kafka.apache.org/documentation/#compaction[log compaction].
The SMT can optionally add metadata fields from the original change event to the final flattened record. This functionality can be used to add things like the operation or the table from the change event, or connector-specific fields like the Postgres LSN field. For more information on what's available see xref:connectors/index.adoc[the documentation for each connector].
In case of duplicate field names (e.g. "ts_ms" exists twice), the struct should be specified to get the correct field (e.g. "source.ts_ms"). The fields will be prefixed with "\\__" or "__<struct>_", depending on the specification of the struct. Please use a comma separated list without spaces.
For `DELETE` events, this option is only supported when the `delete.handling.mode` option is set to "rewrite".
=== Adding metadata fields to the header
The SMT can optionally add metadata fields from the original change event to the header of the final flattened record. This functionality can be used to add things like the operation or the table from the change event, or connector-specific fields like the Postgres LSN field. For more information on what's available see xref:connectors/index.adoc[the documentation for each connector].
In case of duplicate field names (e.g. "ts_ms" exists twice), the struct should be specified to get the correct field (e.g. "source.ts_ms"). The fields will be prefixed with "\\__" or "__<struct>_", depending on the specification of the struct. Please use a comma separated list without spaces.
_The `operation.header` option is deprecated and scheduled for removal. Please use add.headers instead. If both add.headers and operation.header are specified, the latter will be ignored._
_The `add.source.fields` option is deprecated and scheduled for removal. Please use add.fields instead. If both add.fields and add.source.fields are specified, the latter will be ignored._
The SMT can optionally add metadata fields from the original change event's `source` structure to the final flattened record (prefixed with "__"). This functionality can be used to add things like the table from the change event, or connector-specific fields like the Postgres LSN field. For more information on what's available in the source structure see xref:connectors/index.adoc[the documentation for each connector].
|The SMT removes the tombstone generated by Debezium from the stream.
|`delete.handling.mode`
|`drop`
|The SMT can `drop` (the default), `rewrite` or pass delete events (`none`). The rewrite mode will add a `__deleted` column with true/false values based on record operation.
|The column which determines how the events will be routed, the value will the topic name; obtained from the old record state for delete events, and from the new record state otherwise
|Specify a list of metadata fields to add to the flattened message. In case of duplicate field names (e.g. "ts_ms" exists twice), the struct should be specified to get the correct field (e.g. "source.ts_ms"). The fields will be prefixed with "\\__" or "__<struct>__", depending on the specification of the struct. Please use a comma separated list without spaces.
|`add.headers`
|
|Specify a list of metadata fields to add to the header of the flattened message. In case of duplicate field names (e.g. "ts_ms" exists twice), the struct should be specified to get the correct field (e.g. "source.ts_ms"). The fields will be prefixed with "\\__" or "__<struct>__", depending on the specification of the struct. Please use a comma separated list without spaces.
|`operation.header` DEPRECATED
|`false`
|_This option is deprecated and scheduled for removal. Please use add.headers instead. If both add.headers and operation.header are specified, the latter will be ignored._
The SMT adds the event operation (as obtained from the `op` field of the original record) as a message header.
|`add.source.fields` DEPRECATED
|
|_This option is deprecated and scheduled for removal. Please use add.fields instead. If both add.fields and add.source.fields are specified, the latter will be ignored._