tet123/documentation/modules/ROOT/pages/transformations/event-changes.adoc
2023-02-10 13:53:05 +01:00

149 lines
6.8 KiB
Plaintext

:page-aliases: configuration/event-changes.adoc
[id="event-record-changes"]
= Event Record Changes
:toc:
:toc-placement: macro
:linkattrs:
:icons: font
:source-highligher: highlight.js
toc::[]
ifdef::community[]
[NOTE]
====
This single message transformation (SMT) is supported for only the SQL database connectors.
====
endif::community[]
A {prodname} data change event has a complex structure that provides a wealth of information.
However, in some cases, before a downstream consumer can process {prodname} change event messages, it requires additional information about field-level changes that result from the original database change.
To enhance event messages with details about how a database operation modifies fields in the source database, {prodname} provides the `ExtractChangedRecordState` single message transformation (SMT).
The event changes transformation is a
link:https://kafka.apache.org/documentation/#connect_transforms[Kafka Connect SMT].
== Change event structure
{prodname} generates data change events that have a complex structure.
Each event consists of three parts:
* Metadata, which includes but is not limited to:
** The type of operation that changed the data.
** Source information, such as the names of the database and the table in which the change occurred.
** Timestamp that identifies when the change was made.
** Optional transaction information
* Row data before the change
* Row data after the change
The following example shows part of the structure of a typical {prodname} `UPDATE` change event:
[source,json,indent=0]
----
{
"op": "u",
"source": {
...
},
"ts_ms" : "...",
"before" : {
"field1" : "oldvalue1",
"field2" : "oldvalue2"
},
"after" : {
"field1" : "newvalue1",
"field2" : "newvalue2"
}
}
----
ifdef::community[]
More details about change event structure are provided in
xref:{link-connectors}[the documentation for each connector].
endif::community[]
This complex format provides the most information about changes happening in the system.
However, downstream consumers of the message (other connectors, or other parts of the Kafka ecosystem) might expect the message to explicitly identify the fields that a database operation changes or leaves unchanged.
This event changes SMT adds information about these field changes to headers in the change event message.
[[event-changes-behavior]]
== Behavior
The event changes SMT extracts the `before` and `after` fields from a {prodname} `UPDATE` change event in a Kafka record.
The SMT examines the fields in the `before` and `after` event state structures, and then writes the names of fields that changed or are unchanged based on the values of settings that you apply to the SMT configuration.
If the event represents an `INSERT` or `DELETE`, this single message transformation has no effect.
You can configure the event changes SMT for a {prodname} connector, or for a sink connector that consumes messages emitted by a {prodname} connector.
Configure the event changes SMT for a sink connector if you want Apache Kafka to retain the entire original {prodname} change events.
The decision to apply the SMT to a source or sink connector depends on your particular use case.
You can configure the transformation to do any of the following:
* Identify the fields that are changed by an `UPDATE` event by listing them in the user-configured xref:extract-changes-header-changed-name[`header.changed.name`] header.
* Identify the fields that are not changed by an `UPDATE` event by listing them in the user-configured xref:extract-changes-header-unchanged-name[`header.unchanged.name`] header.
The single message transformation can be configured to include both changed and unchanged fields or either, depending on your use case.
== Configuration
Configure the {prodname} event changes SMT in a Kafka Connect source or sink connector by adding the SMT configuration details to your connector's configuration.
To obtain the default behavior which doesn't add any headers, in a `.properties` file, you would specify something like the following:
[source]
----
transforms=changes,...
transforms.changes.type=io.debezium.transforms.ExtractChangedRecordState
----
As with any Kafka Connect connector configuration, you can set `transforms=` to multiple, comma-seperated, SMT aliases in the order in which you want Kafka Connect to apply the SMTs.
The following `.properties` example sets several event changes SMT options:
[source]
----
transforms=changes,...
transforms.changes.type=io.debezium.transforms.ExtractChangedRecordState
transforms.changes.header.changed.name=Changed
transforms.changes.header.unchanged.name=Unchanged
----
`header.changed.name`:: Specifies the name of the header that will include a comma-separated list of fields that were changed.
`header.unchanged.name`:: Specifies the name of the header that will include a comma-separated list of fields that were not changed.
.Customizing the configuration
The connector might emit many types of event messages (heartbeat messages, tombstone messages, or metadata messages about transactions or schema changes).
To apply the transformation to a subset of events, you can define xref:options-for-applying-the-transformation-selectively[an SMT predicate statement that selectively applies the transformation] to specific events only.
[id="options-for-applying-the-event-flattening-transformation-selectively"]
== Options for applying the event changes transformation selectively
In addition to the change event messages that a {prodname} 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.
For more information about how to apply the SMT selectively, see xref:{link-smt-predicates}#applying-transformations-selectively[Configure an SMT predicate for the transformation].
[id="configuration-options"]
== Configuration options
The following table describes the options that you can specify to configure the event changes SMT.
.Descriptions of event changes SMT configuration options
[cols="30%a,25%a,45%a",subs="+attributes",options="header"]
|===
|Option
|Default
|Description
|[[extract-changes-header-changed-name]]<<extract-changes-header-changed-name, `+header.changed.name+`>>
|
|Specifies the name of the header that will include a comma-separated list of fields that were changed.
|[[extract-changes-header-unchanged-name]]<<extract-changes-header-unchanged-name, `+header.unchanged.name+`>>
|
|Specifies the name of the header that will include a comma-separated list of fields that were not changed.