DBZ-695 Tx metadata docs for Db2
This commit is contained in:
parent
e2d365ba4a
commit
7142e6124f
@ -669,6 +669,83 @@ When a row is deleted, the _delete_ event value listed above still works with lo
|
||||
But only if the message value is `null` will Kafka know that it can remove _all messages_ with that same key.
|
||||
To make this possible, Debezium's Db2 connector always follows the _delete_ event with a special _tombstone_ event that has the same key but `null` value.
|
||||
|
||||
[[transaction-metadata]]
|
||||
=== Transaction Metadata
|
||||
|
||||
[NOTE]
|
||||
====
|
||||
This feature is under active development right now (incubating),
|
||||
so the structure of transaction events or other details may still change as development progresses.
|
||||
====
|
||||
|
||||
Debezium can generate events that represents tranaction metadata boundaries and enrich data messages.
|
||||
|
||||
==== Transaction boundaries
|
||||
Debezium generates events for every transaction start and end.
|
||||
Every event contains
|
||||
|
||||
* `status` - `BEGIN` or `END`
|
||||
* `id` - string representation of unique transaction identifier
|
||||
* `event_count` (for `END` events) - total number of events emmitted by the transaction
|
||||
* `data_collections` (for `END` events) - an array of pairs of `data_collection` and `event_count` that provides number of events emitted by changes originating from given data collection
|
||||
|
||||
An example of messages looks like
|
||||
[source,json,indent=0,subs="attributes"]
|
||||
----
|
||||
{
|
||||
"status": "BEGIN",
|
||||
"id": "00000025:00000d08:0025",
|
||||
"event_count": null,
|
||||
"data_collections": null
|
||||
}
|
||||
|
||||
{
|
||||
"status": "END",
|
||||
"id": "00000025:00000d08:0025",
|
||||
"event_count": "2",
|
||||
"data_collections": [
|
||||
{
|
||||
"data_collection": "testDB.dbo.tablea",
|
||||
"event_count": "1"
|
||||
},
|
||||
{
|
||||
"data_collection": "testDB.dbo.tableb",
|
||||
"event_count": "1"
|
||||
}
|
||||
]
|
||||
}
|
||||
----
|
||||
|
||||
==== Data events enrichment
|
||||
When transaction metadata are enabled then data message `Envelope` is enriched with a new field `transaction`.
|
||||
This field provide information about every event in form of composite of fields
|
||||
|
||||
* `id` - string representation of unique transaction identifier
|
||||
* `total_order` - the absolute position the event amongst all events generated by the transaction
|
||||
* `data_collection_order` - the per-data collection position of the event amongst all events emitted by the transaction
|
||||
|
||||
An example of messages looks like
|
||||
[source,json,indent=0,subs="attributes"]
|
||||
----
|
||||
{
|
||||
"before": null,
|
||||
"after": {
|
||||
"pk": "2",
|
||||
"aa": "1"
|
||||
},
|
||||
"source": {
|
||||
...
|
||||
},
|
||||
"op": "c",
|
||||
"ts_ms": "1580390884335",
|
||||
"transaction": {
|
||||
"id": "00000025:00000d08:0025",
|
||||
"total_order": "1",
|
||||
"data_collection_order": "1"
|
||||
}
|
||||
}
|
||||
----
|
||||
|
||||
[[schema-evolution]]
|
||||
=== Database schema evolution
|
||||
|
||||
@ -1464,13 +1541,16 @@ When set to `0` the connector will fail immediately when it cannot obtain the lo
|
||||
This property contains a comma-separated list of fully-qualified tables _(SCHEMA_NAME.TABLE_NAME)_. Select statements for the individual tables are specified in further configuration properties, one for each table, identified by the id `snapshot.select.statement.overrides.[SCHEMA_NAME].[TABLE_NAME]`. The value of those properties is the SELECT statement to use when retrieving data from the specific table during snapshotting. _A possible use case for large append-only tables is setting a specific point where to start (resume) snapshotting, in case a previous snapshotting was interrupted._ +
|
||||
*Note*: This setting has impact on snapshots only. Events captured during log reading are not affected by it.
|
||||
|
||||
|
||||
|
||||
|`sanitize.field.names`
|
||||
|`true` when connector configuration explicitly specifies the `key.converter` or `value.converter` parameters to use Avro, otherwise defaults to `false`.
|
||||
|Whether field names will be sanitized to adhere to Avro naming requirements.
|
||||
See xref:configuration/avro.adoc#names[Avro naming] for more details.
|
||||
|
||||
|`provide.transaction.metadata` (Incubating)
|
||||
|`false`
|
||||
|When set to `true` Debezium generates events with transaction boundaries and enriches data events envelope with transaction metadata.
|
||||
|
||||
See link:#transaction-metadata[Transaction Metadata] for additional details.
|
||||
|
||||
|=======================
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user