You cannot select more than 25 topics
Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
294 lines
9.0 KiB
Markdown
294 lines
9.0 KiB
Markdown
4 years ago
|
# Postgres CDC Connector
|
||
|
|
||
|
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.
|
||
|
|
||
|
Dependencies
|
||
|
------------
|
||
|
|
||
|
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.
|
||
|
|
||
|
### Maven dependency
|
||
|
|
||
|
```
|
||
|
<dependency>
|
||
|
<groupId>com.alibaba.ververica</groupId>
|
||
|
<artifactId>flink-connector-postgres-cdc</artifactId>
|
||
4 years ago
|
<version>1.4.0</version>
|
||
4 years ago
|
</dependency>
|
||
|
```
|
||
|
|
||
|
### SQL Client JAR
|
||
|
|
||
4 years ago
|
Download [flink-sql-connector-postgres-cdc-1.4.0.jar](https://repo1.maven.org/maven2/com/alibaba/ververica/flink-sql-connector-postgres-cdc/1.4.0/flink-sql-connector-postgres-cdc-1.4.0.jar) and put it under `<FLINK_HOME>/lib/`.
|
||
4 years ago
|
|
||
|
How to create a Postgres CDC table
|
||
|
----------------
|
||
|
|
||
|
The Postgres CDC table can be defined as following:
|
||
|
|
||
|
```sql
|
||
|
-- 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;
|
||
|
```
|
||
|
|
||
|
Connector Options
|
||
|
----------------
|
||
|
|
||
4 years ago
|
<div class="highlight">
|
||
|
<table class="colwidths-auto docutils">
|
||
|
<thead>
|
||
4 years ago
|
<tr>
|
||
|
<th class="text-left" style="width: 25%">Option</th>
|
||
|
<th class="text-left" style="width: 8%">Required</th>
|
||
|
<th class="text-left" style="width: 7%">Default</th>
|
||
|
<th class="text-left" style="width: 10%">Type</th>
|
||
|
<th class="text-left" style="width: 50%">Description</th>
|
||
|
</tr>
|
||
|
</thead>
|
||
|
<tbody>
|
||
|
<tr>
|
||
4 years ago
|
<td>connector</td>
|
||
4 years ago
|
<td>required</td>
|
||
|
<td style="word-wrap: break-word;">(none)</td>
|
||
|
<td>String</td>
|
||
|
<td>Specify what connector to use, here should be <code>'postgres-cdc'</code>.</td>
|
||
|
</tr>
|
||
|
<tr>
|
||
4 years ago
|
<td>hostname</td>
|
||
4 years ago
|
<td>required</td>
|
||
|
<td style="word-wrap: break-word;">(none)</td>
|
||
|
<td>String</td>
|
||
|
<td>IP address or hostname of the PostgreSQL database server.</td>
|
||
|
</tr>
|
||
|
<tr>
|
||
4 years ago
|
<td>username</td>
|
||
4 years ago
|
<td>required</td>
|
||
|
<td style="word-wrap: break-word;">(none)</td>
|
||
|
<td>String</td>
|
||
|
<td>Name of the PostgreSQL database to use when connecting to the PostgreSQL database server.</td>
|
||
|
</tr>
|
||
|
<tr>
|
||
4 years ago
|
<td>password</td>
|
||
4 years ago
|
<td>required</td>
|
||
|
<td style="word-wrap: break-word;">(none)</td>
|
||
|
<td>String</td>
|
||
|
<td>Password to use when connecting to the PostgreSQL database server.</td>
|
||
|
</tr>
|
||
|
<tr>
|
||
4 years ago
|
<td>database-name</td>
|
||
4 years ago
|
<td>required</td>
|
||
|
<td style="word-wrap: break-word;">(none)</td>
|
||
|
<td>String</td>
|
||
|
<td>Database name of the PostgreSQL server to monitor.</td>
|
||
|
</tr>
|
||
|
<tr>
|
||
4 years ago
|
<td>schema-name</td>
|
||
4 years ago
|
<td>required</td>
|
||
|
<td style="word-wrap: break-word;">(none)</td>
|
||
|
<td>String</td>
|
||
|
<td>Schema name of the PostgreSQL database to monitor.</td>
|
||
|
</tr>
|
||
|
<tr>
|
||
4 years ago
|
<td>table-name</td>
|
||
4 years ago
|
<td>required</td>
|
||
|
<td style="word-wrap: break-word;">(none)</td>
|
||
|
<td>String</td>
|
||
|
<td>Table name of the PostgreSQL database to monitor.</td>
|
||
|
</tr>
|
||
|
<tr>
|
||
4 years ago
|
<td>port</td>
|
||
4 years ago
|
<td>optional</td>
|
||
|
<td style="word-wrap: break-word;">5432</td>
|
||
|
<td>Integer</td>
|
||
|
<td>Integer port number of the PostgreSQL database server.</td>
|
||
|
</tr>
|
||
|
<tr>
|
||
4 years ago
|
<td>decoding.plugin.name</td>
|
||
4 years ago
|
<td>optional</td>
|
||
|
<td style="word-wrap: break-word;">decoderbufs</td>
|
||
|
<td>String</td>
|
||
|
<td>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.</td>
|
||
|
</tr>
|
||
|
<tr>
|
||
4 years ago
|
<td>slot.name</td>
|
||
4 years ago
|
<td>optional</td>
|
||
|
<td style="word-wrap: break-word;">flink</td>
|
||
|
<td>String</td>
|
||
|
<td>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.
|
||
|
<br/>Slot names must conform to <a href="https://www.postgresql.org/docs/current/static/warm-standby.html#STREAMING-REPLICATION-SLOTS-MANIPULATION">PostgreSQL replication slot naming rules</a>, which state: "Each replication slot has a name, which can contain lower-case letters, numbers, and the underscore character."</td>
|
||
|
</tr>
|
||
|
<tr>
|
||
4 years ago
|
<td>debezium.*</td>
|
||
4 years ago
|
<td>optional</td>
|
||
|
<td style="word-wrap: break-word;">(none)</td>
|
||
|
<td>String</td>
|
||
|
<td>Pass-through Debezium's properties to Debezium Embedded Engine which is used to capture data changes from Postgres server.
|
||
|
For example: <code>'debezium.snapshot.mode' = 'never'</code>.
|
||
|
See more about the <a href="https://debezium.io/documentation/reference/1.2/connectors/postgresql.html#postgresql-connector-properties">Debezium's Postgres Connector properties</a></td>
|
||
|
</tr>
|
||
|
</tbody>
|
||
4 years ago
|
</table>
|
||
|
</div>
|
||
4 years ago
|
|
||
|
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](https://debezium.io/documentation/reference/1.2/connectors/postgresql.html#postgresql-property-slot-name).
|
||
|
|
||
|
Features
|
||
|
--------
|
||
|
|
||
|
### Exactly-Once Processing
|
||
|
|
||
|
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](https://debezium.io/documentation/reference/1.2/connectors/postgresql.html#how-the-postgresql-connector-works).
|
||
|
|
||
|
### Single Thread Reading
|
||
|
|
||
|
The Postgres CDC source can't work in parallel reading, because there is only one task can receive binlog events.
|
||
|
|
||
|
### DataStream Source
|
||
|
|
||
|
The Postgres CDC connector can also be a DataStream source. You can create a SourceFunction as the following shows:
|
||
|
|
||
|
```java
|
||
|
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
|
||
|
import org.apache.flink.streaming.api.functions.source.SourceFunction;
|
||
|
import com.alibaba.ververica.cdc.debezium.StringDebeziumDeserializationSchema;
|
||
|
import com.alibaba.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")
|
||
|
.schemaList("inventory") // monitor all tables under inventory schema
|
||
|
.username("flinkuser")
|
||
|
.password("flinkpw")
|
||
|
.deserializer(new StringDebeziumDeserializationSchema()) // converts SourceRecord to String
|
||
|
.build();
|
||
|
|
||
|
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
|
||
|
|
||
|
env
|
||
|
.addSource(sourceFunction)
|
||
|
.print().setParallelism(1); // use parallelism 1 for sink to keep message ordering
|
||
|
|
||
|
env.execute();
|
||
|
}
|
||
|
}
|
||
|
```
|
||
|
|
||
|
|
||
|
Data Type Mapping
|
||
|
----------------
|
||
|
|
||
4 years ago
|
<div class="wy-table-responsive">
|
||
|
<table class="colwidths-auto docutils">
|
||
4 years ago
|
<thead>
|
||
|
<tr>
|
||
|
<th class="text-left">PostgreSQL type<a href="https://www.postgresql.org/docs/12/datatype.html"></a></th>
|
||
|
<th class="text-left">Flink SQL type<a href="{% link dev/table/types.md %}"></a></th>
|
||
|
</tr>
|
||
|
</thead>
|
||
|
<tbody>
|
||
|
<tr>
|
||
|
<td></td>
|
||
4 years ago
|
<td>TINYINT</td>
|
||
4 years ago
|
</tr>
|
||
|
<tr>
|
||
|
<td>
|
||
4 years ago
|
SMALLINT<br>
|
||
|
INT2<br>
|
||
|
SMALLSERIAL<br>
|
||
|
SERIAL2</td>
|
||
|
<td>SMALLINT</td>
|
||
4 years ago
|
</tr>
|
||
|
<tr>
|
||
|
<td>
|
||
4 years ago
|
INTEGER<br>
|
||
|
SERIAL</td>
|
||
|
<td>INT</td>
|
||
4 years ago
|
</tr>
|
||
|
<tr>
|
||
|
<td>
|
||
4 years ago
|
BIGINT<br>
|
||
|
BIGSERIAL</td>
|
||
|
<td>BIGINT</td>
|
||
4 years ago
|
</tr>
|
||
|
<tr>
|
||
|
<td></td>
|
||
4 years ago
|
<td>DECIMAL(20, 0)</td>
|
||
4 years ago
|
</tr>
|
||
|
<tr>
|
||
4 years ago
|
<td>BIGINT</td>
|
||
|
<td>BIGINT</td>
|
||
4 years ago
|
</tr>
|
||
|
<tr>
|
||
|
<td>
|
||
4 years ago
|
REAL<br>
|
||
|
FLOAT4</td>
|
||
|
<td>FLOAT</td>
|
||
4 years ago
|
</tr>
|
||
|
<tr>
|
||
|
<td>
|
||
4 years ago
|
FLOAT8<br>
|
||
|
DOUBLE PRECISION</td>
|
||
|
<td>DOUBLE</td>
|
||
4 years ago
|
</tr>
|
||
|
<tr>
|
||
|
<td>
|
||
4 years ago
|
NUMERIC(p, s)<br>
|
||
|
DECIMAL(p, s)</td>
|
||
|
<td>DECIMAL(p, s)</td>
|
||
4 years ago
|
</tr>
|
||
|
<tr>
|
||
4 years ago
|
<td>BOOLEAN</td>
|
||
|
<td>BOOLEAN</td>
|
||
4 years ago
|
</tr>
|
||
|
<tr>
|
||
4 years ago
|
<td>DATE</td>
|
||
|
<td>DATE</td>
|
||
4 years ago
|
</tr>
|
||
|
<tr>
|
||
4 years ago
|
<td>TIME [(p)] [WITHOUT TIMEZONE]</td>
|
||
|
<td>TIME [(p)] [WITHOUT TIMEZONE]</td>
|
||
4 years ago
|
</tr>
|
||
|
<tr>
|
||
4 years ago
|
<td>TIMESTAMP [(p)] [WITHOUT TIMEZONE]</td>
|
||
|
<td>TIMESTAMP [(p)] [WITHOUT TIMEZONE]</td>
|
||
4 years ago
|
</tr>
|
||
|
<tr>
|
||
|
<td>
|
||
4 years ago
|
CHAR(n)<br>
|
||
|
CHARACTER(n)<br>
|
||
|
VARCHAR(n)<br>
|
||
|
CHARACTER VARYING(n)<br>
|
||
|
TEXT</td>
|
||
|
<td>STRING</td>
|
||
4 years ago
|
</tr>
|
||
|
<tr>
|
||
4 years ago
|
<td>BYTEA</td>
|
||
|
<td>BYTES</td>
|
||
4 years ago
|
</tr>
|
||
|
</tbody>
|
||
|
</table>
|
||
4 years ago
|
</div>
|