Chapter 7. Debezium connector for PostgreSQL
Debezium’s PostgreSQL connector captures row-level changes in the schemas of a PostgreSQL database. PostgreSQL versions 10, 11, 12 and 13 are supported.
The first time it connects to a PostgreSQL server or cluster, the connector takes a consistent snapshot of all schemas. After that snapshot is complete, the connector continuously captures row-level changes that insert, update, and delete database content and that were committed to a PostgreSQL database. The connector generates data change event records and streams them to Kafka topics. For each table, the default behavior is that the connector streams all generated events to a separate Kafka topic for that table. Applications and services consume data change event records from that topic.
Information and procedures for using a Debezium PostgreSQL connector is organized as follows:
- Section 7.1, “Overview of Debezium PostgreSQL connector”
- Section 7.2, “How Debezium PostgreSQL connectors work”
- Section 7.3, “Descriptions of Debezium PostgreSQL connector data change events”
- Section 7.4, “How Debezium PostgreSQL connectors map data types”
- Section 7.5, “Setting up PostgreSQL to run a Debezium connector”
- Section 7.6, “Deployment of Debezium PostgreSQL connectors”
- Section 7.7, “Monitoring Debezium PostgreSQL connector performance”
- Section 7.8, “How Debezium PostgreSQL connectors handle faults and problems”
7.1. Overview of Debezium PostgreSQL connector
PostgreSQL’s logical decoding feature was introduced in version 9.4. It is a mechanism that allows the extraction of the changes that were committed to the transaction log and the processing of these changes in a user-friendly manner with the help of an output plug-in. The output plug-in enables clients to consume the changes.
The PostgreSQL connector contains two main parts that work together to read and process database changes:
-
pgoutput
is the standard logical decoding output plug-in in PostgreSQL 10+. This is the only supported logical decoding output plug-in in this Debezium release. This plug-in is maintained by the PostgreSQL community, and used by PostgreSQL itself for logical replication. This plug-in is always present so no additional libraries need to be installed. The Debezium connector interprets the raw replication event stream directly into change events. - Java code (the actual Kafka Connect connector) that reads the changes produced by the logical decoding output plug-in by using PostgreSQL’s streaming replication protocol and the PostgreSQL JDBC driver.
The connector produces a change event for every row-level insert, update, and delete operation that was captured and sends change event records for each table in a separate Kafka topic. Client applications read the Kafka topics that correspond to the database tables of interest, and can react to every row-level event they receive from those topics.
PostgreSQL normally purges write-ahead log (WAL) segments after some period of time. This means that the connector does not have the complete history of all changes that have been made to the database. Therefore, when the PostgreSQL connector first connects to a particular PostgreSQL database, it starts by performing a consistent snapshot of each of the database schemas. After the connector completes the snapshot, it continues streaming changes from the exact point at which the snapshot was made. This way, the connector starts with a consistent view of all of the data, and does not omit any changes that were made while the snapshot was being taken.
The connector is tolerant of failures. As the connector reads changes and produces events, it records the WAL position for each event. If the connector stops for any reason (including communication failures, network problems, or crashes), upon restart the connector continues reading the WAL where it last left off. This includes snapshots. If the connector stops during a snapshot, the connector begins a new snapshot when it restarts.
The connector relies on and reflects the PostgreSQL logical decoding feature, which has the following limitations:
- Logical decoding does not support DDL changes. This means that the connector is unable to report DDL change events back to consumers.
-
Logical decoding replication slots are supported on only
primary
servers. When there is a cluster of PostgreSQL servers, the connector can run on only the activeprimary
server. It cannot run onhot
orwarm
standby replicas. If theprimary
server fails or is demoted, the connector stops. After theprimary
server has recovered, you can restart the connector. If a different PostgreSQL server has been promoted toprimary
, adjust the connector configuration before restarting the connector.
Behavior when things go wrong describes what the connector does when there is a problem.
Debezium currently supports databases with UTF-8 character encoding only. With a single byte character encoding, it is not possible to correctly process strings that contain extended ASCII code characters.
7.2. How Debezium PostgreSQL connectors work
To optimally configure and run a Debezium PostgreSQL connector, it is helpful to understand how the connector performs snapshots, streams change events, determines Kafka topic names, and uses metadata.
Details are in the following topics:
- Section 7.2.2, “How Debezium PostgreSQL connectors perform database snapshots”
- Section 7.2.3, “How Debezium PostgreSQL connectors stream change event records”
- Section 7.2.4, “Default names of Kafka topics that receive Debezium PostgreSQL change event records”
- Section 7.2.5, “Metadata in Debezium PostgreSQL change event records”
- Section 7.2.6, “Debezium PostgreSQL connector-generated events that represent transaction boundaries”
7.2.1. Security for PostgreSQL connector
To use the Debezium connector to stream changes from a PostgreSQL database, the connector must operate with specific privileges in the database. Although one way to grant the necessary privileges is to provide the user with superuser
privileges, doing so potentially exposes your PostgreSQL data to unauthorized access. Rather than granting excessive privileges to the Debezium user, it is best to create a dedicated Debezium replication user to which you grant specific privileges.
For more information about configuring privileges for the Debezium PostgreSQL user, see Setting up permissions. For more information about PostgreSQL logical replication security, see the PostgreSQL documentation.
7.2.2. How Debezium PostgreSQL connectors perform database snapshots
Most PostgreSQL servers are configured to not retain the complete history of the database in the WAL segments. This means that the PostgreSQL connector would be unable to see the entire history of the database by reading only the WAL. Consequently, the first time that the connector starts, it performs an initial consistent snapshot of the database. The default behavior for performing a snapshot consists of the following steps. You can change this behavior by setting the snapshot.mode
connector configuration property to a value other than initial
.
-
Start a transaction with a SERIALIZABLE, READ ONLY, DEFERRABLE isolation level to ensure that subsequent reads in this transaction are against a single consistent version of the data. Any changes to the data due to subsequent
INSERT
,UPDATE
, andDELETE
operations by other clients are not visible to this transaction. Obtain an
ACCESS SHARE MODE
lock on each of the tables being tracked to ensure that no structural changes can occur to any of the tables while the snapshot is taking place. These locks do not prevent tableINSERT
,UPDATE
andDELETE
operations from taking place during the snapshot.This step is omitted when
snapshot.mode
is set toexported
, which allows the connector to perform a lock-free snapshot.- Read the current position in the server’s transaction log.
-
Scan the database tables and schemas, generate a
READ
event for each row and write that event to the appropriate table-specific Kafka topic. - Commit the transaction.
- Record the successful completion of the snapshot in the connector offsets.
If the connector fails, is rebalanced, or stops after Step 1 begins but before Step 6 completes, upon restart the connector begins a new snapshot. After the connector completes its initial snapshot, the PostgreSQL connector continues streaming from the position that it read in step 3. This ensures that the connector does not miss any updates. If the connector stops again for any reason, upon restart, the connector continues streaming changes from where it previously left off.
It is strongly recommended that you configure a PostgreSQL connector to set snapshot.mode
to exported
. The initial
, initial only
and always
modes can lose a few events while a connector switches from performing the snapshot to streaming change event records when a database is under heavy load. This is a known issue and the affected snapshot modes will be reworked to use exported
mode internally (DBZ-2337).
Setting | Description |
---|---|
|
The connector always performs a snapshot when it starts. After the snapshot completes, the connector continues streaming changes from step 3 in the above sequence. This mode is useful in these situations:
|
|
The connector never performs snapshots. When a connector is configured this way, its behavior when it starts is as follows. If there is a previously stored LSN in the Kafka offsets topic, the connector continues streaming changes from that position. If no LSN has been stored, the connector starts streaming changes from the point in time when the PostgreSQL logical replication slot was created on the server. The |
| The connector performs a database snapshot and stops before streaming any change event records. If the connector had started but did not complete a snapshot before stopping, the connector restarts the snapshot process and stops when the snapshot completes. |
| The connector performs a database snapshot based on the point in time when the replication slot was created. This mode is an excellent way to perform a snapshot in a lock-free way. |
7.2.3. How Debezium PostgreSQL connectors stream change event records
The PostgreSQL connector typically spends the vast majority of its time streaming changes from the PostgreSQL server to which it is connected. This mechanism relies on PostgreSQL’s replication protocol. This protocol enables clients to receive changes from the server as they are committed in the server’s transaction log at certain positions, which are referred to as Log Sequence Numbers (LSNs).
Whenever the server commits a transaction, a separate server process invokes a callback function from the logical decoding plug-in. This function processes the changes from the transaction, converts them to a specific format (Protobuf or JSON in the case of Debezium plug-in) and writes them on an output stream, which can then be consumed by clients.
The Debezium PostgreSQL connector acts as a PostgreSQL client. When the connector receives changes it transforms the events into Debezium create, update, or delete events that include the LSN of the event. The PostgreSQL connector forwards these change events in records to the Kafka Connect framework, which is running in the same process. The Kafka Connect process asynchronously writes the change event records in the same order in which they were generated to the appropriate Kafka topic.
Periodically, Kafka Connect records the most recent offset in another Kafka topic. The offset indicates source-specific position information that Debezium includes with each event. For the PostgreSQL connector, the LSN recorded in each change event is the offset.
When Kafka Connect gracefully shuts down, it stops the connectors, flushes all event records to Kafka, and records the last offset received from each connector. When Kafka Connect restarts, it reads the last recorded offset for each connector, and starts each connector at its last recorded offset. When the connector restarts, it sends a request to the PostgreSQL server to send the events starting just after that position.
The PostgreSQL connector retrieves schema information as part of the events sent by the logical decoding plug-in. However, the connector does not retrieve information about which columns compose the primary key. The connector obtains this information from the JDBC metadata (side channel). If the primary key definition of a table changes (by adding, removing or renaming primary key columns), there is a tiny period of time when the primary key information from JDBC is not synchronized with the change event that the logical decoding plug-in generates. During this tiny period, a message could be created with an inconsistent key structure. To prevent this inconsistency, update primary key structures as follows:
- Put the database or an application into a read-only mode.
- Let Debezium process all remaining events.
- Stop Debezium.
- Update the primary key definition in the relevant table.
- Put the database or the application into read/write mode.
- Restart Debezium.
PostgreSQL 10+ logical decoding support (pgoutput
)
As of PostgreSQL 10+, there is a logical replication stream mode, called pgoutput
that is natively supported by PostgreSQL. This means that a Debezium PostgreSQL connector can consume that replication stream without the need for additional plug-ins. This is particularly valuable for environments where installation of plug-ins is not supported or not allowed.
See Setting up PostgreSQL for more details.
7.2.4. Default names of Kafka topics that receive Debezium PostgreSQL change event records
The PostgreSQL connector writes events for all insert, update, and delete operations on a single table to a single Kafka topic. By default, the Kafka topic name is serverName.schemaName.tableName where:
-
serverName is the logical name of the connector as specified with the
database.server.name
connector configuration property. - schemaName is the name of the database schema where the operation occurred.
- tableName is the name of the database table in which the operation occurred.
For example, suppose that fulfillment
is the logical server name in the configuration for a connector that is capturing changes in a PostgreSQL installation that has a postgres
database and an inventory
schema that contains four tables: products
, products_on_hand
, customers
, and orders
. The connector would stream records to these four Kafka topics:
-
fulfillment.inventory.products
-
fulfillment.inventory.products_on_hand
-
fulfillment.inventory.customers
-
fulfillment.inventory.orders
Now suppose that the tables are not part of a specific schema but were created in the default public
PostgreSQL schema. The names of the Kafka topics would be:
-
fulfillment.public.products
-
fulfillment.public.products_on_hand
-
fulfillment.public.customers
-
fulfillment.public.orders
7.2.5. Metadata in Debezium PostgreSQL change event records
In addition to a database change event, each record produced by a PostgreSQL connector contains some metadata. Metadata includes where the event occurred on the server, the name of the source partition and the name of the Kafka topic and partition where the event should go, for example:
"sourcePartition": { "server": "fulfillment" }, "sourceOffset": { "lsn": "24023128", "txId": "555", "ts_ms": "1482918357011" }, "kafkaPartition": null
-
sourcePartition
always defaults to the setting of thedatabase.server.name
connector configuration property. sourceOffset
contains information about the location of the server where the event occurred:-
lsn
represents the PostgreSQL Log Sequence Number oroffset
in the transaction log. -
txId
represents the identifier of the server transaction that caused the event. -
ts_ms
represents the server time at which the transaction was committed in the form of the number of milliseconds since the epoch.
-
-
kafkaPartition
with a setting ofnull
means that the connector does not use a specific Kafka partition. The PostgreSQL connector uses only one Kafka Connect partition and it places the generated events into one Kafka partition.
7.2.6. Debezium PostgreSQL connector-generated events that represent transaction boundaries
Debezium can generate events that represent transaction boundaries and that enrich data change event messages. For every transaction BEGIN
and END
, Debezium generates an event that contains the following fields:
-
status
-BEGIN
orEND
-
id
- string representation of unique transaction identifier -
event_count
(forEND
events) - total number of events emitted by the transaction -
data_collections
(forEND
events) - an array of pairs ofdata_collection
andevent_count
that provides the number of events emitted by changes originating from given data collection
Example
{ "status": "BEGIN", "id": "571", "event_count": null, "data_collections": null } { "status": "END", "id": "571", "event_count": 2, "data_collections": [ { "data_collection": "s1.a", "event_count": 1 }, { "data_collection": "s2.a", "event_count": 1 } ] }
Transaction events are written to the topic named database.server.name.transaction
.
Change data event enrichment
When transaction metadata is enabled the data message Envelope
is enriched with a new transaction
field. This field provides information about every event in the form of a composite of fields:
-
id
- string representation of unique transaction identifier -
total_order
- absolute position of the event among all events generated by the transaction -
data_collection_order
- the per-data collection position of the event among all events that were emitted by the transaction
Following is an example of a message:
{ "before": null, "after": { "pk": "2", "aa": "1" }, "source": { ... }, "op": "c", "ts_ms": "1580390884335", "transaction": { "id": "571", "total_order": "1", "data_collection_order": "1" } }
7.3. Descriptions of Debezium PostgreSQL connector data change events
The Debezium PostgreSQL connector generates a data change event for each row-level INSERT
, UPDATE
, and DELETE
operation. Each event contains a key and a value. The structure of the key and the value depends on the table that was changed.
Debezium and Kafka Connect are designed around continuous streams of event messages. However, the structure of these events may change over time, which can be difficult for consumers to handle. To address this, each event contains the schema for its content or, if you are using a schema registry, a schema ID that a consumer can use to obtain the schema from the registry. This makes each event self-contained.
The following skeleton JSON shows the basic four parts of a change event. However, how you configure the Kafka Connect converter that you choose to use in your application determines the representation of these four parts in change events. A schema
field is in a change event only when you configure the converter to produce it. Likewise, the event key and event payload are in a change event only if you configure a converter to produce it. If you use the JSON converter and you configure it to produce all four basic change event parts, change events have this structure:
{ "schema": { 1 ... }, "payload": { 2 ... }, "schema": { 3 ... }, "payload": { 4 ... }, }
Item | Field name | Description |
---|---|---|
1 |
|
The first |
2 |
|
The first |
3 |
|
The second |
4 |
|
The second |
By default behavior is that the connector streams change event records to topics with names that are the same as the event’s originating table.
Starting with Kafka 0.10, Kafka can optionally record the event key and value with the timestamp at which the message was created (recorded by the producer) or written to the log by Kafka.
The PostgreSQL connector ensures that all Kafka Connect schema names adhere to the Avro schema name format. This means that the logical server name must start with a Latin letter or an underscore, that is, a-z, A-Z, or _. Each remaining character in the logical server name and each character in the schema and table names must be a Latin letter, a digit, or an underscore, that is, a-z, A-Z, 0-9, or \_. If there is an invalid character it is replaced with an underscore character.
This can lead to unexpected conflicts if the logical server name, a schema name, or a table name contains invalid characters, and the only characters that distinguish names from one another are invalid and thus replaced with underscores.
Details are in the following topics:
7.3.1. About keys in Debezium PostgreSQL change events
For a given table, the change event’s key has a structure that contains a field for each column in the primary key of the table at the time the event was created. Alternatively, if the table has REPLICA IDENTITY
set to FULL
or USING INDEX
there is a field for each unique key constraint.
Consider a customers
table defined in the public
database schema and the example of a change event key for that table.
Example table
CREATE TABLE customers ( id SERIAL, first_name VARCHAR(255) NOT NULL, last_name VARCHAR(255) NOT NULL, email VARCHAR(255) NOT NULL, PRIMARY KEY(id) );
Example change event key
If the database.server.name
connector configuration property has the value PostgreSQL_server
, every change event for the customers
table while it has this definition has the same key structure, which in JSON looks like this:
{ "schema": { 1 "type": "struct", "name": "PostgreSQL_server.public.customers.Key", 2 "optional": false, 3 "fields": [ 4 { "name": "id", "index": "0", "schema": { "type": "INT32", "optional": "false" } } ] }, "payload": { 5 "id": "1" }, }
Item | Field name | Description |
---|---|---|
1 |
|
The schema portion of the key specifies a Kafka Connect schema that describes what is in the key’s |
2 |
|
Name of the schema that defines the structure of the key’s payload. This schema describes the structure of the primary key for the table that was changed. Key schema names have the format connector-name.database-name.table-name.
|
3 |
|
Indicates whether the event key must contain a value in its |
4 |
|
Specifies each field that is expected in the |
5 |
|
Contains the key for the row for which this change event was generated. In this example, the key, contains a single |
Although the column.exclude.list
and column.include.list
connector configuration properties allow you to capture only a subset of table columns, all columns in a primary or unique key are always included in the event’s key.
If the table does not have a primary or unique key, then the change event’s key is null. The rows in a table without a primary or unique key constraint cannot be uniquely identified.
7.3.2. About values in Debezium PostgreSQL change events
The value in a change event is a bit more complicated than the key. Like the key, the value has a schema
section and a payload
section. The schema
section contains the schema that describes the Envelope
structure of the payload
section, including its nested fields. Change events for operations that create, update or delete data all have a value payload with an envelope structure.
Consider the same sample table that was used to show an example of a change event key:
CREATE TABLE customers ( id SERIAL, first_name VARCHAR(255) NOT NULL, last_name VARCHAR(255) NOT NULL, email VARCHAR(255) NOT NULL, PRIMARY KEY(id) );
The value portion of a change event for a change to this table varies according to the REPLICA IDENTITY
setting and the operation that the event is for.
Details follow in these sections:
Replica identity
REPLICA IDENTITY is a PostgreSQL-specific table-level setting that determines the amount of information that is available to the logical decoding plug-in for UPDATE
and DELETE
events. More specifically, the setting of REPLICA IDENTITY
controls what (if any) information is available for the previous values of the table columns involved, whenever an UPDATE
or DELETE
event occurs.
There are 4 possible values for REPLICA IDENTITY
:
DEFAULT
- The default behavior is thatUPDATE
andDELETE
events contain the previous values for the primary key columns of a table if that table has a primary key. For anUPDATE
event, only the primary key columns with changed values are present.If a table does not have a primary key, the connector does not emit
UPDATE
orDELETE
events for that table. For a table without a primary key, the connector emits only create events. Typically, a table without a primary key is used for appending messages to the end of the table, which means thatUPDATE
andDELETE
events are not useful.-
NOTHING
- Emitted events forUPDATE
andDELETE
operations do not contain any information about the previous value of any table column. -
FULL
- Emitted events forUPDATE
andDELETE
operations contain the previous values of all columns in the table. -
INDEX
index-name - Emitted events forUPDATE
andDELETE
operations contain the previous values of the columns contained in the specified index.UPDATE
events also contain the indexed columns with the updated values.
create events
The following example shows the value portion of a change event that the connector generates for an operation that creates data in the customers
table:
{ "schema": { 1 "type": "struct", "fields": [ { "type": "struct", "fields": [ { "type": "int32", "optional": false, "field": "id" }, { "type": "string", "optional": false, "field": "first_name" }, { "type": "string", "optional": false, "field": "last_name" }, { "type": "string", "optional": false, "field": "email" } ], "optional": true, "name": "PostgreSQL_server.inventory.customers.Value", 2 "field": "before" }, { "type": "struct", "fields": [ { "type": "int32", "optional": false, "field": "id" }, { "type": "string", "optional": false, "field": "first_name" }, { "type": "string", "optional": false, "field": "last_name" }, { "type": "string", "optional": false, "field": "email" } ], "optional": true, "name": "PostgreSQL_server.inventory.customers.Value", "field": "after" }, { "type": "struct", "fields": [ { "type": "string", "optional": false, "field": "version" }, { "type": "string", "optional": false, "field": "connector" }, { "type": "string", "optional": false, "field": "name" }, { "type": "int64", "optional": false, "field": "ts_ms" }, { "type": "boolean", "optional": true, "default": false, "field": "snapshot" }, { "type": "string", "optional": false, "field": "db" }, { "type": "string", "optional": false, "field": "schema" }, { "type": "string", "optional": false, "field": "table" }, { "type": "int64", "optional": true, "field": "txId" }, { "type": "int64", "optional": true, "field": "lsn" }, { "type": "int64", "optional": true, "field": "xmin" } ], "optional": false, "name": "io.debezium.connector.postgresql.Source", 3 "field": "source" }, { "type": "string", "optional": false, "field": "op" }, { "type": "int64", "optional": true, "field": "ts_ms" } ], "optional": false, "name": "PostgreSQL_server.inventory.customers.Envelope" 4 }, "payload": { 5 "before": null, 6 "after": { 7 "id": 1, "first_name": "Anne", "last_name": "Kretchmar", "email": "annek@noanswer.org" }, "source": { 8 "version": "1.5.4.Final", "connector": "postgresql", "name": "PostgreSQL_server", "ts_ms": 1559033904863, "snapshot": true, "db": "postgres", "sequence": "[\"24023119\",\"24023128\"]" "schema": "public", "table": "customers", "txId": 555, "lsn": 24023128, "xmin": null }, "op": "c", 9 "ts_ms": 1559033904863 10 } }
Item | Field name | Description |
---|---|---|
1 |
| The value’s schema, which describes the structure of the value’s payload. A change event’s value schema is the same in every change event that the connector generates for a particular table. |
2 |
|
In the |
3 |
|
|
4 |
|
|
5 |
|
The value’s actual data. This is the information that the change event is providing. |
6 |
|
An optional field that specifies the state of the row before the event occurred. When the Note
Whether or not this field is available is dependent on the |
7 |
|
An optional field that specifies the state of the row after the event occurred. In this example, the |
8 |
| Mandatory field that describes the source metadata for the event. This field contains information that you can use to compare this event with other events, with regard to the origin of the events, the order in which the events occurred, and whether events were part of the same transaction. The source metadata includes:
|
9 |
|
Mandatory string that describes the type of operation that caused the connector to generate the event. In this example,
|
10 |
|
Optional field that displays the time at which the connector processed the event. The time is based on the system clock in the JVM running the Kafka Connect task. |
update events
The value of a change event for an update in the sample customers
table has the same schema as a create event for that table. Likewise, the event value’s payload has the same structure. However, the event value payload contains different values in an update event. Here is an example of a change event value in an event that the connector generates for an update in the customers
table:
{ "schema": { ... }, "payload": { "before": { 1 "id": 1 }, "after": { 2 "id": 1, "first_name": "Anne Marie", "last_name": "Kretchmar", "email": "annek@noanswer.org" }, "source": { 3 "version": "1.5.4.Final", "connector": "postgresql", "name": "PostgreSQL_server", "ts_ms": 1559033904863, "snapshot": false, "db": "postgres", "schema": "public", "table": "customers", "txId": 556, "lsn": 24023128, "xmin": null }, "op": "u", 4 "ts_ms": 1465584025523 5 } }
Item | Field name | Description |
---|---|---|
1 |
|
An optional field that contains values that were in the row before the database commit. In this example, only the primary key column, |
2 |
|
An optional field that specifies the state of the row after the event occurred. In this example, the |
3 |
|
Mandatory field that describes the source metadata for the event. The
|
4 |
|
Mandatory string that describes the type of operation. In an update event value, the |
5 |
|
Optional field that displays the time at which the connector processed the event. The time is based on the system clock in the JVM running the Kafka Connect task. |
Updating the columns for a row’s primary/unique key changes the value of the row’s key. When a key changes, Debezium outputs three events: a DELETE
event and a tombstone event with the old key for the row, followed by an event with the new key for the row. Details are in the next section.
Primary key updates
An UPDATE
operation that changes a row’s primary key field(s) is known as a primary key change. For a primary key change, in place of sending an UPDATE
event record, the connector sends a DELETE
event record for the old key and a CREATE
event record for the new (updated) key. These events have the usual structure and content, and in addition, each one has a message header related to the primary key change:
-
The
DELETE
event record has__debezium.newkey
as a message header. The value of this header is the new primary key for the updated row. -
The
CREATE
event record has__debezium.oldkey
as a message header. The value of this header is the previous (old) primary key that the updated row had.
delete events
The value in a delete change event has the same schema
portion as create and update events for the same table. The payload
portion in a delete event for the sample customers
table looks like this:
{ "schema": { ... }, "payload": { "before": { 1 "id": 1 }, "after": null, 2 "source": { 3 "version": "1.5.4.Final", "connector": "postgresql", "name": "PostgreSQL_server", "ts_ms": 1559033904863, "snapshot": false, "db": "postgres", "schema": "public", "table": "customers", "txId": 556, "lsn": 46523128, "xmin": null }, "op": "d", 4 "ts_ms": 1465581902461 5 } }
Item | Field name | Description |
---|---|---|
1 |
|
Optional field that specifies the state of the row before the event occurred. In a delete event value, the |
2 |
|
Optional field that specifies the state of the row after the event occurred. In a delete event value, the |
3 |
|
Mandatory field that describes the source metadata for the event. In a delete event value, the
|
4 |
|
Mandatory string that describes the type of operation. The |
5 |
|
Optional field that displays the time at which the connector processed the event. The time is based on the system clock in the JVM running the Kafka Connect task. |
A delete change event record provides a consumer with the information it needs to process the removal of this row.
For a consumer to be able to process a delete event generated for a table that does not have a primary key, set the table’s REPLICA IDENTITY
to FULL
. When a table does not have a primary key and the table’s REPLICA IDENTITY
is set to DEFAULT
or NOTHING
, a delete event has no before
field.
PostgreSQL connector events are designed to work with Kafka log compaction. Log compaction enables removal of some older messages as long as at least the most recent message for every key is kept. This lets Kafka reclaim storage space while ensuring that the topic contains a complete data set and can be used for reloading key-based state.
Tombstone events
When a row is deleted, the delete event value still works with log compaction, because Kafka can remove all earlier messages that have that same key. However, for Kafka to remove all messages that have that same key, the message value must be null
. To make this possible, the PostgreSQL connector follows a delete event with a special tombstone event that has the same key but a null
value.
truncate events
A truncate change event signals that a table has been truncated. The message key is null
in this case, the message value looks like this:
{ "schema": { ... }, "payload": { "source": { 1 "version": "1.5.4.Final", "connector": "postgresql", "name": "PostgreSQL_server", "ts_ms": 1559033904863, "snapshot": false, "db": "postgres", "schema": "public", "table": "customers", "txId": 556, "lsn": 46523128, "xmin": null }, "op": "t", 2 "ts_ms": 1559033904961 3 } }
Item | Field name | Description |
---|---|---|
1 |
|
Mandatory field that describes the source metadata for the event. In a truncate event value, the
|
2 |
|
Mandatory string that describes the type of operation. The |
3 |
|
Optional field that displays the time at which the connector processed the event. The time is based on the system clock in the JVM running the Kafka Connect task. |
In case a single TRUNCATE
statement applies to multiple tables, one truncate change event record for each truncated table will be emitted.
Note that since truncate events represent a change made to an entire table and don’t have a message key, unless you’re working with topics with a single partition, there are no ordering guarantees for the change events pertaining to a table (create, update, etc.) and truncate events for that table. For instance a consumer may receive an update event only after a truncate event for that table, when those events are read from different partitions.
7.4. How Debezium PostgreSQL connectors map data types
The PostgreSQL connector represents changes to rows with events that are structured like the table in which the row exists. The event contains a field for each column value. How that value is represented in the event depends on the PostgreSQL data type of the column. The following sections describe how the connector maps PostgreSQL data types to a literal type and a semantic type in event fields.
-
literal type describes how the value is literally represented using Kafka Connect schema types:
INT8
,INT16
,INT32
,INT64
,FLOAT32
,FLOAT64
,BOOLEAN
,STRING
,BYTES
,ARRAY
,MAP
, andSTRUCT
. - semantic type describes how the Kafka Connect schema captures the meaning of the field using the name of the Kafka Connect schema for the field.
Details are in the following sections:
Basic types
The following table describes how the connector maps basic types.
PostgreSQL data type | Literal type (schema type) | Semantic type (schema name) and Notes |
---|---|---|
|
| n/a |
|
| n/a |
|
|
|
|
|
|
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
n/a |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| n/a |
|
| n/a |
|
|
n/a |
|
|
n/a |
|
|
n/a |
|
|
n/a |
|
|
n/a |
|
|
n/a |
|
|
|
Temporal types
Other than PostgreSQL’s TIMESTAMPTZ
and TIMETZ
data types, which contain time zone information, how temporal types are mapped depends on the value of the time.precision.mode
connector configuration property. The following sections describe these mappings:
time.precision.mode=adaptive
When the time.precision.mode
property is set to adaptive
, the default, the connector determines the literal type and semantic type based on the column’s data type definition. This ensures that events exactly represent the values in the database.
PostgreSQL data type | Literal type (schema type) | Semantic type (schema name) and Notes |
---|---|---|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
time.precision.mode=adaptive_time_microseconds
When the time.precision.mode
configuration property is set to adaptive_time_microseconds
, the connector determines the literal type and semantic type for temporal types based on the column’s data type definition. This ensures that events exactly represent the values in the database, except all TIME
fields are captured as microseconds.
PostgreSQL data type | Literal type (schema type) | Semantic type (schema name) and Notes |
---|---|---|
|
|
|
|
|
|
|
|
|
|
|
|
time.precision.mode=connect
When the time.precision.mode
configuration property is set to connect
, the connector uses Kafka Connect logical types. This may be useful when consumers can handle only the built-in Kafka Connect logical types and are unable to handle variable-precision time values. However, since PostgreSQL supports microsecond precision, the events generated by a connector with the connect
time precision mode results in a loss of precision when the database column has a fractional second precision value that is greater than 3.
PostgreSQL data type | Literal type (schema type) | Semantic type (schema name) and Notes |
---|---|---|
|
|
|
|
|
|
|
|
|
TIMESTAMP type
The TIMESTAMP
type represents a timestamp without time zone information. Such columns are converted into an equivalent Kafka Connect value based on UTC. For example, the TIMESTAMP
value "2018-06-20 15:13:16.945104" is represented by an io.debezium.time.MicroTimestamp
with the value "1529507596945104" when time.precision.mode
is not set to connect
.
The timezone of the JVM running Kafka Connect and Debezium does not affect this conversion.
PostgreSQL supports using +/-infinite
values in TIMESTAMP
columns. These special values are converted to timestamps with value 9223372036825200000
in case of positive infinity or -9223372036832400000
in case of negative infinity. This behaviour mimics the standard behaviour of PostgreSQL JDBC driver - see org.postgresql.PGStatement
interface for reference.
Decimal types
The setting of the PostgreSQL connector configuration property, decimal.handling.mode
determines how the connector maps decimal types.
When the decimal.handling.mode
property is set to precise
, the connector uses the Kafka Connect org.apache.kafka.connect.data.Decimal
logical type for all DECIMAL
and NUMERIC
columns. This is the default mode.
PostgreSQL data type | Literal type (schema type) | Semantic type (schema name) and Notes |
---|---|---|
|
|
|
|
|
|
There is an exception to this rule. When the NUMERIC
or DECIMAL
types are used without scale constraints, the values coming from the database have a different (variable) scale for each value. In this case, the connector uses io.debezium.data.VariableScaleDecimal
, which contains both the value and the scale of the transferred value.
PostgreSQL data type | Literal type (schema type) | Semantic type (schema name) and Notes |
---|---|---|
|
|
|
|
|
|
When the decimal.handling.mode
property is set to double
, the connector represents all DECIMAL
and NUMERIC
values as Java double values and encodes them as shown in the following table.
PostgreSQL data type | Literal type (schema type) | Semantic type (schema name) |
---|---|---|
|
| |
|
|
The last possible setting for the decimal.handling.mode
configuration property is string
. In this case, the connector represents DECIMAL
and NUMERIC
values as their formatted string representation, and encodes them as shown in the following table.
PostgreSQL data type | Literal type (schema type) | Semantic type (schema name) |
---|---|---|
|
| |
|
|
PostgreSQL supports NaN
(not a number) as a special value to be stored in DECIMAL
/NUMERIC
values when the setting of decimal.handling.mode
is string
or double
. In this case, the connector encodes NaN
as either Double.NaN
or the string constant NAN
.
HSTORE type
When the hstore.handling.mode
connector configuration property is set to json
(the default), the connector represents HSTORE
values as string representations of JSON values and encodes them as shown in the following table. When the hstore.handling.mode
property is set to map
, the connector uses the MAP
schema type for HSTORE
values.
PostgreSQL data type | Literal type (schema type) | Semantic type (schema name) and Notes |
---|---|---|
|
|
|
|
|
n/a |
Domain types
PostgreSQL supports user-defined types that are based on other underlying types. When such column types are used, Debezium exposes the column’s representation based on the full type hierarchy.
Capturing changes in columns that use PostgreSQL domain types requires special consideration. When a column is defined to contain a domain type that extends one of the default database types and the domain type defines a custom length or scale, the generated schema inherits that defined length or scale.
When a column is defined to contain a domain type that extends another domain type that defines a custom length or scale, the generated schema does not inherit the defined length or scale because that information is not available in the PostgreSQL driver’s column metadata.
Network address types
PostgreSQL has data types that can store IPv4, IPv6, and MAC addresses. It is better to use these types instead of plain text types to store network addresses. Network address types offer input error checking and specialized operators and functions.
PostgreSQL data type | Literal type (schema type) | Semantic type (schema name) and Notes |
---|---|---|
|
|
n/a |
|
|
n/a |
|
|
n/a |
|
|
n/a |
PostGIS types
The PostgreSQL connector supports all PostGIS data types.
PostGIS data type | Literal type (schema type) | Semantic type (schema name) and Notes |
---|---|---|
|
|
For format details, see Open Geospatial Consortium Simple Features Access specification. |
|
|
For format details, see Open Geospatial Consortium Simple Features Access specification. |
Toasted values
PostgreSQL has a hard limit on the page size. This means that values that are larger than around 8 KBs need to be stored by using TOAST storage. This impacts replication messages that are coming from the database. Values that were stored by using the TOAST mechanism and that have not been changed are not included in the message, unless they are part of the table’s replica identity. There is no safe way for Debezium to read the missing value out-of-bands directly from the database, as this would potentially lead to race conditions. Consequently, Debezium follows these rules to handle toasted values:
-
Tables with
REPLICA IDENTITY FULL
- TOAST column values are part of thebefore
andafter
fields in change events just like any other column. -
Tables with
REPLICA IDENTITY DEFAULT
- When receiving anUPDATE
event from the database, any unchanged TOAST column value that is not part of the replica identity is not contained in the event. Similarly, when receiving aDELETE
event, no TOAST columns, if any, are in thebefore
field. As Debezium cannot safely provide the column value in this case, the connector returns a placeholder value as defined by the connector configuration property,toasted.value.placeholder
.
7.5. Setting up PostgreSQL to run a Debezium connector
This release of Debezium supports only the native pgoutput
logical replication stream. To set up PostgreSQL so that it uses the pgoutput
plug-in, you must enable a replication slot, and configure a user with sufficient privileges to perform the replication.
Details are in the following topics:
-
Section 7.5.1, “Configuring a replication slot for the Debezium
pgoutput
plug-in” - Section 7.5.2, “Setting up PostgreSQL permissions for the Debezium connector”
- Section 7.5.3, “Setting privileges to enable Debezium to create PostgreSQL publications”
- Section 7.5.4, “Configuring PostgreSQL to allow replication with the Debezium connector host”
- Section 7.5.5, “Configuring PostgreSQL to manage Debezium WAL disk space consumption”
7.5.1. Configuring a replication slot for the Debezium pgoutput
plug-in
PostgreSQL’s logical decoding uses replication slots. To configure a replication slot, specify the following in the postgresql.conf
file:
wal_level=logical max_wal_senders=1 max_replication_slots=1
These settings instruct the PostgreSQL server as follows:
-
wal_level
- Use logical decoding with the write-ahead log. -
max_wal_senders
- Use a maximum of one separate process for processing WAL changes. -
max_replication_slots
- Allow a maximum of one replication slot to be created for streaming WAL changes.
Replication slots are guaranteed to retain all WAL entries that are required for Debezium even during Debezium outages. Consequently, it is important to closely monitor replication slots to avoid:
- Too much disk consumption
- Any conditions, such as catalog bloat, that can happen if a replication slot stays unused for too long
For more information, see the PostgreSQL documentation for replication slots.
Familiarity with the mechanics and configuration of the PostgreSQL write-ahead log is helpful for using the Debezium PostgreSQL connector.
7.5.2. Setting up PostgreSQL permissions for the Debezium connector
Setting up a PostgreSQL server to run a Debezium connector requires a database user that can perform replications. Replication can be performed only by a database user that has appropriate permissions and only for a configured number of hosts.
Although, by default, superusers have the necessary REPLICATION
and LOGIN
roles, as mentioned in Security, it is best not to provide the Debezium replication user with elevated privileges. Instead, create a Debezium user that has the the minimum required privileges.
Prerequisites
- PostgreSQL administrative permissions.
Procedure
To provide a user with replication permissions, define a PostgreSQL role that has at least the
REPLICATION
andLOGIN
permissions, and then grant that role to the user. For example:CREATE ROLE <name> REPLICATION LOGIN;
7.5.3. Setting privileges to enable Debezium to create PostgreSQL publications
Debezium streams change events for PostgreSQL source tables from publications that are created for the tables. Publications contain a filtered set of change events that are generated from one or more tables. The data in each publication is filtered based on the publication specification. The specification can be created by the PostgreSQL database administrator or by the Debezium connector. To permit the Debezium PostgreSQL connector to create publications and specify the data to replicate to them, the connector must operate with specific privileges in the database.
There are several options for determining how publications are created. In general, it is best to manually create publications for the tables that you want to capture, before you set up the connector. However, you can configure your environment in a way that permits Debezium to create publications automatically, and to specify the data that is added to them.
Debezium uses include list and exclude list properties to specify how data is inserted in the publication. For more information about the options for enabling Debezium to create publications, see publication.autocreate.mode
.
For Debezium to create a PostgreSQL publication, it must run as a user that has the following privileges:
- Replication privileges in the database to add the table to a publication.
-
CREATE
privileges on the database to add publications. -
SELECT
privileges on the tables to copy the initial table data. Table owners automatically haveSELECT
permission for the table.
To add tables to a publication, the user be an owner of the table. But because the source table already exists, you need a mechanism to share ownership with the original owner. To enable shared ownership, you create a PostgreSQL replication group, and then add the existing table owner and the replication user to the group.
Procedure
Create a replication group.
CREATE ROLE <replication_group>;
Add the original owner of the table to the group.
GRANT REPLICATION_GROUP TO <original_owner>;
Add the Debezium replication user to the group.
GRANT REPLICATION_GROUP TO <replication_user>;
Transfer ownership of the table to
<replication_group>
.ALTER TABLE <table_name> OWNER TO REPLICATION_GROUP;
For Debezium to specify the capture configuration, the value of publication.autocreate.mode
must be set to filtered
.
7.5.4. Configuring PostgreSQL to allow replication with the Debezium connector host
To enable Debezium to replicate PostgreSQL data, you must configure the database to permit replication with the host that runs the PostgreSQL connector. To specify the clients that are permitted to replicate with the database, add entries to the PostgreSQL host-based authentication file, pg_hba.conf
. For more information about the pg_hba.conf
file, see the PostgreSQL documentation.
Procedure
Add entries to the
pg_hba.conf
file to specify the Debezium connector hosts that can replicate with the database host. For example,pg_hba.conf
file example:local replication <youruser> trust 1 host replication <youruser> 127.0.0.1/32 trust 2 host replication <youruser> ::1/128 trust 3
- 1 1 1 1 1 1 1 1
- Instructs the server to allow replication for
<youruser>
locally, that is, on the server machine. - 2 2 2 2 2 2 2 2
- Instructs the server to allow
<youruser>
onlocalhost
to receive replication changes usingIPV4
. - 3 3 3 3 3 3 3 3
- Instructs the server to allow
<youruser>
onlocalhost
to receive replication changes usingIPV6
.
For more information about network masks, see the PostgreSQL documentation.
7.5.5. Configuring PostgreSQL to manage Debezium WAL disk space consumption
In certain cases, it is possible for PostgreSQL disk space consumed by WAL files to spike or increase out of usual proportions. There are several possible reasons for this situation:
The LSN up to which the connector has received data is available in the
confirmed_flush_lsn
column of the server’spg_replication_slots
view. Data that is older than this LSN is no longer available, and the database is responsible for reclaiming the disk space.Also in the
pg_replication_slots
view, therestart_lsn
column contains the LSN of the oldest WAL that the connector might require. If the value forconfirmed_flush_lsn
is regularly increasing and the value ofrestart_lsn
lags then the database needs to reclaim the space.The database typically reclaims disk space in batch blocks. This is expected behavior and no action by a user is necessary.
-
There are many updates in a database that is being tracked but only a tiny number of updates are related to the table(s) and schema(s) for which the connector is capturing changes. This situation can be easily solved with periodic heartbeat events. Set the
heartbeat.interval.ms
connector configuration property. The PostgreSQL instance contains multiple databases and one of them is a high-traffic database. Debezium captures changes in another database that is low-traffic in comparison to the other database. Debezium then cannot confirm the LSN as replication slots work per-database and Debezium is not invoked. As WAL is shared by all databases, the amount used tends to grow until an event is emitted by the database for which Debezium is capturing changes. To overcome this, it is necessary to:
-
Enable periodic heartbeat record generation with the
heartbeat.interval.ms
connector configuration property. - Regularly emit change events from the database for which Debezium is capturing changes.
A separate process would then periodically update the table by either inserting a new row or repeatedly updating the same row. PostgreSQL then invokes Debezium, which confirms the latest LSN and allows the database to reclaim the WAL space. This task can be automated by means of the
heartbeat.action.query
connector configuration property.-
Enable periodic heartbeat record generation with the
7.6. Deployment of Debezium PostgreSQL connectors
To deploy a Debezium PostgreSQL connector, add the connector files to Kafka Connect, create a custom container to run the connector, and add connector configuration to your container. Details are in the following topics:
7.6.1. Deploying Debezium PostgreSQL connectors
To deploy a Debezium PostgreSQL connector, you need to build a custom Kafka Connect container image that contains the Debezium connector archive and push this container image to a container registry. You then need to create two custom resources (CRs):
-
A
KafkaConnect
CR that defines your Kafka Connect instance. Theimage
property in the CR specifies the name of the container image that you create to run your Debezium connector. You apply this CR to the OpenShift instance where Red Hat AMQ Streams is deployed. AMQ Streams offers operators and images that bring Apache Kafka to OpenShift. -
A
KafkaConnector
CR that defines your Debezium Db2 connector. Apply this CR to the same OpenShift instance where you applied theKafkaConnect
CR.
Prerequisites
- PostgreSQL is running and you performed the steps to set up PostgreSQL to run a Debezium connector.
- AMQ Streams is deployed on OpenShift and is running Apache Kafka and Kafka Connect. For more information, see Deploying and Upgrading AMQ Streams on OpenShift.
- Podman or Docker is installed.
-
You have an account and permissions to create and manage containers in the container registry (such as
quay.io
ordocker.io
) to which you plan to add the container that will run your Debezium connector.
Procedure
Create the Debezium PostgreSQL container for Kafka Connect:
- Download the Debezium PostgreSQL connector archive.
Extract the Debezium PostgreSQL connector archive to create a directory structure for the connector plug-in, for example:
./my-plugins/ ├── debezium-connector-postgresql │ ├── ...
Create a Docker file that uses
registry.redhat.io/amq7/amq-streams-kafka-28-rhel8:1.8.0
as the base image. For example, from a terminal window, enter the following, replacingmy-plugins
with the name of your plug-ins directory:cat <<EOF >debezium-container-for-postgresql.yaml 1 FROM registry.redhat.io/amq7/amq-streams-kafka-28-rhel8:1.8.0 USER root:root COPY ./<my-plugins>/ /opt/kafka/plugins/ 2 USER 1001 EOF
The command creates a Docker file with the name
debezium-container-for-postgresql.yaml
in the current directory.Build the container image from the
debezium-container-for-postgresql.yaml
Docker file that you created in the previous step. From the directory that contains the file, open a terminal window and enter one of the following commands:podman build -t debezium-container-for-postgresql:latest .
docker build -t debezium-container-for-postgresql:latest .
The
build
command builds a container image with the namedebezium-container-for-postgresql
.Push your custom image to a container registry such as
quay.io
or an internal container registry. The container registry must be available to the OpenShift instance where you want to deploy the image. Enter one of the following commands:podman push <myregistry.io>/debezium-container-for-postgresql:latest
docker push <myregistry.io>/debezium-container-for-postgresql:latest
Create a new Debezium PostgreSQL
KafkaConnect
custom resource (CR). For example, create aKafkaConnect
CR with the namedbz-connect.yaml
that specifiesannotations
andimage
properties as shown in the following example:apiVersion: kafka.strimzi.io/v1beta2 kind: KafkaConnect metadata: name: my-connect-cluster annotations: strimzi.io/use-connector-resources: "true" 1 spec: image: debezium-container-for-postgresql 2
- 1
metadata.annotations
indicates to the Cluster Operator thatKafkaConnector
resources are used to configure connectors in this Kafka Connect cluster.- 2
spec.image
specifies the name of the image that you created to run your Debezium connector. This property overrides theSTRIMZI_DEFAULT_KAFKA_CONNECT_IMAGE
variable in the Cluster Operator.
Apply your
KafkaConnect
CR to the OpenShift Kafka instance by running the following command:oc create -f dbz-connect.yaml
This updates your Kafka Connect environment in OpenShift to add a Kafka Connector instance that specifies the name of the image that you created to run your Debezium connector.
Create a
KafkaConnector
custom resource that configures your Debezium PostgreSQL connector instance.You configure a Debezium PostgreSQL connector in a
.yaml
file that specifies the configuration properties for the connector. The connector configuration might instruct Debezium to produce events for a subset of the schemas and tables, or it might set properties so that Debezium ignores, masks, or truncates values in specified columns that are sensitive, too large, or not needed. For the complete list of the configuration properties that you can set for the Debezium PostgreSQL connector, see PostgreSQL connector properties.The following example configures a Debezium connector that connects to a PostgreSQL server host,
192.168.99.100
, on port5432
. This host has a database namedsampledb
, a schema namedpublic
, andfulfillment
is the server’s logical name.fulfillment-connector.yaml
apiVersion: kafka.strimzi.io/v1beta2 kind: KafkaConnector metadata: name: fulfillment-connector 1 labels: strimzi.io/cluster: my-connect-cluster spec: class: io.debezium.connector.postgresql.PostgresConnector tasksMax: 1 2 config: 3 database.hostname: 192.168.99.100 4 database.port: 5432 database.user: debezium database.password: dbz database.dbname: sampledb database.server.name: fulfillment 5 schema.include.list: public 6 plugin.name: pgoutput 7
- 1
- The name of the connector.
- 2
- Only one task should operate at any one time. Because the PostgreSQL connector reads the PostgreSQL server’s
binlog
, using a single connector task ensures proper order and event handling. The Kafka Connect service uses connectors to start one or more tasks that do the work, and it automatically distributes the running tasks across the cluster of Kafka Connect services. If any of the services stop or crash, those tasks will be redistributed to running services. - 3
- The connector’s configuration.
- 4
- The name of the database host that is running the PostgreSQL server. In this example, the database host name is
192.168.99.100
. - 5
- A unique server name. The server name is the logical identifier for the PostgreSQL server or cluster of servers. This name is used as the prefix for all Kafka topics that receive change event records.
- 6
- The connector captures changes in only the
public
schema. It is possible to configure the connector to capture changes in only the tables that you choose. Seetable.include.list
connector configuration property. - 7
- The name of the PostgreSQL logical decoding plug-in installed on the PostgreSQL server. While the only supported value for PostgreSQL 10 and later is
pgoutput
, you must explicitly setplugin.name
topgoutput
.
Create your connector instance with Kafka Connect. For example, if you saved your
KafkaConnector
resource in thefulfillment-connector.yaml
file, you would run the following command:oc apply -f fulfillment-connector.yaml
This registers
fulfillment-connector
and the connector starts to run against thesampledb
database as defined in theKafkaConnector
CR.Verify that the connector was created and has started:
Display the Kafka Connect log output to verify that the connector was created and has started to capture changes in the specified database:
oc logs $(oc get pods -o name -l strimzi.io/cluster=my-connect-cluster)
Review the log output to verify that Debezium performs the initial snapshot. The log displays output that is similar to the following messages:
... INFO Starting snapshot for ... ... INFO Snapshot is using user 'debezium' ...
If the connector starts correctly without errors, it creates a topic for each table whose changes the connector is capturing. For the example CR, there would be a topic for each table in the
public
schema. Downstream applications can subscribe to these topics.Verify that the connector created the topics by running the following command:
oc get kafkatopics
Results
When the connector starts, it performs a consistent snapshot of the PostgreSQL server databases that the connector is configured for. The connector then starts generating data change events for row-level operations and streaming change event records to Kafka topics.
7.6.2. Description of Debezium PostgreSQL connector configuration properties
The Debezium PostgreSQL connector has many configuration properties that you can use to achieve the right connector behavior for your application. Many properties have default values. Information about the properties is organized as follows:
The following configuration properties are required unless a default value is available.
Property | Default | Description |
---|---|---|
Unique name for the connector. Attempting to register again with the same name will fail. This property is required by all Kafka Connect connectors. | ||
The name of the Java class for the connector. Always use a value of | ||
| The maximum number of tasks that should be created for this connector. The PostgreSQL connector always uses a single task and therefore does not use this value, so the default is always acceptable. | |
| The name of the PostgreSQL logical decoding plug-in installed on the PostgreSQL server.
The only supported value is | |
| The name of the PostgreSQL logical decoding slot that was created for streaming changes from a particular plug-in for a particular database/schema. The server uses this slot to stream events to the Debezium connector that you are configuring. Slot names must conform to PostgreSQL replication slot naming rules, which state: "Each replication slot has a name, which can contain lower-case letters, numbers, and the underscore character." | |
| Whether or not to delete the logical replication slot when the connector stops in a graceful, expected way. The default behavior is that the replication slot remains configured for the connector when the connector stops. When the connector restarts, having the same replication slot enables the connector to start processing where it left off.
Set to | |
|
The name of the PostgreSQL publication created for streaming changes when using This publication is created at start-up if it does not already exist and it includes all tables. Debezium then applies its own include/exclude list filtering, if configured, to limit the publication to change events for the specific tables of interest. The connector user must have superuser permissions to create this publication, so it is usually preferable to create the publication before starting the connector for the first time. If the publication already exists, either for all tables or configured with a subset of tables, Debezium uses the publication as it is defined. | |
IP address or hostname of the PostgreSQL database server. | ||
| Integer port number of the PostgreSQL database server. | |
Name of the PostgreSQL database user for connecting to the PostgreSQL database server. | ||
Password to use when connecting to the PostgreSQL database server. | ||
The name of the PostgreSQL database from which to stream the changes. | ||
Logical name that identifies and provides a namespace for the particular PostgreSQL database server or cluster in which Debezium is capturing changes. Only alphanumeric characters and underscores should be used in the database server logical name. The logical name should be unique across all other connectors, since it is used as a topic name prefix for all Kafka topics that receive records from this connector. | ||
An optional, comma-separated list of regular expressions that match names of schemas for which you want to capture changes. Any schema name not included in | ||
An optional, comma-separated list of regular expressions that match names of schemas for which you do not want to capture changes. Any schema whose name is not included in | ||
An optional, comma-separated list of regular expressions that match fully-qualified table identifiers for tables whose changes you want to capture. Any table not included in | ||
An optional, comma-separated list of regular expressions that match fully-qualified table identifiers for tables whose changes you do not want to capture. Any table not included in | ||
An optional, comma-separated list of regular expressions that match the fully-qualified names of columns that should be included in change event record values. Fully-qualified names for columns are of the form schemaName.tableName.columnName. Do not also set the | ||
An optional, comma-separated list of regular expressions that match the fully-qualified names of columns that should be excluded from change event record values. Fully-qualified names for columns are of the form schemaName.tableName.columnName. Do not also set the | ||
|
Time, date, and timestamps can be represented with different kinds of precision: | |
|
Specifies how the connector should handle values for | |
|
Specifies how the connector should handle values for | |
|
Specifies how the connector should handle values for | |
|
Whether to use an encrypted connection to the PostgreSQL server. Options include: | |
The path to the file that contains the SSL certificate for the client. See the PostgreSQL documentation for more information. | ||
The path to the file that contains the SSL private key of the client. See the PostgreSQL documentation for more information. | ||
The password to access the client private key from the file specified by | ||
The path to the file that contains the root certificate(s) against which the server is validated. See the PostgreSQL documentation for more information. | ||
| Enable TCP keep-alive probe to verify that the database connection is still alive. See the PostgreSQL documentation for more information. | |
|
Controls whether a delete event is followed by a tombstone event. | |
n/a |
An optional, comma-separated list of regular expressions that match the fully-qualified names of character-based columns. Fully-qualified names for columns are of the form schemaName.tableName.columnName. In change event records, values in these columns are truncated if they are longer than the number of characters specified by length in the property name. You can specify multiple properties with different lengths in a single configuration. Length must be a positive integer, for example, | |
n/a |
An optional, comma-separated list of regular expressions that match the fully-qualified names of character-based columns. Fully-qualified names for columns are of the form schemaName.tableName.columnName. In change event values, the values in the specified table columns are replaced with length number of asterisk ( | |
n/a |
An optional, comma-separated list of regular expressions that match the fully-qualified names of character-based columns. Fully-qualified names for columns are of the form schemaName.tableName.columnName. In the resulting change event record, the values for the specified columns are replaced with pseudonyms.
A pseudonym consists of the hashed value that results from applying the specified hashAlgorithm and salt. Based on the hash function that is used, referential integrity is maintained, while column values are replaced with pseudonyms. Supported hash functions are described in the MessageDigest section of the Java Cryptography Architecture Standard Algorithm Name Documentation. column.mask.hash.SHA-256.with.salt.CzQMA0cB5K = inventory.orders.customerName, inventory.shipment.customerName
If necessary, the pseudonym is automatically shortened to the length of the column. The connector configuration can include multiple properties that specify different hash algorithms and salts. | |
n/a |
An optional, comma-separated list of regular expressions that match the fully-qualified names of columns. Fully-qualified names for columns are of the form databaseName.tableName.columnName, or databaseName.schemaName.tableName.columnName. | |
n/a |
An optional, comma-separated list of regular expressions that match the database-specific data type name for some columns. Fully-qualified data type names are of the form databaseName.tableName.typeName, or databaseName.schemaName.tableName.typeName. | |
empty string |
A semicolon separated list of tables with regular expressions that match table column names. The connector maps values in matching columns to key fields in change event records that it sends to Kafka topics. This is useful when a table does not have a primary key, or when you want to order change event records in a Kafka topic according to a field that is not a primary key. | |
all_tables |
Applies only when streaming changes by using the | |
bytes |
Specifies how binary ( | |
bytes |
Specifies how whether |
The following advanced configuration properties have defaults that work in most situations and therefore rarely need to be specified in the connector’s configuration.
Property | Default | Description |
---|---|---|
|
Specifies the criteria for performing a snapshot when the connector starts: | |
All tables specified in |
An optional, comma-separated list of regular expressions that match names of schemas specified in | |
| Positive integer value that specifies the maximum amount of time (in milliseconds) to wait to obtain table locks when performing a snapshot. If the connector cannot acquire table locks in this time interval, the snapshot fails. How the connector performs snapshots provides details. | |
Controls which table rows are included in snapshots. This property affects snapshots only. It does not affect events that are generated by the logical decoding plug-in. Specify a comma-separated list of fully-qualified table names in the form databaseName.tableName. | ||
|
Specifies how the connector should react to exceptions during processing of events: | |
| Positive integer value for the maximum size of the blocking queue. The connector places change events received from streaming replication in the blocking queue before writing them to Kafka. This queue can provide backpressure when, for example, writing records to Kafka is slower that it should be or Kafka is not available. | |
| Positive integer value that specifies the maximum size of each batch of events that the connector processes. | |
| Long value for the maximum size in bytes of the blocking queue. The feature is disabled by default, it will be active if it’s set with a positive long value. | |
| Positive integer value that specifies the number of milliseconds the connector should wait for new change events to appear before it starts processing a batch of events. Defaults to 1000 milliseconds, or 1 second. | |
|
Specifies connector behavior when the connector encounters a field whose data type is unknown. The default behavior is that the connector omits the field from the change event and logs a warning. Note
Consumers risk backward compatibility issues when | |
A semicolon separated list of SQL statements that the connector executes when it establishes a JDBC connection to the database. To use a semicolon as a character and not as a delimiter, specify two consecutive semicolons, | ||
|
Controls how frequently the connector sends heartbeat messages to a Kafka topic. The default behavior is that the connector does not send heartbeat messages. | |
|
Controls the name of the topic to which the connector sends heartbeat messages. The topic name has this pattern: | |
Specifies a query that the connector executes on the source database when the connector sends a heartbeat message. | ||
|
Specify the conditions that trigger a refresh of the in-memory schema for a table. | |
An interval in milliseconds that the connector should wait before performing a snapshot when the connector starts. If you are starting multiple connectors in a cluster, this property is useful for avoiding snapshot interruptions, which might cause re-balancing of connectors. | ||
| During a snapshot, the connector reads table content in batches of rows. This property specifies the maximum number of rows in a batch. | |
Semicolon separated list of parameters to pass to the configured logical decoding plug-in. For example, | ||
| Indicates whether field names are sanitized to adhere to Avro naming requirements. | |
| If connecting to a replication slot fails, this is the maximum number of consecutive attempts to connect. | |
| The number of milliseconds to wait between retry attempts when the connector fails to connect to a replication slot. | |
|
Specifies the constant that the connector provides to indicate that the original value is a toasted value that is not provided by the database. If the setting of | |
|
Determines whether the connector generates events with transaction boundaries and enriches change event envelopes with transaction metadata. Specify | |
10000 (10 seconds) | The number of milliseconds to wait before restarting a connector after a retriable error occurs. |
Pass-through connector configuration properties
The connector also supports pass-through configuration properties that are used when creating the Kafka producer and consumer.
Be sure to consult the Kafka documentation for all of the configuration properties for Kafka producers and consumers. The PostgreSQL connector does use the new consumer configuration properties.
7.7. Monitoring Debezium PostgreSQL connector performance
The Debezium PostgreSQL connector provides two types of metrics that are in addition to the built-in support for JMX metrics that Zookeeper, Kafka, and Kafka Connect provide.
- Snapshot metrics provide information about connector operation while performing a snapshot.
- Streaming metrics provide information about connector operation when the connector is capturing changes and streaming change event records.
Debezium monitoring documentation provides details for how to expose these metrics by using JMX.
7.7.1. Monitoring Debezium during snapshots of PostgreSQL databases
The MBean is debezium.postgres:type=connector-metrics,context=snapshot,server=<database.server.name>
.
Attributes | Type | Description |
---|---|---|
| The last snapshot event that the connector has read. | |
| The number of milliseconds since the connector has read and processed the most recent event. | |
| The total number of events that this connector has seen since last started or reset. | |
| The number of events that have been filtered by include/exclude list filtering rules configured on the connector. | |
| The list of tables that are monitored by the connector. | |
| The length the queue used to pass events between the snapshotter and the main Kafka Connect loop. | |
| The free capacity of the queue used to pass events between the snapshotter and the main Kafka Connect loop. | |
| The total number of tables that are being included in the snapshot. | |
| The number of tables that the snapshot has yet to copy. | |
| Whether the snapshot was started. | |
| Whether the snapshot was aborted. | |
| Whether the snapshot completed. | |
| The total number of seconds that the snapshot has taken so far, even if not complete. | |
| Map containing the number of rows scanned for each table in the snapshot. Tables are incrementally added to the Map during processing. Updates every 10,000 rows scanned and upon completing a table. | |
|
The maximum buffer of the queue in bytes. It will be enabled if | |
| The current data of records in the queue in bytes. |
7.7.2. Monitoring Debezium PostgreSQL connector record streaming
The MBean is debezium.postgres:type=connector-metrics,context=streaming,server=<database.server.name>
.
Attributes | Type | Description |
---|---|---|
| The last streaming event that the connector has read. | |
| The number of milliseconds since the connector has read and processed the most recent event. | |
| The total number of events that this connector has seen since last started or reset. | |
| The number of events that have been filtered by include/exclude list filtering rules configured on the connector. | |
| The list of tables that are monitored by the connector. | |
| The length the queue used to pass events between the streamer and the main Kafka Connect loop. | |
| The free capacity of the queue used to pass events between the streamer and the main Kafka Connect loop. | |
| Flag that denotes whether the connector is currently connected to the database server. | |
| The number of milliseconds between the last change event’s timestamp and the connector processing it. The values will incoporate any differences between the clocks on the machines where the database server and the connector are running. | |
| The number of processed transactions that were committed. | |
| The coordinates of the last received event. | |
| Transaction identifier of the last processed transaction. | |
| The maximum buffer of the queue in bytes. | |
| The current data of records in the queue in bytes. |
7.8. How Debezium PostgreSQL connectors handle faults and problems
Debezium is a distributed system that captures all changes in multiple upstream databases; it never misses or loses an event. When the system is operating normally or being managed carefully then Debezium provides exactly once delivery of every change event record.
If a fault does happen then the system does not lose any events. However, while it is recovering from the fault, it might repeat some change events. In these abnormal situations, Debezium, like Kafka, provides at least once delivery of change events.
Details are in the following sections:
Configuration and startup errors
In the following situations, the connector fails when trying to start, reports an error/exception in the log, and stops running:
- The connector’s configuration is invalid.
- The connector cannot successfully connect to PostgreSQL by using the specified connection parameters.
- The connector is restarting from a previously-recorded position in the PostgreSQL WAL (by using the LSN) and PostgreSQL no longer has that history available.
In these cases, the error message has details about the problem and possibly a suggested workaround. After you correct the configuration or address the PostgreSQL problem, restart the connector.
The PostgreSQL connector externally stores the last processed offset in the form of a PostgreSQL LSN. After a connector restarts and connects to a server instance, the connector communicates with the server to continue streaming from that particular offset. This offset is available as long as the Debezium replication slot remains intact. Never drop a replication slot on the primary server or you will lose data. See the next section for failure cases in which a slot has been removed.
Cluster failures
As of release 12, PostgreSQL allows logical replication slots only on primary servers. This means that you can point a Debezium PostgreSQL connector to only the active primary server of a database cluster. Also, replication slots themselves are not propagated to replicas. If the primary server goes down, a new primary must be promoted.
The new primary must have a replication slot that is configured for use by the pgoutput
plug-in and the database in which you want to capture changes. Only then can you point the connector to the new server and restart the connector.
There are important caveats when failovers occur and you should pause Debezium until you can verify that you have an intact replication slot that has not lost data. After a failover:
- There must be a process that re-creates the Debezium replication slot before allowing the application to write to the new primary. This is crucial. Without this process, your application can miss change events.
- You might need to verify that Debezium was able to read all changes in the slot before the old primary failed.
One reliable method of recovering and verifying whether any changes were lost is to recover a backup of the failed primary to the point immediately before it failed. While this can be administratively difficult, it allows you to inspect the replication slot for any unconsumed changes.
Kafka Connect process stops gracefully
Suppose that Kafka Connect is being run in distributed mode and a Kafka Connect process is stopped gracefully. Prior to shutting down that process, Kafka Connect migrates the process’s connector tasks to another Kafka Connect process in that group. The new connector tasks start processing exactly where the prior tasks stopped. There is a short delay in processing while the connector tasks are stopped gracefully and restarted on the new processes.
Kafka Connect process crashes
If the Kafka Connector process stops unexpectedly, any connector tasks it was running terminate without recording their most recently processed offsets. When Kafka Connect is being run in distributed mode, Kafka Connect restarts those connector tasks on other processes. However, PostgreSQL connectors resume from the last offset that was recorded by the earlier processes. This means that the new replacement tasks might generate some of the same change events that were processed just prior to the crash. The number of duplicate events depends on the offset flush period and the volume of data changes just before the crash.
Because there is a chance that some events might be duplicated during a recovery from failure, consumers should always anticipate some duplicate events. Debezium changes are idempotent, so a sequence of events always results in the same state.
In each change event record, Debezium connectors insert source-specific information about the origin of the event, including the PostgreSQL server’s time of the event, the ID of the server transaction, and the position in the write-ahead log where the transaction changes were written. Consumers can keep track of this information, especially the LSN, to determine whether an event is a duplicate.
Connector is stopped for a duration
If the connector is gracefully stopped, the database can continue to be used. Any changes are recorded in the PostgreSQL WAL. When the connector restarts, it resumes streaming changes where it left off. That is, it generates change event records for all database changes that were made while the connector was stopped.
A properly configured Kafka cluster is able to handle massive throughput. Kafka Connect is written according to Kafka best practices, and given enough resources a Kafka Connect connector can also handle very large numbers of database change events. Because of this, after being stopped for a while, when a Debezium connector restarts, it is very likely to catch up with the database changes that were made while it was stopped. How quickly this happens depends on the capabilities and performance of Kafka and the volume of changes being made to the data in PostgreSQL.