The Postgres CDC connector allows for reading snapshot data and incremental data from PostgreSQL database. This document describes how to setup the Postgres CDC connector to run SQL queries against PostgreSQL databases.
In order to setup the Postgres CDC connector, the following table provides dependency information for both projects using a build automation tool (such as Maven or SBT) and SQL Client with SQL JAR bundles.
<dependency>
<groupId>com.ververica</groupId>
<artifactId>flink-connector-postgres-cdc</artifactId>
<!-- the dependency is available only for stable releases. -->
<version>2.2-SNAPSHOT</version>
</dependency>
Download link is available only for stable releases.
Download flink-sql-connector-postgres-cdc-2.2-SNAPSHOT.jar and put it under <FLINK_HOME>/lib/
.
The Postgres CDC table can be defined as following:
-- register a PostgreSQL table 'shipments' in Flink SQL
CREATE TABLE shipments (
shipment_id INT,
order_id INT,
origin STRING,
destination STRING,
is_arrived BOOLEAN
) WITH (
'connector' = 'postgres-cdc',
'hostname' = 'localhost',
'port' = '5432',
'username' = 'postgres',
'password' = 'postgres',
'database-name' = 'postgres',
'schema-name' = 'public',
'table-name' = 'shipments'
);
-- read snapshot and binlogs from shipments table
SELECT * FROM shipments;
Option | Required | Default | Type | Description |
---|---|---|---|---|
connector | required | (none) | String | Specify what connector to use, here should be 'postgres-cdc' . |
hostname | required | (none) | String | IP address or hostname of the PostgreSQL database server. |
username | required | (none) | String | Name of the PostgreSQL database to use when connecting to the PostgreSQL database server. |
password | required | (none) | String | Password to use when connecting to the PostgreSQL database server. |
database-name | required | (none) | String | Database name of the PostgreSQL server to monitor. |
schema-name | required | (none) | String | Schema name of the PostgreSQL database to monitor. |
table-name | required | (none) | String | Table name of the PostgreSQL database to monitor. |
port | optional | 5432 | Integer | Integer port number of the PostgreSQL database server. |
decoding.plugin.name | optional | decoderbufs | String | The name of the Postgres logical decoding plug-in installed on the server. Supported values are decoderbufs, wal2json, wal2json_rds, wal2json_streaming, wal2json_rds_streaming and pgoutput. |
slot.name | optional | flink | String | 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 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." |
debezium.* | optional | (none) | String | Pass-through Debezium's properties to Debezium Embedded Engine which is used to capture data changes from Postgres server.
For example: 'debezium.snapshot.mode' = 'never' .
See more about the Debezium's Postgres Connector properties |
Note: slot.name
is recommended to set for different tables to avoid the potential PSQLException: ERROR: replication slot "flink" is active for PID 974
error. See more here.
The following format metadata can be exposed as read-only (VIRTUAL) columns in a table definition.
Key | DataType | Description |
---|---|---|
table_name | STRING NOT NULL | Name of the table that contain the row. |
schema_name | STRING NOT NULL | Name of the schema that contain the row. |
database_name | STRING NOT NULL | Name of the database that contain the row. |
op_ts | TIMESTAMP_LTZ(3) NOT NULL | It indicates the time that the change was made in the database. If the record is read from snapshot of the table instead of the change stream, the value is always 0. |
During scanning snapshot of database tables, since there is no recoverable position, we can't perform checkpoints. In order to not perform checkpoints, Postgres CDC source will keep the checkpoint waiting to timeout. The timeout checkpoint will be recognized as failed checkpoint, by default, this will trigger a failover for the Flink job. So if the database table is large, it is recommended to add following Flink configurations to avoid failover because of the timeout checkpoints:
execution.checkpointing.interval: 10min
execution.checkpointing.tolerable-failed-checkpoints: 100
restart-strategy: fixed-delay
restart-strategy.fixed-delay.attempts: 2147483647
The extended CREATE TABLE example demonstrates the syntax for exposing these metadata fields:
CREATE TABLE products (
db_name STRING METADATA FROM 'database_name' VIRTUAL,
table_name STRING METADATA FROM 'table_name' VIRTUAL,
operation_ts TIMESTAMP_LTZ(3) METADATA FROM 'op_ts' VIRTUAL,
shipment_id INT,
order_id INT,
origin STRING,
destination STRING,
is_arrived BOOLEAN
) WITH (
'connector' = 'postgres-cdc',
'hostname' = 'localhost',
'port' = '5432',
'username' = 'postgres',
'password' = 'postgres',
'database-name' = 'postgres',
'schema-name' = 'public',
'table-name' = 'shipments'
);
The Postgres CDC connector is a Flink Source connector which will read database snapshot first and then continues to read binlogs with exactly-once processing even failures happen. Please read How the connector works.
The Postgres CDC source can't work in parallel reading, because there is only one task can receive binlog events.
The Postgres CDC connector can also be a DataStream source. You can create a SourceFunction as the following shows:
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import com.ververica.cdc.debezium.JsonDebeziumDeserializationSchema;
import com.ververica.cdc.connectors.postgres.PostgreSQLSource;
public class PostgreSQLSourceExample {
public static void main(String[] args) throws Exception {
SourceFunction<String> sourceFunction = PostgreSQLSource.<String>builder()
.hostname("localhost")
.port(5432)
.database("postgres") // monitor postgres database
.schemaList("inventory") // monitor inventory schema
.tableList("inventory.products") // monitor products table
.username("flinkuser")
.password("flinkpw")
.deserializer(new JsonDebeziumDeserializationSchema()) // converts SourceRecord to JSON String
.build();
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env
.addSource(sourceFunction)
.print().setParallelism(1); // use parallelism 1 for sink to keep message ordering
env.execute();
}
}
Note: Please refer Deserialization for more details about the JSON deserialization.